[sample data] handle index aliases when installing/uninstalling datasets (#122689)

* refactor install/uninstall routes

* only skip failing tests

* check for aliases when uninstalling sample datasets

* fix return value

* add unit tests

* factorize installer creation

* add tests for the alias scenario

Co-authored-by: Kibana Machine <42973632+kibanamachine@users.noreply.github.com>
This commit is contained in:
Pierre Gayvallet 2022-01-17 10:27:52 +01:00 committed by GitHub
parent b0f4e88f3d
commit 683ab102ae
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
10 changed files with 741 additions and 210 deletions

View file

@ -0,0 +1,13 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
export class SampleDataInstallError extends Error {
constructor(message: string, public readonly httpCode: number) {
super(message);
}
}

View file

@ -0,0 +1,74 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
import { IScopedClusterClient, Logger } from 'kibana/server';
import type { DataIndexSchema } from './sample_dataset_registry_types';
import {
translateTimeRelativeToDifference,
translateTimeRelativeToWeek,
} from './translate_timestamp';
import { loadData } from './load_data';
export const insertDataIntoIndex = ({
dataIndexConfig,
logger,
esClient,
index,
nowReference,
}: {
dataIndexConfig: DataIndexSchema;
index: string;
nowReference: string;
esClient: IScopedClusterClient;
logger: Logger;
}) => {
const updateTimestamps = (doc: any) => {
dataIndexConfig.timeFields
.filter((timeFieldName: string) => doc[timeFieldName])
.forEach((timeFieldName: string) => {
doc[timeFieldName] = dataIndexConfig.preserveDayOfWeekTimeOfDay
? translateTimeRelativeToWeek(
doc[timeFieldName],
dataIndexConfig.currentTimeMarker,
nowReference
)
: translateTimeRelativeToDifference(
doc[timeFieldName],
dataIndexConfig.currentTimeMarker,
nowReference
);
});
return doc;
};
const bulkInsert = async (docs: unknown[]) => {
const insertCmd = { index: { _index: index } };
const bulk: unknown[] = [];
docs.forEach((doc: unknown) => {
bulk.push(insertCmd);
bulk.push(updateTimestamps(doc));
});
const { body: resp } = await esClient.asCurrentUser.bulk({
body: bulk,
});
if (resp.errors) {
const errMsg = `sample_data install errors while bulk inserting. Elasticsearch response: ${JSON.stringify(
resp,
null,
''
)}`;
logger.warn(errMsg);
return Promise.reject(
new Error(`Unable to load sample data into index "${index}", see kibana logs for details`)
);
}
};
return loadData(dataIndexConfig.dataPath, bulkInsert); // this returns a Promise
};

View file

@ -12,7 +12,10 @@ import { createUnzip } from 'zlib';
const BULK_INSERT_SIZE = 500;
export function loadData(path: any, bulkInsert: (docs: any[]) => Promise<void>) {
export function loadData(
path: string,
bulkInsert: (docs: unknown[]) => Promise<void>
): Promise<number> {
return new Promise((resolve, reject) => {
let count: number = 0;
let docs: any[] = [];

View file

@ -6,73 +6,12 @@
* Side Public License, v 1.
*/
import { Readable } from 'stream';
import { schema } from '@kbn/config-schema';
import { IRouter, Logger, IScopedClusterClient } from 'src/core/server';
import { IRouter, Logger } from 'src/core/server';
import { SampleDatasetSchema } from '../lib/sample_dataset_registry_types';
import { createIndexName } from '../lib/create_index_name';
import {
dateToIso8601IgnoringTime,
translateTimeRelativeToDifference,
translateTimeRelativeToWeek,
} from '../lib/translate_timestamp';
import { loadData } from '../lib/load_data';
import { SampleDataUsageTracker } from '../usage/usage';
import { getSavedObjectsClient } from './utils';
import { getUniqueObjectTypes } from '../lib/utils';
const insertDataIntoIndex = (
dataIndexConfig: any,
index: string,
nowReference: string,
esClient: IScopedClusterClient,
logger: Logger
) => {
function updateTimestamps(doc: any) {
dataIndexConfig.timeFields
.filter((timeFieldName: string) => doc[timeFieldName])
.forEach((timeFieldName: string) => {
doc[timeFieldName] = dataIndexConfig.preserveDayOfWeekTimeOfDay
? translateTimeRelativeToWeek(
doc[timeFieldName],
dataIndexConfig.currentTimeMarker,
nowReference
)
: translateTimeRelativeToDifference(
doc[timeFieldName],
dataIndexConfig.currentTimeMarker,
nowReference
);
});
return doc;
}
const bulkInsert = async (docs: any) => {
const insertCmd = { index: { _index: index } };
const bulk: any[] = [];
docs.forEach((doc: any) => {
bulk.push(insertCmd);
bulk.push(updateTimestamps(doc));
});
const { body: resp } = await esClient.asCurrentUser.bulk({
body: bulk,
});
if (resp.errors) {
const errMsg = `sample_data install errors while bulk inserting. Elasticsearch response: ${JSON.stringify(
resp,
null,
''
)}`;
logger.warn(errMsg);
return Promise.reject(
new Error(`Unable to load sample data into index "${index}", see kibana logs for details`)
);
}
};
return loadData(dataIndexConfig.dataPath, bulkInsert); // this returns a Promise
};
import { getSampleDataInstaller } from './utils';
import { SampleDataInstallError } from '../errors';
export function createInstallRoute(
router: IRouter,
@ -95,86 +34,38 @@ export function createInstallRoute(
if (!sampleDataset) {
return res.notFound();
}
// @ts-ignore Custom query validation used
const now = query.now ? new Date(query.now) : new Date();
const nowReference = dateToIso8601IgnoringTime(now);
const counts = {};
for (let i = 0; i < sampleDataset.dataIndices.length; i++) {
const dataIndexConfig = sampleDataset.dataIndices[i];
const index = createIndexName(sampleDataset.id, dataIndexConfig.id);
// clean up any old installation of dataset
try {
await context.core.elasticsearch.client.asCurrentUser.indices.delete({
index,
});
} catch (err) {
// ignore delete errors
}
try {
await context.core.elasticsearch.client.asCurrentUser.indices.create({
index,
body: {
settings: { index: { number_of_shards: 1, auto_expand_replicas: '0-1' } },
mappings: { properties: dataIndexConfig.fields },
},
});
} catch (err) {
const errMsg = `Unable to create sample data index "${index}", error: ${err.message}`;
logger.warn(errMsg);
return res.customError({ body: errMsg, statusCode: err.status });
}
try {
const count = await insertDataIntoIndex(
dataIndexConfig,
index,
nowReference,
context.core.elasticsearch.client,
logger
);
(counts as any)[index] = count;
} catch (err) {
const errMsg = `sample_data install errors while loading data. Error: ${err}`;
throw new Error(errMsg);
}
}
const { getImporter } = context.core.savedObjects;
const objectTypes = getUniqueObjectTypes(sampleDataset.savedObjects);
const savedObjectsClient = getSavedObjectsClient(context, objectTypes);
const importer = getImporter(savedObjectsClient);
const savedObjects = sampleDataset.savedObjects.map(({ version, ...obj }) => obj);
const readStream = Readable.from(savedObjects);
const sampleDataInstaller = getSampleDataInstaller({
datasetId: sampleDataset.id,
sampleDatasets,
logger,
context,
});
try {
const { errors = [] } = await importer.import({
readStream,
overwrite: true,
createNewCopies: false,
const installResult = await sampleDataInstaller.install(params.id, now);
// track the usage operation in a non-blocking way
usageTracker.addInstall(params.id);
return res.ok({
body: {
elasticsearchIndicesCreated: installResult.createdDocsPerIndex,
kibanaSavedObjectsLoaded: installResult.createdSavedObjects,
},
});
if (errors.length > 0) {
const errMsg = `sample_data install errors while loading saved objects. Errors: ${JSON.stringify(
errors.map(({ type, id, error }) => ({ type, id, error })) // discard other fields
)}`;
logger.warn(errMsg);
return res.customError({ body: errMsg, statusCode: 500 });
} catch (e) {
if (e instanceof SampleDataInstallError) {
return res.customError({
body: {
message: e.message,
},
statusCode: e.httpCode,
});
}
} catch (err) {
const errMsg = `import failed, error: ${err.message}`;
throw new Error(errMsg);
throw e;
}
usageTracker.addInstall(params.id);
// FINALLY
return res.ok({
body: {
elasticsearchIndicesCreated: counts,
kibanaSavedObjectsLoaded: sampleDataset.savedObjects.length,
},
});
}
);
}

View file

@ -6,15 +6,12 @@
* Side Public License, v 1.
*/
import { isBoom } from '@hapi/boom';
import { schema } from '@kbn/config-schema';
import type { IRouter, Logger } from 'src/core/server';
import { SampleDatasetSchema } from '../lib/sample_dataset_registry_types';
import { createIndexName } from '../lib/create_index_name';
import { SampleDataUsageTracker } from '../usage/usage';
import { findSampleObjects } from '../lib/find_sample_objects';
import { getUniqueObjectTypes } from '../lib/utils';
import { getSavedObjectsClient } from './utils';
import { getSampleDataInstaller } from './utils';
import { SampleDataInstallError } from '../errors';
export function createUninstallRoute(
router: IRouter,
@ -31,62 +28,33 @@ export function createUninstallRoute(
},
async (context, request, response) => {
const sampleDataset = sampleDatasets.find(({ id }) => id === request.params.id);
if (!sampleDataset) {
return response.notFound();
}
for (let i = 0; i < sampleDataset.dataIndices.length; i++) {
const dataIndexConfig = sampleDataset.dataIndices[i];
const index = createIndexName(sampleDataset.id, dataIndexConfig.id);
try {
// TODO: don't delete the index if sample data exists in other spaces (#116677)
await context.core.elasticsearch.client.asCurrentUser.indices.delete({ index });
} catch (err) {
// if the index doesn't exist, ignore the error and proceed
if (err.body.status !== 404) {
return response.customError({
statusCode: err.body.status,
body: {
message: `Unable to delete sample data index "${index}", error: ${err.body.error.type}`,
},
});
}
}
}
const objects = sampleDataset.savedObjects.map(({ type, id }) => ({ type, id }));
const objectTypes = getUniqueObjectTypes(objects);
const client = getSavedObjectsClient(context, objectTypes);
const findSampleObjectsResult = await findSampleObjects({ client, logger, objects });
const objectsToDelete = findSampleObjectsResult.filter(({ foundObjectId }) => foundObjectId);
const deletePromises = objectsToDelete.map(({ type, foundObjectId }) =>
client.delete(type, foundObjectId!).catch((err) => {
// if the object doesn't exist, ignore the error and proceed
if (isBoom(err) && err.output.statusCode === 404) {
return;
}
throw err;
})
);
const sampleDataInstaller = getSampleDataInstaller({
datasetId: sampleDataset.id,
sampleDatasets,
logger,
context,
});
try {
await Promise.all(deletePromises);
} catch (err) {
return response.customError({
statusCode: err.body.status,
body: {
message: `Unable to delete sample dataset saved objects, error: ${err.body.error.type}`,
},
});
await sampleDataInstaller.uninstall(request.params.id);
// track the usage operation in a non-blocking way
usageTracker.addUninstall(request.params.id);
return response.noContent();
} catch (e) {
if (e instanceof SampleDataInstallError) {
return response.customError({
body: {
message: e.message,
},
statusCode: e.httpCode,
});
}
throw e;
}
// track the usage operation in a non-blocking way
usageTracker.addUninstall(request.params.id);
return response.noContent();
}
);
}

View file

@ -6,12 +6,41 @@
* Side Public License, v 1.
*/
import type { RequestHandlerContext } from 'src/core/server';
import type { RequestHandlerContext, Logger } from 'src/core/server';
import type { SampleDatasetSchema } from '../lib/sample_dataset_registry_types';
import { SampleDataInstaller } from '../sample_data_installer';
import { getUniqueObjectTypes } from '../lib/utils';
export function getSavedObjectsClient(context: RequestHandlerContext, objectTypes: string[]) {
export const getSampleDataInstaller = ({
datasetId,
context,
sampleDatasets,
logger,
}: {
datasetId: string;
context: RequestHandlerContext;
sampleDatasets: SampleDatasetSchema[];
logger: Logger;
}) => {
const sampleDataset = sampleDatasets.find(({ id }) => id === datasetId)!;
const { getImporter, client: soClient } = context.core.savedObjects;
const objectTypes = getUniqueObjectTypes(sampleDataset.savedObjects);
const savedObjectsClient = getSavedObjectsClient(context, objectTypes);
const soImporter = getImporter(savedObjectsClient);
return new SampleDataInstaller({
esClient: context.core.elasticsearch.client,
soImporter,
soClient,
logger,
sampleDatasets,
});
};
export const getSavedObjectsClient = (context: RequestHandlerContext, objectTypes: string[]) => {
const { getClient, typeRegistry } = context.core.savedObjects;
const includedHiddenTypes = objectTypes.filter((supportedType) =>
typeRegistry.isHidden(supportedType)
);
return getClient({ includedHiddenTypes });
}
};

View file

@ -0,0 +1,17 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
export const insertDataIntoIndexMock = jest.fn();
jest.doMock('./lib/insert_data_into_index', () => ({
insertDataIntoIndex: insertDataIntoIndexMock,
}));
export const findSampleObjectsMock = jest.fn();
jest.doMock('./lib/find_sample_objects', () => ({
findSampleObjects: findSampleObjectsMock,
}));

View file

@ -0,0 +1,331 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
import { Readable } from 'stream';
import { insertDataIntoIndexMock, findSampleObjectsMock } from './sample_data_installer.test.mocks';
import type { SavedObjectsImportFailure } from 'kibana/server';
import {
savedObjectsClientMock,
savedObjectsServiceMock,
elasticsearchServiceMock,
loggingSystemMock,
} from '../../../../../core/server/mocks';
import type { SampleDatasetSchema } from './lib/sample_dataset_registry_types';
import { SampleDataInstaller } from './sample_data_installer';
import { SampleDataInstallError } from './errors';
const testDatasets: SampleDatasetSchema[] = [
{
id: 'test_single_data_index',
name: 'Test with a single data index',
description: 'See name',
previewImagePath: 'previewImagePath',
darkPreviewImagePath: 'darkPreviewImagePath',
overviewDashboard: 'overviewDashboard',
defaultIndex: 'defaultIndex',
savedObjects: [
{
id: 'some-dashboard',
type: 'dashboard',
attributes: {
hello: 'dolly',
},
references: [],
},
{
id: 'another-dashboard',
type: 'dashboard',
attributes: {
foo: 'bar',
},
references: [],
},
],
dataIndices: [
{
id: 'test_single_data_index',
dataPath: '/dataPath',
fields: { someField: { type: 'keyword' } },
currentTimeMarker: '2018-01-09T00:00:00',
timeFields: ['@timestamp'],
preserveDayOfWeekTimeOfDay: true,
},
],
},
];
describe('SampleDataInstaller', () => {
let esClient: ReturnType<typeof elasticsearchServiceMock.createScopedClusterClient>;
let soClient: ReturnType<typeof savedObjectsClientMock.create>;
let soImporter: ReturnType<typeof savedObjectsServiceMock.createImporter>;
let logger: ReturnType<typeof loggingSystemMock.createLogger>;
let installer: SampleDataInstaller;
beforeEach(() => {
esClient = elasticsearchServiceMock.createScopedClusterClient();
soClient = savedObjectsClientMock.create();
soImporter = savedObjectsServiceMock.createImporter();
logger = loggingSystemMock.createLogger();
installer = new SampleDataInstaller({
esClient,
soClient,
soImporter,
logger,
sampleDatasets: testDatasets,
});
soImporter.import.mockResolvedValue({
success: true,
successCount: 1,
errors: [],
warnings: [],
});
soClient.delete.mockResolvedValue({});
esClient.asCurrentUser.indices.getAlias.mockImplementation(() => {
throw new Error('alias not found');
});
findSampleObjectsMock.mockResolvedValue([]);
});
afterEach(() => {
insertDataIntoIndexMock.mockReset();
findSampleObjectsMock.mockReset();
});
describe('#install', () => {
it('cleanups the data index before installing', async () => {
await installer.install('test_single_data_index');
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledWith({
index: 'kibana_sample_data_test_single_data_index',
});
});
it('creates the data index', async () => {
await installer.install('test_single_data_index');
expect(esClient.asCurrentUser.indices.create).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.create).toHaveBeenCalledWith({
index: 'kibana_sample_data_test_single_data_index',
body: {
settings: { index: { number_of_shards: 1, auto_expand_replicas: '0-1' } },
mappings: { properties: { someField: { type: 'keyword' } } },
},
});
});
it('inserts the data into the index', async () => {
await installer.install('test_single_data_index');
expect(insertDataIntoIndexMock).toHaveBeenCalledTimes(1);
expect(insertDataIntoIndexMock).toHaveBeenCalledWith({
index: 'kibana_sample_data_test_single_data_index',
nowReference: expect.any(String),
logger,
esClient,
dataIndexConfig: testDatasets[0].dataIndices[0],
});
});
it('imports the saved objects', async () => {
await installer.install('test_single_data_index');
expect(soImporter.import).toHaveBeenCalledTimes(1);
expect(soImporter.import).toHaveBeenCalledWith({
readStream: expect.any(Readable),
overwrite: true,
createNewCopies: false,
});
});
it('throws a SampleDataInstallError with code 404 when the dataset is not found', async () => {
try {
await installer.install('unknown_data_set');
expect('should have returned an error').toEqual('but it did not');
} catch (e) {
expect(e).toBeInstanceOf(SampleDataInstallError);
expect((e as SampleDataInstallError).httpCode).toEqual(404);
}
});
it('does not throw when the index removal fails', async () => {
esClient.asCurrentUser.indices.delete.mockImplementation(() => {
throw new Error('cannot delete index');
});
await expect(installer.install('test_single_data_index')).resolves.toBeDefined();
});
it('throws a SampleDataInstallError when the index creation fails', async () => {
esClient.asCurrentUser.indices.create.mockImplementation(() => {
// eslint-disable-next-line no-throw-literal
throw {
message: 'Cannot create index',
status: 500,
};
});
try {
await installer.install('test_single_data_index');
expect('should have returned an error').toEqual('but it did not');
} catch (e) {
expect(e).toBeInstanceOf(SampleDataInstallError);
expect((e as SampleDataInstallError).httpCode).toEqual(500);
}
});
it('throws a SampleDataInstallError if the savedObject import returns any error', async () => {
soImporter.import.mockResolvedValue({
success: true,
successCount: 1,
errors: [{ type: 'type', id: 'id' } as SavedObjectsImportFailure],
warnings: [],
});
try {
await installer.install('test_single_data_index');
expect('should have returned an error').toEqual('but it did not');
} catch (e) {
expect(e).toBeInstanceOf(SampleDataInstallError);
expect(e.message).toContain('sample_data install errors while loading saved objects');
expect((e as SampleDataInstallError).httpCode).toEqual(500);
}
});
describe('when the data index is using an alias', () => {
it('deletes the alias and the index', async () => {
const indexName = 'target_index';
esClient.asCurrentUser.indices.getAlias.mockResolvedValue(
elasticsearchServiceMock.createApiResponse({
body: {
[indexName]: {
aliases: {
kibana_sample_data_test_single_data_index: {},
},
},
},
})
);
await installer.install('test_single_data_index');
expect(esClient.asCurrentUser.indices.deleteAlias).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.deleteAlias).toHaveBeenCalledWith({
name: 'kibana_sample_data_test_single_data_index',
index: indexName,
});
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledWith({
index: indexName,
});
});
});
});
describe('#uninstall', () => {
it('deletes the data index', async () => {
await installer.uninstall('test_single_data_index');
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledWith({
index: 'kibana_sample_data_test_single_data_index',
});
});
it('deletes the saved objects', async () => {
findSampleObjectsMock.mockResolvedValue([
{ type: 'dashboard', id: 'foo', foundObjectId: 'foo' },
{ type: 'dashboard', id: 'hello', foundObjectId: 'dolly' },
]);
await installer.uninstall('test_single_data_index');
expect(soClient.delete).toHaveBeenCalledTimes(2);
expect(soClient.delete).toHaveBeenCalledWith('dashboard', 'foo');
expect(soClient.delete).toHaveBeenCalledWith('dashboard', 'dolly');
});
it('throws a SampleDataInstallError with code 404 when the dataset is not found', async () => {
try {
await installer.uninstall('unknown_data_set');
expect('should have returned an error').toEqual('but it did not');
} catch (e) {
expect(e).toBeInstanceOf(SampleDataInstallError);
expect((e as SampleDataInstallError).httpCode).toEqual(404);
}
});
it('does not throw when the index removal fails', async () => {
esClient.asCurrentUser.indices.delete.mockImplementation(() => {
throw new Error('cannot delete index');
});
await expect(installer.uninstall('test_single_data_index')).resolves.toBeDefined();
});
it('throws a SampleDataInstallError if any SO deletion fails', async () => {
findSampleObjectsMock.mockResolvedValue([
{ type: 'dashboard', id: 'foo', foundObjectId: 'foo' },
{ type: 'dashboard', id: 'hello', foundObjectId: 'dolly' },
]);
soClient.delete.mockImplementation(async (type: string, id: string) => {
if (id === 'dolly') {
throw new Error('could not delete dolly');
}
return {};
});
try {
await installer.uninstall('test_single_data_index');
expect('should have returned an error').toEqual('but it did not');
} catch (e) {
expect(e).toBeInstanceOf(SampleDataInstallError);
expect((e as SampleDataInstallError).httpCode).toEqual(500);
}
});
describe('when the data index is using an alias', () => {
it('deletes the alias and the index', async () => {
const indexName = 'target_index';
esClient.asCurrentUser.indices.getAlias.mockResolvedValue(
elasticsearchServiceMock.createApiResponse({
body: {
[indexName]: {
aliases: {
kibana_sample_data_test_single_data_index: {},
},
},
},
})
);
await installer.uninstall('test_single_data_index');
expect(esClient.asCurrentUser.indices.deleteAlias).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.deleteAlias).toHaveBeenCalledWith({
name: 'kibana_sample_data_test_single_data_index',
index: indexName,
});
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledTimes(1);
expect(esClient.asCurrentUser.indices.delete).toHaveBeenCalledWith({
index: indexName,
});
});
});
});
});

View file

@ -0,0 +1,205 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
import { Readable } from 'stream';
import { isBoom } from '@hapi/boom';
import type {
IScopedClusterClient,
ISavedObjectsImporter,
Logger,
SavedObjectsClientContract,
} from 'src/core/server';
import type { SampleDatasetSchema, DataIndexSchema } from './lib/sample_dataset_registry_types';
import { dateToIso8601IgnoringTime } from './lib/translate_timestamp';
import { createIndexName } from './lib/create_index_name';
import { insertDataIntoIndex } from './lib/insert_data_into_index';
import { SampleDataInstallError } from './errors';
import { findSampleObjects } from './lib/find_sample_objects';
export interface SampleDataInstallerOptions {
esClient: IScopedClusterClient;
soClient: SavedObjectsClientContract;
soImporter: ISavedObjectsImporter;
sampleDatasets: SampleDatasetSchema[];
logger: Logger;
}
export interface SampleDataInstallResult {
createdDocsPerIndex: Record<string, number>;
createdSavedObjects: number;
}
/**
* Utility class in charge of installing and uninstalling sample datasets
*/
export class SampleDataInstaller {
private readonly esClient: IScopedClusterClient;
private readonly soClient: SavedObjectsClientContract;
private readonly soImporter: ISavedObjectsImporter;
private readonly sampleDatasets: SampleDatasetSchema[];
private readonly logger: Logger;
constructor({
esClient,
soImporter,
soClient,
sampleDatasets,
logger,
}: SampleDataInstallerOptions) {
this.esClient = esClient;
this.soClient = soClient;
this.soImporter = soImporter;
this.sampleDatasets = sampleDatasets;
this.logger = logger;
}
async install(
datasetId: string,
installDate: Date = new Date()
): Promise<SampleDataInstallResult> {
const sampleDataset = this.sampleDatasets.find(({ id }) => id === datasetId);
if (!sampleDataset) {
throw new SampleDataInstallError(`Sample dataset ${datasetId} not found`, 404);
}
const nowReference = dateToIso8601IgnoringTime(installDate);
const createdDocsPerIndex: Record<string, number> = {};
for (let i = 0; i < sampleDataset.dataIndices.length; i++) {
const dataIndex = sampleDataset.dataIndices[i];
const indexName = createIndexName(sampleDataset.id, dataIndex.id);
// clean up any old installation of dataset
await this.uninstallDataIndex(sampleDataset, dataIndex);
await this.installDataIndex(sampleDataset, dataIndex);
const injectedCount = await insertDataIntoIndex({
index: indexName,
nowReference,
logger: this.logger,
esClient: this.esClient,
dataIndexConfig: dataIndex,
});
createdDocsPerIndex[indexName] = injectedCount;
}
const createdSavedObjects = await this.importSavedObjects(sampleDataset);
return {
createdDocsPerIndex,
createdSavedObjects,
};
}
async uninstall(datasetId: string) {
const sampleDataset = this.sampleDatasets.find(({ id }) => id === datasetId);
if (!sampleDataset) {
throw new SampleDataInstallError(`Sample dataset ${datasetId} not found`, 404);
}
for (let i = 0; i < sampleDataset.dataIndices.length; i++) {
const dataIndex = sampleDataset.dataIndices[i];
await this.uninstallDataIndex(sampleDataset, dataIndex);
}
const deletedObjects = await this.deleteSavedObjects(sampleDataset);
return {
deletedSavedObjects: deletedObjects,
};
}
private async uninstallDataIndex(dataset: SampleDatasetSchema, dataIndex: DataIndexSchema) {
let index = createIndexName(dataset.id, dataIndex.id);
try {
// if the sample data was reindexed using UA, the index name is actually an alias pointing to the reindexed
// index. In that case, we need to get rid of the alias and to delete the underlying index
const { body: response } = await this.esClient.asCurrentUser.indices.getAlias({
name: index,
});
const aliasName = index;
index = Object.keys(response)[0];
await this.esClient.asCurrentUser.indices.deleteAlias({ name: aliasName, index });
} catch (err) {
// ignore errors from missing alias
}
try {
await this.esClient.asCurrentUser.indices.delete({
index,
});
} catch (err) {
// ignore delete errors
}
}
private async installDataIndex(dataset: SampleDatasetSchema, dataIndex: DataIndexSchema) {
const index = createIndexName(dataset.id, dataIndex.id);
try {
await this.esClient.asCurrentUser.indices.create({
index,
body: {
settings: { index: { number_of_shards: 1, auto_expand_replicas: '0-1' } },
mappings: { properties: dataIndex.fields },
},
});
} catch (err) {
const errMsg = `Unable to create sample data index "${index}", error: ${err.message}`;
this.logger.warn(errMsg);
throw new SampleDataInstallError(errMsg, err.status);
}
}
private async importSavedObjects(dataset: SampleDatasetSchema) {
const savedObjects = dataset.savedObjects.map(({ version, ...obj }) => obj);
const readStream = Readable.from(savedObjects);
const { errors = [] } = await this.soImporter.import({
readStream,
overwrite: true,
createNewCopies: false,
});
if (errors.length > 0) {
const errMsg = `sample_data install errors while loading saved objects. Errors: ${JSON.stringify(
errors.map(({ type, id, error }) => ({ type, id, error })) // discard other fields
)}`;
this.logger.warn(errMsg);
throw new SampleDataInstallError(errMsg, 500);
}
return savedObjects.length;
}
private async deleteSavedObjects(dataset: SampleDatasetSchema) {
const objects = dataset.savedObjects.map(({ type, id }) => ({ type, id }));
const findSampleObjectsResult = await findSampleObjects({
client: this.soClient,
logger: this.logger,
objects,
});
const objectsToDelete = findSampleObjectsResult.filter(({ foundObjectId }) => foundObjectId);
const deletePromises = objectsToDelete.map(({ type, foundObjectId }) =>
this.soClient.delete(type, foundObjectId!).catch((err) => {
// if the object doesn't exist, ignore the error and proceed
if (isBoom(err) && err.output.statusCode === 404) {
return;
}
throw err;
})
);
try {
await Promise.all(deletePromises);
} catch (err) {
throw new SampleDataInstallError(
`Unable to delete sample dataset saved objects, error: ${
err.body?.error?.type ?? err.message
}`,
err.body?.status ?? 500
);
}
return objectsToDelete.length;
}
}

View file

@ -21,8 +21,7 @@ export default function ({ getService }: FtrProviderContext) {
const FLIGHTS_CANVAS_APPLINK_PATH =
'/app/canvas#/workpad/workpad-a474e74b-aedc-47c3-894a-db77e62c41e0'; // includes default ID of the flights canvas applink path
// Failing: See https://github.com/elastic/kibana/issues/121051
describe.skip('sample data apis', () => {
describe('sample data apis', () => {
before(async () => {
await esArchiver.emptyKibanaIndex();
});
@ -63,22 +62,23 @@ export default function ({ getService }: FtrProviderContext) {
});
});
it('should load elasticsearch index containing sample data with dates relative to current time', async () => {
const resp = await es.search<{ timestamp: string }>({
index: 'kibana_sample_data_flights',
body: {
sort: [{ timestamp: { order: 'desc' } }],
},
// Failing: See https://github.com/elastic/kibana/issues/121051
describe.skip('dates', () => {
it('should load elasticsearch index containing sample data with dates relative to current time', async () => {
const resp = await es.search<{ timestamp: string }>({
index: 'kibana_sample_data_flights',
body: {
sort: [{ timestamp: { order: 'desc' } }],
},
});
const doc = resp.hits.hits[0];
const docMilliseconds = Date.parse(doc._source!.timestamp);
const nowMilliseconds = Date.now();
const delta = Math.abs(nowMilliseconds - docMilliseconds);
expect(delta).to.be.lessThan(MILLISECOND_IN_WEEK * 5);
});
const doc = resp.hits.hits[0];
const docMilliseconds = Date.parse(doc._source!.timestamp);
const nowMilliseconds = Date.now();
const delta = Math.abs(nowMilliseconds - docMilliseconds);
expect(delta).to.be.lessThan(MILLISECOND_IN_WEEK * 5);
});
describe('parameters', () => {
it('should load elasticsearch index containing sample data with dates relative to now parameter', async () => {
const nowString = `2000-01-01T00:00:00`;
await supertest.post(`${apiPath}/flights?now=${nowString}`).set('kbn-xsrf', 'kibana');