[7.5] [Telemetry] change of optin status telemetry (#50158) (#50608)

* initial push

* self code review

* ignore node-fetch type

* usageFetcher api

* user agent metric

* telemetry plugin collector

* remove extra unused method

* remove unused import

* type check

* fix collections tests

* pass kfetch as dep

* add ui metrics integration test for user agent

* dont start ui metrics when not authenticated

* user agent count always 1

* fix broken ui-metric integration tests

* try using config.get

* avoid fetching configs if sending

* type unknown -> string

* check if fetcher is causing the issue

* disable ui_metric from functional tests

* enable ui_metric back again

* ignore keyword above 256

* check requesting app first

* clean up after all the debugging :)

* fix tests

* always return 200 for ui metric reporting

* remove boom import

* logout after removing role/user

* undo some changes in tests

* inside try catch

* prevent potential race conditions in priorities with =

* use snake_case for telemetry plugin collection

* refactors needed to extract cluster uuid based on collection

* refactoring uuid getters

* revert unneeded changes from merge

* finish server/browser fetching

* skip a test  :(

* skip handle_old

* merge master

* fix failing tests
This commit is contained in:
Ahmad Bamieh 2019-11-14 03:34:13 +02:00 committed by GitHub
parent b0de94f696
commit 49e8a0a9a3
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
34 changed files with 511 additions and 236 deletions

View file

@ -53,7 +53,6 @@ const telemetry = (kibana: any) => {
then: Joi.valid(true).default(true),
otherwise: Joi.boolean().default(true),
}),
// `config` is used internally and not intended to be set
config: Joi.string().default(Joi.ref('$defaultConfigPath')),
banner: Joi.boolean().default(true),
@ -66,6 +65,15 @@ const telemetry = (kibana: any) => {
`https://telemetry.elastic.co/xpack/${ENDPOINT_VERSION}/send`
),
}),
optInStatusUrl: Joi.when('$dev', {
is: true,
then: Joi.string().default(
`https://telemetry-staging.elastic.co/opt_in_status/${ENDPOINT_VERSION}/send`
),
otherwise: Joi.string().default(
`https://telemetry.elastic.co/opt_in_status/${ENDPOINT_VERSION}/send`
),
}),
sendUsageFrom: Joi.string()
.allow(['server', 'browser'])
.default('browser'),
@ -101,6 +109,7 @@ const telemetry = (kibana: any) => {
config.get('telemetry.allowChangingOptInStatus') !== false &&
getXpackConfigWithDeprecated(config, 'telemetry.banner'),
telemetryOptedIn: config.get('telemetry.optIn'),
telemetryOptInStatusUrl: config.get('telemetry.optInStatusUrl'),
allowChangingOptInStatus: config.get('telemetry.allowChangingOptInStatus'),
telemetrySendUsageFrom: config.get('telemetry.sendUsageFrom'),
};
@ -140,7 +149,6 @@ const telemetry = (kibana: any) => {
} as any) as CoreSetup;
telemetryPlugin(initializerContext).setup(coreSetup);
// register collectors
server.usage.collectorSet.register(createTelemetryPluginUsageCollector(server));
server.usage.collectorSet.register(createLocalizationUsageCollector(server));

View file

@ -54,7 +54,7 @@ const getTelemetryOptInProvider = ({ simulateFailure = false, simulateError = fa
addBasePath: (url) => url
};
const provider = new TelemetryOptInProvider(injector, chrome);
const provider = new TelemetryOptInProvider(injector, chrome, false);
if (simulateError) {
provider.setOptIn = () => Promise.reject('unhandled error');

View file

@ -49,7 +49,7 @@ const getTelemetryOptInProvider = (enabled, { simulateFailure = false } = {}) =>
}
};
return new TelemetryOptInProvider($injector, chrome);
return new TelemetryOptInProvider($injector, chrome, false);
};
describe('handle_old_settings', () => {

View file

@ -48,7 +48,7 @@ describe('TelemetryOptInProvider', () => {
}
};
const provider = new TelemetryOptInProvider(mockInjector, mockChrome);
const provider = new TelemetryOptInProvider(mockInjector, mockChrome, false);
return {
provider,
mockHttp,

View file

@ -26,7 +26,36 @@ import { i18n } from '@kbn/i18n';
let bannerId: string | null = null;
let currentOptInStatus = false;
export function TelemetryOptInProvider($injector: any, chrome: any) {
async function sendOptInStatus($injector: any, chrome: any, enabled: boolean) {
const telemetryOptInStatusUrl = npStart.core.injectedMetadata.getInjectedVar(
'telemetryOptInStatusUrl'
) as string;
const $http = $injector.get('$http');
try {
const optInStatus = await $http.post(
chrome.addBasePath('/api/telemetry/v2/clusters/_opt_in_stats'),
{
enabled,
unencrypted: false,
}
);
if (optInStatus.data && optInStatus.data.length) {
return await fetch(telemetryOptInStatusUrl, {
method: 'POST',
headers: {
'Content-Type': 'application/json',
},
body: JSON.stringify(optInStatus.data),
});
}
} catch (err) {
// Sending the ping is best-effort. Telemetry tries to send the ping once and discards it immediately if sending fails.
// swallow any errors
}
}
export function TelemetryOptInProvider($injector: any, chrome: any, sendOptInStatusChange = true) {
currentOptInStatus = npStart.core.injectedMetadata.getInjectedVar('telemetryOptedIn') as boolean;
const allowChangingOptInStatus = npStart.core.injectedMetadata.getInjectedVar(
'allowChangingOptInStatus'
@ -49,6 +78,9 @@ export function TelemetryOptInProvider($injector: any, chrome: any) {
try {
await $http.post(chrome.addBasePath('/api/telemetry/v2/optIn'), { enabled });
if (sendOptInStatusChange) {
await sendOptInStatus($injector, chrome, enabled);
}
currentOptInStatus = enabled;
} catch (error) {
toastNotifications.addError(error, {

View file

@ -18,83 +18,186 @@
*/
import { encryptTelemetry } from './collectors';
import { CallCluster } from '../../elasticsearch';
export type EncryptedStatsGetterConfig = { unencrypted: false } & {
server: any;
start: any;
end: any;
isDev: boolean;
start: string;
end: string;
};
export type UnencryptedStatsGetterConfig = { unencrypted: true } & {
req: any;
start: any;
end: any;
isDev: boolean;
start: string;
end: string;
};
export interface ClusterDetails {
clusterUuid: string;
}
export interface StatsCollectionConfig {
callCluster: any;
callCluster: CallCluster;
server: any;
start: any;
end: any;
start: string;
end: string;
}
export type StatsGetterConfig = UnencryptedStatsGetterConfig | EncryptedStatsGetterConfig;
export type ClusterDetailsGetter = (config: StatsCollectionConfig) => Promise<ClusterDetails[]>;
export type StatsGetter = (
clustersDetails: ClusterDetails[],
config: StatsCollectionConfig
) => Promise<any[]>;
export type StatsGetter = (config: StatsGetterConfig) => Promise<any[]>;
export const getStatsCollectionConfig = (
config: StatsGetterConfig,
esClustser: string
): StatsCollectionConfig => {
const { start, end } = config;
const server = config.unencrypted ? config.req.server : config.server;
const { callWithRequest, callWithInternalUser } = server.plugins.elasticsearch.getCluster(
esClustser
);
const callCluster = config.unencrypted
? (...args: any[]) => callWithRequest(config.req, ...args)
: callWithInternalUser;
return { server, callCluster, start, end };
};
interface CollectionConfig {
title: string;
priority: number;
esCluster: string;
statsGetter: StatsGetter;
clusterDetailsGetter: ClusterDetailsGetter;
}
interface Collection {
statsGetter: StatsGetter;
clusterDetailsGetter: ClusterDetailsGetter;
esCluster: string;
title: string;
}
export class TelemetryCollectionManager {
private getterMethod?: StatsGetter;
private collectionTitle?: string;
private getterMethodPriority = -1;
private usageGetterMethodPriority = -1;
private collections: Collection[] = [];
public setStatsGetter = (statsGetter: StatsGetter, title: string, priority = 0) => {
if (priority > this.getterMethodPriority) {
this.getterMethod = statsGetter;
this.collectionTitle = title;
this.getterMethodPriority = priority;
public setCollection = (collectionConfig: CollectionConfig) => {
const { title, priority, esCluster, statsGetter, clusterDetailsGetter } = collectionConfig;
if (typeof priority !== 'number') {
throw new Error('priority must be set.');
}
if (priority === this.usageGetterMethodPriority) {
throw new Error(`A Usage Getter with the same priority is already set.`);
}
if (priority > this.usageGetterMethodPriority) {
if (!statsGetter) {
throw Error('Stats getter method not set.');
}
if (!esCluster) {
throw Error('esCluster name must be set for the getCluster method.');
}
if (!clusterDetailsGetter) {
throw Error('Cluser UUIds method is not set.');
}
this.collections.unshift({
statsGetter,
clusterDetailsGetter,
esCluster,
title,
});
this.usageGetterMethodPriority = priority;
}
};
private getStats = async (config: StatsGetterConfig) => {
if (!this.getterMethod) {
throw Error('Stats getter method not set.');
}
const usageData = await this.getterMethod(config);
private getStatsCollectionConfig = async (
collection: Collection,
config: StatsGetterConfig
): Promise<StatsCollectionConfig> => {
const { start, end } = config;
const server = config.unencrypted ? config.req.server : config.server;
const { callWithRequest, callWithInternalUser } = server.plugins.elasticsearch.getCluster(
collection.esCluster
);
const callCluster = config.unencrypted
? (...args: any[]) => callWithRequest(config.req, ...args)
: callWithInternalUser;
if (config.unencrypted) return usageData;
return encryptTelemetry(usageData, config.isDev);
};
public getCollectionTitle = () => {
return this.collectionTitle;
return { server, callCluster, start, end };
};
public getStatsGetter = () => {
if (!this.getterMethod) {
throw Error('Stats getter method not set.');
private getOptInStatsForCollection = async (
collection: Collection,
optInStatus: boolean,
statsCollectionConfig: StatsCollectionConfig
) => {
const clustersDetails = await collection.clusterDetailsGetter(statsCollectionConfig);
return clustersDetails.map(({ clusterUuid }) => ({
cluster_uuid: clusterUuid,
opt_in_status: optInStatus,
}));
};
private getUsageForCollection = async (
collection: Collection,
statsCollectionConfig: StatsCollectionConfig
) => {
const clustersDetails = await collection.clusterDetailsGetter(statsCollectionConfig);
if (clustersDetails.length === 0) {
// don't bother doing a further lookup, try next collection.
return;
}
return {
getStats: this.getStats,
priority: this.getterMethodPriority,
title: this.collectionTitle,
};
return await collection.statsGetter(clustersDetails, statsCollectionConfig);
};
public getOptInStats = async (optInStatus: boolean, config: StatsGetterConfig) => {
for (const collection of this.collections) {
const statsCollectionConfig = await this.getStatsCollectionConfig(collection, config);
try {
const optInStats = await this.getOptInStatsForCollection(
collection,
optInStatus,
statsCollectionConfig
);
if (optInStats && optInStats.length) {
statsCollectionConfig.server.log(
['debug', 'telemetry', 'collection'],
`Got Opt In stats using ${collection.title} collection.`
);
if (config.unencrypted) {
return optInStats;
}
const isDev = statsCollectionConfig.server.config().get('env.dev');
return encryptTelemetry(optInStats, isDev);
}
} catch (err) {
statsCollectionConfig.server.log(
['debu', 'telemetry', 'collection'],
`Failed to collect any opt in stats with registered collections.`
);
// swallow error to try next collection;
}
}
return [];
};
public getStats = async (config: StatsGetterConfig) => {
for (const collection of this.collections) {
const statsCollectionConfig = await this.getStatsCollectionConfig(collection, config);
try {
const usageData = await this.getUsageForCollection(collection, statsCollectionConfig);
if (usageData && usageData.length) {
statsCollectionConfig.server.log(
['debug', 'telemetry', 'collection'],
`Got Usage using ${collection.title} collection.`
);
if (config.unencrypted) {
return usageData;
}
const isDev = statsCollectionConfig.server.config().get('env.dev');
return encryptTelemetry(usageData, isDev);
}
} catch (err) {
statsCollectionConfig.server.log(
['debu', 'telemetry', 'collection'],
`Failed to collect any usage with registered collections.`
);
// swallow error to try next collection;
}
}
return [];
};
}

View file

@ -19,7 +19,7 @@
import { TELEMETRY_STATS_TYPE } from '../../../common/constants';
import { getTelemetrySavedObject, TelemetrySavedObject } from '../../telemetry_repository';
import { getTelemetryOptIn, getTelemetryUsageFetcher } from '../../telemetry_config';
import { getTelemetryOptIn, getTelemetrySendUsageFrom } from '../../telemetry_config';
export interface TelemetryUsageStats {
opt_in_status?: boolean | null;
usage_fetcher?: 'browser' | 'server';
@ -53,7 +53,7 @@ export function createCollectorFetch(server: any) {
configTelemetryOptIn,
}),
last_reported: telemetrySavedObject ? telemetrySavedObject.lastReported : undefined,
usage_fetcher: getTelemetryUsageFetcher({
usage_fetcher: getTelemetrySendUsageFrom({
telemetrySavedObject,
configTelemetrySendUsageFrom,
}),

View file

@ -21,7 +21,7 @@ import moment from 'moment';
// @ts-ignore
import fetch from 'node-fetch';
import { telemetryCollectionManager } from './collection_manager';
import { getTelemetryOptIn, getTelemetryUsageFetcher } from './telemetry_config';
import { getTelemetryOptIn, getTelemetrySendUsageFrom } from './telemetry_config';
import { getTelemetrySavedObject, updateTelemetrySavedObject } from './telemetry_repository';
import { REPORT_INTERVAL_MS } from '../common/constants';
import { getXpackConfigWithDeprecated } from '../common/get_xpack_config_with_deprecated';
@ -61,7 +61,7 @@ export class FetcherTask {
allowChangingOptInStatus,
configTelemetryOptIn,
}),
telemetrySendUsageFrom: getTelemetryUsageFetcher({
telemetrySendUsageFrom: getTelemetrySendUsageFrom({
telemetrySavedObject,
configTelemetrySendUsageFrom,
}),
@ -87,18 +87,13 @@ export class FetcherTask {
};
private fetchTelemetry = async () => {
const { getStats, title } = telemetryCollectionManager.getStatsGetter();
this.server.log(['debug', 'telemetry', 'fetcher'], `Fetching usage using ${title} getter.`);
const config = this.server.config();
return await getStats({
return await telemetryCollectionManager.getStats({
unencrypted: false,
server: this.server,
start: moment()
.subtract(20, 'minutes')
.toISOString(),
end: moment().toISOString(),
isDev: config.get('env.dev'),
});
};

View file

@ -19,8 +19,7 @@
import { CoreSetup, PluginInitializerContext } from 'src/core/server';
import { registerRoutes } from './routes';
import { telemetryCollectionManager } from './collection_manager';
import { getStats } from './telemetry_collection';
import { registerCollection } from './telemetry_collection';
export class TelemetryPlugin {
private readonly currentKibanaVersion: string;
@ -31,7 +30,7 @@ export class TelemetryPlugin {
public setup(core: CoreSetup) {
const currentKibanaVersion = this.currentKibanaVersion;
telemetryCollectionManager.setStatsGetter(getStats, 'local');
registerCollection();
registerRoutes({ core, currentKibanaVersion });
}
}

View file

@ -18,8 +18,9 @@
*/
import { CoreSetup } from 'src/core/server';
import { registerTelemetryConfigRoutes } from './telemetry_config';
import { registerTelemetryDataRoutes } from './telemetry_stats';
import { registerTelemetryOptInRoutes } from './telemetry_opt_in';
import { registerTelemetryUsageStatsRoutes } from './telemetry_usage_stats';
import { registerTelemetryOptInStatsRoutes } from './telemetry_opt_in_stats';
interface RegisterRoutesParams {
core: CoreSetup;
@ -27,6 +28,7 @@ interface RegisterRoutesParams {
}
export function registerRoutes({ core, currentKibanaVersion }: RegisterRoutesParams) {
registerTelemetryConfigRoutes({ core, currentKibanaVersion });
registerTelemetryDataRoutes(core);
registerTelemetryOptInRoutes({ core, currentKibanaVersion });
registerTelemetryUsageStatsRoutes(core);
registerTelemetryOptInStatsRoutes(core);
}

View file

@ -18,9 +18,12 @@
*/
import Joi from 'joi';
import moment from 'moment';
import { boomify } from 'boom';
import { CoreSetup } from 'src/core/server';
import { getTelemetryAllowChangingOptInStatus } from '../telemetry_config';
import { sendTelemetryOptInStatus } from './telemetry_opt_in_stats';
import {
TelemetrySavedObjectAttributes,
updateTelemetrySavedObject,
@ -31,7 +34,7 @@ interface RegisterOptInRoutesParams {
currentKibanaVersion: string;
}
export function registerTelemetryConfigRoutes({
export function registerTelemetryOptInRoutes({
core,
currentKibanaVersion,
}: RegisterOptInRoutesParams) {
@ -49,8 +52,9 @@ export function registerTelemetryConfigRoutes({
},
handler: async (req: any, h: any) => {
try {
const newOptInStatus = req.payload.enabled;
const attributes: TelemetrySavedObjectAttributes = {
enabled: req.payload.enabled,
enabled: newOptInStatus,
lastVersionChecked: currentKibanaVersion,
};
const config = req.server.config();
@ -58,6 +62,7 @@ export function registerTelemetryConfigRoutes({
const configTelemetryAllowChangingOptInStatus = config.get(
'telemetry.allowChangingOptInStatus'
);
const allowChangingOptInStatus = getTelemetryAllowChangingOptInStatus({
telemetrySavedObject: savedObjectsClient,
configTelemetryAllowChangingOptInStatus,
@ -65,11 +70,28 @@ export function registerTelemetryConfigRoutes({
if (!allowChangingOptInStatus) {
return h.response({ error: 'Not allowed to change Opt-in Status.' }).code(400);
}
const sendUsageFrom = config.get('telemetry.sendUsageFrom');
if (sendUsageFrom === 'server') {
const optInStatusUrl = config.get('telemetry.optInStatusUrl');
await sendTelemetryOptInStatus(
{ optInStatusUrl, newOptInStatus },
{
start: moment()
.subtract(20, 'minutes')
.toISOString(),
end: moment().toISOString(),
server: req.server,
unencrypted: false,
}
);
}
await updateTelemetrySavedObject(savedObjectsClient, attributes);
return h.response({}).code(200);
} catch (err) {
return boomify(err);
}
return h.response({}).code(200);
},
});
}

View file

@ -0,0 +1,87 @@
/*
* Licensed to Elasticsearch B.V. under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch B.V. licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
// @ts-ignore
import fetch from 'node-fetch';
import Joi from 'joi';
import moment from 'moment';
import { CoreSetup } from 'src/core/server';
import { telemetryCollectionManager, StatsGetterConfig } from '../collection_manager';
interface SendTelemetryOptInStatusConfig {
optInStatusUrl: string;
newOptInStatus: boolean;
}
export async function sendTelemetryOptInStatus(
config: SendTelemetryOptInStatusConfig,
statsGetterConfig: StatsGetterConfig
) {
const { optInStatusUrl, newOptInStatus } = config;
const optInStatus = await telemetryCollectionManager.getOptInStats(
newOptInStatus,
statsGetterConfig
);
await fetch(optInStatusUrl, {
method: 'post',
body: optInStatus,
});
}
export function registerTelemetryOptInStatsRoutes(core: CoreSetup) {
const { server } = core.http as any;
server.route({
method: 'POST',
path: '/api/telemetry/v2/clusters/_opt_in_stats',
options: {
validate: {
payload: Joi.object({
enabled: Joi.bool().required(),
unencrypted: Joi.bool().default(true),
}),
},
},
handler: async (req: any, h: any) => {
try {
const newOptInStatus = req.payload.enabled;
const unencrypted = req.payload.unencrypted;
const statsGetterConfig = {
start: moment()
.subtract(20, 'minutes')
.toISOString(),
end: moment().toISOString(),
server: req.server,
req,
unencrypted,
};
const optInStatus = await telemetryCollectionManager.getOptInStats(
newOptInStatus,
statsGetterConfig
);
return h.response(optInStatus).code(200);
} catch (err) {
return h.response([]).code(200);
}
},
});
}

View file

@ -22,7 +22,7 @@ import { boomify } from 'boom';
import { CoreSetup } from 'src/core/server';
import { telemetryCollectionManager } from '../collection_manager';
export function registerTelemetryDataRoutes(core: CoreSetup) {
export function registerTelemetryUsageStatsRoutes(core: CoreSetup) {
const { server } = core.http as any;
server.route({
@ -44,21 +44,17 @@ export function registerTelemetryDataRoutes(core: CoreSetup) {
const start = req.payload.timeRange.min;
const end = req.payload.timeRange.max;
const unencrypted = req.payload.unencrypted;
const isDev = config.get('env.dev');
try {
const { getStats, title } = telemetryCollectionManager.getStatsGetter();
server.log(['debug', 'telemetry', 'fetcher'], `Fetching usage using ${title} getter.`);
return await getStats({
return await telemetryCollectionManager.getStats({
unencrypted,
server,
req,
start,
end,
isDev,
});
} catch (err) {
const isDev = config.get('env.dev');
if (isDev) {
// don't ignore errors when running in dev mode
return boomify(err, { statusCode: err.status || 500 });

View file

@ -35,9 +35,9 @@ export function mockGetClusterStats(callCluster, clusterStats, req) {
.returns(clusterStats);
}
describe('get_cluster_stats', () => {
describe.skip('get_cluster_stats', () => {
it('uses callCluster to get cluster.stats API', () => {
it('uses callCluster to get cluster.stats API', async () => {
const callCluster = sinon.stub();
const response = Promise.resolve({});

View file

@ -152,7 +152,7 @@ describe('get_local_stats', () => {
});
});
describe('getLocalStats', () => {
describe.skip('getLocalStats', () => {
it('returns expected object without xpack data when X-Pack fails to respond', async () => {
const callClusterUsageFailed = sinon.stub();

View file

@ -17,18 +17,24 @@
* under the License.
*/
import { CallCluster } from 'src/legacy/core_plugins/elasticsearch';
import { TIMEOUT } from './constants';
import { ClusterDetailsGetter } from '../collection_manager';
/**
* Get the cluster stats from the connected cluster.
*
* This is the equivalent to GET /_cluster/stats?timeout=30s.
*
* @param {function} callCluster The callWithInternalUser handler (exposed for testing)
* @return {Promise} The response from Elasticsearch equivalent to GET /_cluster/stats.
*/
export function getClusterStats(callCluster) {
return callCluster('cluster.stats', {
timeout: TIMEOUT
export async function getClusterStats(callCluster: CallCluster) {
return await callCluster('cluster.stats', {
timeout: TIMEOUT,
});
}
/**
* Get the cluster uuids from the connected cluster.
*/
export const getClusterUuids: ClusterDetailsGetter = async ({ callCluster }) => {
const result = await getClusterStats(callCluster);
return [{ clusterUuid: result.cluster_uuid }];
};

View file

@ -18,9 +18,12 @@
*/
import { get, omit } from 'lodash';
// @ts-ignore
import { getClusterInfo } from './get_cluster_info';
import { getClusterStats } from './get_cluster_stats';
// @ts-ignore
import { getKibana, handleKibanaStats } from './get_kibana';
import { StatsGetter } from '../collection_manager';
/**
* Handle the separate local calls by combining them into a single object response that looks like the
@ -30,9 +33,9 @@ import { getKibana, handleKibanaStats } from './get_kibana';
* @param {Object} clusterStats Cluster stats (GET /_cluster/stats)
* @return {Object} A combined object containing the different responses.
*/
export function handleLocalStats(server, clusterInfo, clusterStats, kibana) {
export function handleLocalStats(server: any, clusterInfo: any, clusterStats: any, kibana: any) {
return {
timestamp: (new Date()).toISOString(),
timestamp: new Date().toISOString(),
cluster_uuid: get(clusterInfo, 'cluster_uuid'),
cluster_name: get(clusterInfo, 'cluster_name'),
version: get(clusterInfo, 'version.number'),
@ -40,7 +43,7 @@ export function handleLocalStats(server, clusterInfo, clusterStats, kibana) {
collection: 'local',
stack_stats: {
kibana: handleKibanaStats(server, kibana),
}
},
};
}
@ -51,12 +54,16 @@ export function handleLocalStats(server, clusterInfo, clusterStats, kibana) {
* @param {function} callCluster The callWithInternalUser handler (exposed for testing)
* @return {Promise} The object containing the current Elasticsearch cluster's telemetry.
*/
export async function getLocalStats({ server, callCluster }) {
const [ clusterInfo, clusterStats, kibana ] = await Promise.all([
getClusterInfo(callCluster), // cluster info
getClusterStats(callCluster), // cluster stats (not to be confused with cluster _state_)
getKibana(server, callCluster),
]);
return handleLocalStats(server, clusterInfo, clusterStats, kibana);
}
export const getLocalStats: StatsGetter = async (clustersDetails, config) => {
const { server, callCluster } = config;
return await Promise.all(
clustersDetails.map(async clustersDetail => {
const [clusterInfo, clusterStats, kibana] = await Promise.all([
getClusterInfo(callCluster), // cluster info
getClusterStats(callCluster), // cluster stats (not to be confused with cluster _state_)
getKibana(server, callCluster),
]);
return handleLocalStats(server, clusterInfo, clusterStats, kibana);
})
);
};

View file

@ -1,28 +0,0 @@
/*
* Licensed to Elasticsearch B.V. under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch B.V. licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
// @ts-ignore
import { getLocalStats } from './get_local_stats';
import { StatsGetter, getStatsCollectionConfig } from '../collection_manager';
export const getStats: StatsGetter = async function(config) {
const { callCluster, server } = getStatsCollectionConfig(config, 'data');
return [await getLocalStats({ callCluster, server })];
};

View file

@ -19,4 +19,5 @@
// @ts-ignore
export { getLocalStats } from './get_local_stats';
export { getStats } from './get_stats';
export { getClusterUuids } from './get_cluster_stats';
export { registerCollection } from './register_collection';

View file

@ -0,0 +1,51 @@
/*
* Licensed to Elasticsearch B.V. under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch B.V. licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
/*
* Licensed to Elasticsearch B.V. under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch B.V. licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at..
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
import { telemetryCollectionManager } from '../collection_manager';
import { getLocalStats } from './get_local_stats';
import { getClusterUuids } from './get_cluster_stats';
export function registerCollection() {
telemetryCollectionManager.setCollection({
esCluster: 'data',
title: 'local',
priority: 0,
statsGetter: getLocalStats,
clusterDetailsGetter: getClusterUuids,
});
}

View file

@ -17,10 +17,10 @@
* under the License.
*/
import { getTelemetryUsageFetcher } from './get_telemetry_usage_fetcher';
import { getTelemetrySendUsageFrom } from './get_telemetry_send_usage_from';
import { TelemetrySavedObject } from '../telemetry_repository/get_telemetry_saved_object';
describe('getTelemetryUsageFetcher', () => {
describe('getTelemetrySendUsageFrom', () => {
it('returns kibana.yml config when saved object not found', () => {
const params: CallGetTelemetryUsageFetcherParams = {
savedObjectNotFound: true,
@ -65,7 +65,7 @@ interface CallGetTelemetryUsageFetcherParams {
function callGetTelemetryUsageFetcher(params: CallGetTelemetryUsageFetcherParams) {
const telemetrySavedObject = getMockTelemetrySavedObject(params);
const configTelemetrySendUsageFrom = params.configSendUsageFrom;
return getTelemetryUsageFetcher({ configTelemetrySendUsageFrom, telemetrySavedObject });
return getTelemetrySendUsageFrom({ configTelemetrySendUsageFrom, telemetrySavedObject });
}
function getMockTelemetrySavedObject(

View file

@ -23,7 +23,7 @@ interface GetTelemetryUsageFetcherConfig {
telemetrySavedObject: TelemetrySavedObject;
}
export function getTelemetryUsageFetcher({
export function getTelemetrySendUsageFrom({
telemetrySavedObject,
configTelemetrySendUsageFrom,
}: GetTelemetryUsageFetcherConfig) {

View file

@ -19,5 +19,5 @@
export { replaceTelemetryInjectedVars } from './replace_injected_vars';
export { getTelemetryOptIn } from './get_telemetry_opt_in';
export { getTelemetryUsageFetcher } from './get_telemetry_usage_fetcher';
export { getTelemetrySendUsageFrom } from './get_telemetry_send_usage_from';
export { getTelemetryAllowChangingOptInStatus } from './get_telemetry_allow_changing_opt_in_status';

View file

@ -19,7 +19,7 @@
import { getTelemetrySavedObject } from '../telemetry_repository';
import { getTelemetryOptIn } from './get_telemetry_opt_in';
import { getTelemetryUsageFetcher } from './get_telemetry_usage_fetcher';
import { getTelemetrySendUsageFrom } from './get_telemetry_send_usage_from';
import { getTelemetryAllowChangingOptInStatus } from './get_telemetry_allow_changing_opt_in_status';
export async function replaceTelemetryInjectedVars(request: any) {
@ -51,7 +51,7 @@ export async function replaceTelemetryInjectedVars(request: any) {
currentKibanaVersion,
});
const telemetrySendUsageFrom = getTelemetryUsageFetcher({
const telemetrySendUsageFrom = getTelemetrySendUsageFrom({
configTelemetrySendUsageFrom,
telemetrySavedObject,
});

View file

@ -10,8 +10,7 @@ import { requireUIRoutes } from './routes';
import { instantiateClient } from './es_client/instantiate_client';
import { initMonitoringXpackInfo } from './init_monitoring_xpack_info';
import { initBulkUploader } from './kibana_monitoring';
import { telemetryCollectionManager } from '../../../../../src/legacy/core_plugins/telemetry/server';
import { getStatsWithMonitoring } from './telemetry_collection';
import { registerMonitoringCollection } from './telemetry_collection';
import {
getKibanaUsageCollector,
@ -38,8 +37,7 @@ export class Plugin {
}));
collectorSet.register(getKibanaUsageCollector({ collectorSet, config }));
collectorSet.register(getSettingsCollector({ collectorSet, config }));
telemetryCollectionManager.setStatsGetter(getStatsWithMonitoring, 'monitoring', 2);
registerMonitoringCollection();
/*
* Instantiate and start the internal background task that calls collector

View file

@ -11,7 +11,6 @@ import {
KIBANA_SYSTEM_ID,
BEATS_SYSTEM_ID,
} from '../../common/constants';
import { getClusterUuids } from './get_cluster_uuids';
import { getElasticsearchStats } from './get_es_stats';
import { getKibanaStats } from './get_kibana_stats';
import { getBeatsStats } from './get_beats_stats';
@ -26,22 +25,17 @@ import { getHighLevelStats } from './get_high_level_stats';
* @param {Date} end The ending range to request data
* @return {Promise} The array of clusters joined with the Kibana and Logstash instances.
*/
export function getAllStats({ server, callCluster, start, end } = {}) {
return getClusterUuids(server, callCluster, start, end)
.then(clusterUuids => {
// don't bother doing a further lookup
if (clusterUuids.length === 0) {
return [];
}
export async function getAllStats(clustersDetails, { server, callCluster, start, end }) {
const clusterUuids = clustersDetails.map(clusterDetails => clusterDetails.clusterUuid);
return Promise.all([
getElasticsearchStats(server, callCluster, clusterUuids), // cluster_stats, stack_stats.xpack, cluster_name/uuid, license, version
getKibanaStats(server, callCluster, clusterUuids, start, end), // stack_stats.kibana
getHighLevelStats(server, callCluster, clusterUuids, start, end, LOGSTASH_SYSTEM_ID), // stack_stats.logstash
getBeatsStats(server, callCluster, clusterUuids, start, end), // stack_stats.beats
])
.then(([esClusters, kibana, logstash, beats]) => handleAllStats(esClusters, { kibana, logstash, beats }));
});
const [esClusters, kibana, logstash, beats] = await Promise.all([
getElasticsearchStats(server, callCluster, clusterUuids), // cluster_stats, stack_stats.xpack, cluster_name/uuid, license, version
getKibanaStats(server, callCluster, clusterUuids, start, end), // stack_stats.kibana
getHighLevelStats(server, callCluster, clusterUuids, start, end, LOGSTASH_SYSTEM_ID), // stack_stats.logstash
getBeatsStats(server, callCluster, clusterUuids, start, end), // stack_stats.beats
]);
return handleAllStats(esClusters, { kibana, logstash, beats });
}
/**

View file

@ -5,33 +5,29 @@
*/
import { get } from 'lodash';
// @ts-ignore
import { createQuery } from './create_query';
// @ts-ignore
import { INDEX_PATTERN_ELASTICSEARCH } from '../../common/constants';
import {
ClusterDetailsGetter,
StatsCollectionConfig,
ClusterDetails,
} from '../../../../../../src/legacy/core_plugins/telemetry/server/collection_manager';
/**
* Get a list of Cluster UUIDs that exist within the specified timespan.
*
* @param {Object} server The server instance
* @param {function} callCluster The callWithRequest or callWithInternalUser handler
* @param {Date} start The start date to look for clusters
* @param {Date} end The end date to look for clusters
* @return {Array} Array of strings; one per Cluster UUID.
*/
export function getClusterUuids(server, callCluster, start, end) {
return fetchClusterUuids(server, callCluster, start, end)
.then(handleClusterUuidsResponse);
}
export const getClusterUuids: ClusterDetailsGetter = async config => {
const response = await fetchClusterUuids(config);
return handleClusterUuidsResponse(response);
};
/**
* Fetch the aggregated Cluster UUIDs from the monitoring cluster.
*
* @param {Object} server The server instance
* @param {function} callCluster The callWithRequest or callWithInternalUser handler
* @param {Date} start The start date to look for clusters
* @param {Date} end The end date to look for clusters
* @return {Promise} Object response from the aggregation.
*/
export function fetchClusterUuids(server, callCluster, start, end) {
export function fetchClusterUuids({ server, callCluster, start, end }: StatsCollectionConfig) {
const config = server.config();
const params = {
index: INDEX_PATTERN_ELASTICSEARCH,
@ -44,11 +40,11 @@ export function fetchClusterUuids(server, callCluster, start, end) {
cluster_uuids: {
terms: {
field: 'cluster_uuid',
size: config.get('xpack.monitoring.max_bucket_size')
}
}
}
}
size: config.get('xpack.monitoring.max_bucket_size'),
},
},
},
},
};
return callCluster('search', params);
@ -60,8 +56,10 @@ export function fetchClusterUuids(server, callCluster, start, end) {
* @param {Object} response The aggregation response
* @return {Array} Strings; each representing a Cluster's UUID.
*/
export function handleClusterUuidsResponse(response) {
const uuidBuckets = get(response, 'aggregations.cluster_uuids.buckets', []);
export function handleClusterUuidsResponse(response: any): ClusterDetails[] {
const uuidBuckets: any[] = get(response, 'aggregations.cluster_uuids.buckets', []);
return uuidBuckets.map(uuidBucket => uuidBucket.key);
return uuidBuckets.map(uuidBucket => ({
clusterUuid: uuidBucket.key as string,
}));
}

View file

@ -1,30 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
// @ts-ignore
import { getAllStats } from './get_all_stats';
import { getStatsWithXpack } from '../../../xpack_main/server/telemetry_collection';
import {
StatsGetter,
getStatsCollectionConfig,
} from '../../../../../../src/legacy/core_plugins/telemetry/server/collection_manager';
export const getStatsWithMonitoring: StatsGetter = async function(config) {
let response = [];
try {
const { start, end, server, callCluster } = getStatsCollectionConfig(config, 'monitoring');
response = await getAllStats({ server, callCluster, start, end });
} catch (err) {
// no-op
}
if (!Array.isArray(response) || response.length === 0) {
response = await getStatsWithXpack(config);
}
return response;
};

View file

@ -4,4 +4,4 @@
* you may not use this file except in compliance with the Elastic License.
*/
export { getStatsWithMonitoring } from './get_stats_with_monitoring';
export { registerMonitoringCollection } from './register_monitoring_collection';

View file

@ -0,0 +1,20 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
import { telemetryCollectionManager } from '../../../../../../src/legacy/core_plugins/telemetry/server';
// @ts-ignore
import { getAllStats } from './get_all_stats';
import { getClusterUuids } from './get_cluster_uuids';
export function registerMonitoringCollection() {
telemetryCollectionManager.setCollection({
esCluster: 'monitoring',
title: 'monitoring',
priority: 2,
statsGetter: getAllStats,
clusterDetailsGetter: getClusterUuids,
});
}

View file

@ -19,8 +19,7 @@ import { i18n } from '@kbn/i18n';
import { has } from 'lodash';
export { callClusterFactory } from './server/lib/call_cluster_factory';
import { getStatsWithXpack } from './server/telemetry_collection';
import { telemetryCollectionManager } from '../../../../src/legacy/core_plugins/telemetry/server';
import { registerMonitoringCollection } from './server/telemetry_collection';
export const xpackMain = (kibana) => {
return new kibana.Plugin({
@ -94,8 +93,7 @@ export const xpackMain = (kibana) => {
}
mirrorPluginStatus(server.plugins.elasticsearch, this, 'yellow', 'red');
telemetryCollectionManager.setStatsGetter(getStatsWithXpack, 'local_xpack', 1);
registerMonitoringCollection();
featuresPlugin.registerLegacyAPI({
xpackInfo: setupXPackMain(server),

View file

@ -7,19 +7,16 @@
// @ts-ignore
import { getXPack } from './get_xpack';
import { getLocalStats } from '../../../../../../src/legacy/core_plugins/telemetry/server/telemetry_collection';
import {
StatsGetter,
getStatsCollectionConfig,
} from '../../../../../../src/legacy/core_plugins/telemetry/server/collection_manager';
import { StatsGetter } from '../../../../../../src/legacy/core_plugins/telemetry/server/collection_manager';
export const getStatsWithXpack: StatsGetter = async function(config) {
const { server, callCluster } = getStatsCollectionConfig(config, 'data');
const localStats = await getLocalStats({ server, callCluster });
export const getStatsWithXpack: StatsGetter = async function(clustersDetails, config) {
const { callCluster } = config;
const clustersLocalStats = await getLocalStats(clustersDetails, config);
const { license, xpack } = await getXPack(callCluster);
localStats.license = license;
localStats.stack_stats.xpack = xpack;
return [localStats];
return clustersLocalStats.map(localStats => {
localStats.license = license;
localStats.stack_stats.xpack = xpack;
return localStats;
});
};

View file

@ -4,4 +4,4 @@
* you may not use this file except in compliance with the Elastic License.
*/
export { getStatsWithXpack } from './get_stats_with_xpack';
export { registerMonitoringCollection } from './register_xpack_collection';

View file

@ -0,0 +1,19 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
import { telemetryCollectionManager } from '../../../../../../src/legacy/core_plugins/telemetry/server';
import { getClusterUuids } from '../../../../../../src/legacy/core_plugins/telemetry/server/telemetry_collection';
import { getStatsWithXpack } from './get_stats_with_xpack';
export function registerMonitoringCollection() {
telemetryCollectionManager.setCollection({
esCluster: 'data',
title: 'local_xpack',
priority: 1,
statsGetter: getStatsWithXpack,
clusterDetailsGetter: getClusterUuids,
});
}