[Data] Add updated essql expression function (#132332)

* Add `essql` expression function to the data plugin
* Fix `phraseFilter` function name
* Update SQL search strategy to close the cursor when the search is complete
* Update SQL search strategy to return the time it took to run a search
This commit is contained in:
Michael Dokolin 2022-05-26 21:29:09 +02:00 committed by GitHub
parent 1bc354e28f
commit 103a5ef5ac
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
28 changed files with 671 additions and 214 deletions

View file

@ -130,7 +130,7 @@ const formatValue = (params: any[]) =>
export const buildRangeFilter = (
field: DataViewFieldBase,
params: RangeFilterParams,
indexPattern: DataViewBase,
indexPattern?: DataViewBase,
formattedValue?: string
): RangeFilter | ScriptedRangeFilter | MatchAllRangeFilter => {
params = mapValues(params, (value: any) => (field.type === 'number' ? parseFloat(value) : value));
@ -153,7 +153,7 @@ export const buildRangeFilter = (
}, 0);
const meta: RangeFilterMeta = {
index: indexPattern.id,
index: indexPattern?.id,
params: {},
field: field.name,
...(formattedValue ? { formattedValue } : {}),

View file

@ -80,6 +80,23 @@ describe('get_time', () => {
});
clock.restore();
});
test('build range filter when a data view is omitted', () => {
const filter = getTime(
undefined,
{ from: 'now-60y', to: 'now' },
{ fieldName: 'something' }
) as RangeFilter;
expect(filter).toHaveProperty(
'query.range.something',
expect.objectContaining({
gte: expect.any(String),
lte: expect.any(String),
format: 'strict_date_optional_time',
})
);
});
});
describe('getRelativeTime', () => {
test('do not coerce relative time to absolute time when given flag', () => {

View file

@ -6,6 +6,7 @@
* Side Public License, v 1.
*/
import { KBN_FIELD_TYPES } from '@kbn/field-types';
import dateMath from '@kbn/datemath';
import { omitBy } from 'lodash';
import { buildRangeFilter } from '@kbn/es-query';
@ -73,6 +74,18 @@ export function getRelativeTime(
);
}
function getTimeField(indexPattern?: IIndexPattern, fieldName?: string) {
if (!indexPattern && fieldName) {
return { name: fieldName, type: KBN_FIELD_TYPES.DATE };
}
if (!indexPattern) {
return;
}
return indexPattern.fields.find((f) => f.name === (fieldName || indexPattern.timeFieldName));
}
function createTimeRangeFilter(
indexPattern: IIndexPattern | undefined,
timeRange: TimeRange,
@ -80,12 +93,7 @@ function createTimeRangeFilter(
forceNow?: Date,
coerceRelativeTimeToAbsoluteTime: boolean = true
) {
if (!indexPattern) {
return;
}
const field = indexPattern.fields.find(
(f) => f.name === (fieldName || indexPattern.timeFieldName)
);
const field = getTimeField(indexPattern, fieldName);
if (!field) {
return;
}

View file

@ -0,0 +1,282 @@
/*
* 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.
*/
// eslint-disable-next-line @kbn/eslint/no-restricted-paths
import type { KibanaRequest } from '@kbn/core/server';
import { buildEsQuery } from '@kbn/es-query';
import { castEsToKbnFieldTypeName, ES_FIELD_TYPES, KBN_FIELD_TYPES } from '@kbn/field-types';
import { i18n } from '@kbn/i18n';
import {
Datatable,
DatatableColumnType,
ExpressionFunctionDefinition,
} from '@kbn/expressions-plugin/common';
import { RequestAdapter } from '@kbn/inspector-plugin/common';
import { zipObject } from 'lodash';
import { Observable, defer, throwError } from 'rxjs';
import { catchError, map, switchMap, tap } from 'rxjs/operators';
// eslint-disable-next-line @kbn/eslint/no-restricted-paths
import type { NowProviderPublicContract } from '../../../public';
import { getEsQueryConfig } from '../../es_query';
import { getTime } from '../../query';
import { UiSettingsCommon } from '../..';
import {
ISearchGeneric,
KibanaContext,
SqlRequestParams,
SqlSearchStrategyRequest,
SqlSearchStrategyResponse,
SQL_SEARCH_STRATEGY,
} from '..';
type Input = KibanaContext | null;
type Output = Observable<Datatable>;
interface Arguments {
query: string;
parameter: Array<string | number | boolean>;
count: number;
timezone: string;
timeField?: string;
}
export type EssqlExpressionFunctionDefinition = ExpressionFunctionDefinition<
'essql',
Input,
Arguments,
Output
>;
interface EssqlFnArguments {
getStartDependencies(getKibanaRequest: () => KibanaRequest): Promise<EssqlStartDependencies>;
}
interface EssqlStartDependencies {
nowProvider?: NowProviderPublicContract;
search: ISearchGeneric;
uiSettings: UiSettingsCommon;
}
function normalizeType(type: string): DatatableColumnType {
switch (type) {
case ES_FIELD_TYPES._INDEX:
case ES_FIELD_TYPES.GEO_POINT:
case ES_FIELD_TYPES.IP:
return KBN_FIELD_TYPES.STRING;
case '_version':
return KBN_FIELD_TYPES.NUMBER;
case 'datetime':
return KBN_FIELD_TYPES.DATE;
default:
return castEsToKbnFieldTypeName(type) as DatatableColumnType;
}
}
function sanitize(value: string) {
return value.replace(/[\(\)]/g, '_');
}
export const getEssqlFn = ({ getStartDependencies }: EssqlFnArguments) => {
const essql: EssqlExpressionFunctionDefinition = {
name: 'essql',
type: 'datatable',
inputTypes: ['kibana_context', 'null'],
help: i18n.translate('data.search.essql.help', {
defaultMessage: 'Queries Elasticsearch using Elasticsearch SQL.',
}),
args: {
query: {
aliases: ['_', 'q'],
types: ['string'],
help: i18n.translate('data.search.essql.query.help', {
defaultMessage: 'An Elasticsearch SQL query.',
}),
},
parameter: {
aliases: ['param'],
types: ['string', 'number', 'boolean'],
multi: true,
help: i18n.translate('data.search.essql.parameter.help', {
defaultMessage: 'A parameter to be passed to the SQL query.',
}),
},
count: {
types: ['number'],
help: i18n.translate('data.search.essql.count.help', {
defaultMessage:
'The number of documents to retrieve. For better performance, use a smaller data set.',
}),
default: 1000,
},
timezone: {
aliases: ['tz'],
types: ['string'],
default: 'UTC',
help: i18n.translate('data.search.essql.timezone.help', {
defaultMessage:
'The timezone to use for date operations. Valid ISO8601 formats and UTC offsets both work.',
}),
},
timeField: {
aliases: ['timeField'],
types: ['string'],
help: i18n.translate('data.search.essql.timeField.help', {
defaultMessage: 'The time field to use in the time range filter set in the context.',
}),
},
},
fn(
input,
{ count, parameter, query, timeField, timezone },
{ abortSignal, inspectorAdapters, getKibanaRequest }
) {
return defer(() =>
getStartDependencies(() => {
const request = getKibanaRequest?.();
if (!request) {
throw new Error(
'A KibanaRequest is required to run queries on the server. ' +
'Please provide a request object to the expression execution params.'
);
}
return request;
})
).pipe(
switchMap(({ nowProvider, search, uiSettings }) => {
const params: SqlRequestParams = {
query,
fetch_size: count,
time_zone: timezone,
params: parameter,
field_multi_value_leniency: true,
};
if (input) {
const esQueryConfigs = getEsQueryConfig(
uiSettings as Parameters<typeof getEsQueryConfig>[0]
);
const timeFilter =
input.timeRange &&
getTime(undefined, input.timeRange, {
fieldName: timeField,
forceNow: nowProvider?.get(),
});
params.filter = buildEsQuery(
undefined,
input.query || [],
[...(input.filters ?? []), ...(timeFilter ? [timeFilter] : [])],
esQueryConfigs
);
}
let startTime = Date.now();
const logInspectorRequest = () => {
if (!inspectorAdapters.requests) {
inspectorAdapters.requests = new RequestAdapter();
}
const request = inspectorAdapters.requests.start(
i18n.translate('data.search.dataRequest.title', {
defaultMessage: 'Data',
}),
{
description: i18n.translate('data.search.es_search.dataRequest.description', {
defaultMessage:
'This request queries Elasticsearch to fetch the data for the visualization.',
}),
},
startTime
);
startTime = Date.now();
return request;
};
return search<SqlSearchStrategyRequest, SqlSearchStrategyResponse>(
{ params },
{ abortSignal, strategy: SQL_SEARCH_STRATEGY }
).pipe(
catchError((error) => {
if (!error.err) {
error.message = `Unexpected error from Elasticsearch: ${error.message}`;
} else {
const { type, reason } = error.err.attributes;
if (type === 'parsing_exception') {
error.message = `Couldn't parse Elasticsearch SQL query. You may need to add double quotes to names containing special characters. Check your query and try again. Error: ${reason}`;
} else {
error.message = `Unexpected error from Elasticsearch: ${type} - ${reason}`;
}
}
return throwError(() => error);
}),
tap({
next({ rawResponse, took }) {
logInspectorRequest()
.stats({
hits: {
label: i18n.translate('data.search.es_search.hitsLabel', {
defaultMessage: 'Hits',
}),
value: `${rawResponse.rows.length}`,
description: i18n.translate('data.search.es_search.hitsDescription', {
defaultMessage: 'The number of documents returned by the query.',
}),
},
queryTime: {
label: i18n.translate('data.search.es_search.queryTimeLabel', {
defaultMessage: 'Query time',
}),
value: i18n.translate('data.search.es_search.queryTimeValue', {
defaultMessage: '{queryTime}ms',
values: { queryTime: took },
}),
description: i18n.translate('data.search.es_search.queryTimeDescription', {
defaultMessage:
'The time it took to process the query. ' +
'Does not include the time to send the request or parse it in the browser.',
}),
},
})
.json(params)
.ok({ json: rawResponse });
},
error(error) {
logInspectorRequest().error({ json: error });
},
})
);
}),
map(({ rawResponse: body }) => {
const columns =
body.columns?.map(({ name, type }) => ({
id: sanitize(name),
name: sanitize(name),
meta: { type: normalizeType(type) },
})) ?? [];
const columnNames = columns.map(({ name }) => name);
const rows = body.rows.map((row) => zipObject(columnNames, row));
return {
type: 'datatable',
meta: {
type: 'essql',
},
columns,
rows,
} as Datatable;
})
);
},
};
return essql;
};

View file

@ -19,14 +19,14 @@ interface Arguments {
}
export type ExpressionFunctionPhraseFilter = ExpressionFunctionDefinition<
'rangeFilter',
'phraseFilter',
null,
Arguments,
KibanaFilter
>;
export const phraseFilterFunction: ExpressionFunctionPhraseFilter = {
name: 'rangeFilter',
name: 'phraseFilter',
type: 'kibana_filter',
inputTypes: ['null'],
help: i18n.translate('data.search.functions.phraseFilter.help', {

View file

@ -15,9 +15,20 @@ import { IKibanaSearchRequest, IKibanaSearchResponse } from '../../types';
export const SQL_SEARCH_STRATEGY = 'sql';
export type SqlRequestParams =
export type SqlRequestParams = (
| Omit<SqlQueryRequest, 'keep_alive' | 'keep_on_completion'>
| Omit<SqlGetAsyncRequest, 'id' | 'keep_alive' | 'keep_on_completion'>;
| Omit<SqlGetAsyncRequest, 'id' | 'keep_alive' | 'keep_on_completion'>
) & {
/**
* Does not close the cursor on search completion.
*/
keep_cursor?: boolean;
};
export type SqlSearchStrategyRequest = IKibanaSearchRequest<SqlRequestParams>;
export type SqlSearchStrategyResponse = IKibanaSearchResponse<SqlQueryResponse>;
export interface SqlSearchStrategyResponse extends IKibanaSearchResponse<SqlQueryResponse> {
/**
* A metric showing how long did the search take.
*/
took: number;
}

View file

@ -265,7 +265,7 @@ export {
} from './query';
export { NowProvider } from './now_provider';
export type { NowProviderInternalContract } from './now_provider';
export type { NowProviderInternalContract, NowProviderPublicContract } from './now_provider';
export type {
QueryState,

View file

@ -0,0 +1,46 @@
/*
* 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 { StartServicesAccessor } from '@kbn/core/public';
import { UiSettingsCommon } from '../../../common';
import { DataPublicPluginStart, DataStartDependencies } from '../../types';
import { getEssqlFn } from '../../../common/search/expressions/essql';
/**
* This is some glue code that takes in `core.getStartServices`, extracts the dependencies
* needed for this function, and wraps them behind a `getStartDependencies` function that
* is then called at runtime.
*
* We do this so that we can be explicit about exactly which dependencies the function
* requires, without cluttering up the top-level `plugin.ts` with this logic. It also
* makes testing the expression function a bit easier since `getStartDependencies` is
* the only thing you should need to mock.
*
* @param getStartServices - core's StartServicesAccessor for this plugin
* @internal
*/
export function getEssql({
getStartServices,
}: {
getStartServices: StartServicesAccessor<DataStartDependencies, DataPublicPluginStart>;
}) {
return getEssqlFn({
async getStartDependencies() {
const [
{ uiSettings },
,
{
nowProvider,
search: { search },
},
] = await getStartServices();
return { nowProvider, search, uiSettings: uiSettings as unknown as UiSettingsCommon };
},
});
}

View file

@ -8,5 +8,6 @@
export * from './esaggs';
export * from './esdsl';
export * from './essql';
export * from '../../../common/search/expressions';
export * from './eql';

View file

@ -58,7 +58,7 @@ import { AggsService, AggsStartDependencies } from './aggs';
import { IKibanaSearchResponse, IndexPatternsContract, SearchRequest } from '..';
import { ISearchInterceptor, SearchInterceptor } from './search_interceptor';
import { createUsageCollector, SearchUsageCollector } from './collectors';
import { getEsaggs, getEsdsl, getEql } from './expressions';
import { getEsaggs, getEsdsl, getEssql, getEql } from './expressions';
import { ISessionsClient, ISessionService, SessionsClient, SessionService } from './session';
import { ConfigSchema } from '../../config';
import {
@ -172,6 +172,11 @@ export class SearchService implements Plugin<ISearchSetup, ISearchStart> {
getStartServices: StartServicesAccessor<DataStartDependencies, DataPublicPluginStart>;
})
);
expressions.registerFunction(
getEssql({ getStartServices } as {
getStartServices: StartServicesAccessor<DataStartDependencies, DataPublicPluginStart>;
})
);
expressions.registerFunction(
getEql({ getStartServices } as {
getStartServices: StartServicesAccessor<DataStartDependencies, DataPublicPluginStart>;

View file

@ -0,0 +1,43 @@
/*
* 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 { StartServicesAccessor } from '@kbn/core/server';
import { DataPluginStart, DataPluginStartDependencies } from '../../plugin';
import { getEssqlFn } from '../../../common/search/expressions/essql';
/**
* This is some glue code that takes in `core.getStartServices`, extracts the dependencies
* needed for this function, and wraps them behind a `getStartDependencies` function that
* is then called at runtime.
*
* We do this so that we can be explicit about exactly which dependencies the function
* requires, without cluttering up the top-level `plugin.ts` with this logic. It also
* makes testing the expression function a bit easier since `getStartDependencies` is
* the only thing you should need to mock.
*
* @param getStartServices - core's StartServicesAccessor for this plugin
* @internal
*/
export function getEssql({
getStartServices,
}: {
getStartServices: StartServicesAccessor<DataPluginStartDependencies, DataPluginStart>;
}) {
return getEssqlFn({
getStartDependencies: async (getKibanaRequest) => {
const [{ savedObjects, uiSettings }, , { search }] = await getStartServices();
const request = getKibanaRequest();
const savedObjectsClient = savedObjects.getScopedClient(request);
return {
search: search.asScoped(request).search,
uiSettings: uiSettings.asScopedToClient(savedObjectsClient),
};
},
});
}

View file

@ -8,4 +8,5 @@
export * from './esaggs';
export * from './esdsl';
export * from './essql';
export * from './eql';

View file

@ -85,7 +85,7 @@ import {
eqlRawResponse,
SQL_SEARCH_STRATEGY,
} from '../../common/search';
import { getEsaggs, getEsdsl, getEql } from './expressions';
import { getEsaggs, getEsdsl, getEssql, getEql } from './expressions';
import {
getShardDelayBucketAgg,
SHARD_DELAY_AGG_NAME,
@ -221,6 +221,7 @@ export class SearchService implements Plugin<ISearchSetup, ISearchStart> {
expressions.registerFunction(getEsaggs({ getStartServices: core.getStartServices }));
expressions.registerFunction(getEsdsl({ getStartServices: core.getStartServices }));
expressions.registerFunction(getEssql({ getStartServices: core.getStartServices }));
expressions.registerFunction(getEql({ getStartServices: core.getStartServices }));
expressions.registerFunction(cidrFunction);
expressions.registerFunction(dateRangeFunction);

View file

@ -14,6 +14,7 @@ import { SqlSearchStrategyResponse } from '../../../../common';
*/
export function toAsyncKibanaSearchResponse(
response: SqlQueryResponse,
startTime: number,
warning?: string
): SqlSearchStrategyResponse {
return {
@ -21,6 +22,7 @@ export function toAsyncKibanaSearchResponse(
rawResponse: response,
isPartial: response.is_partial,
isRunning: response.is_running,
took: Date.now() - startTime,
...(warning ? { warning } : {}),
};
}

View file

@ -6,6 +6,7 @@
* Side Public License, v 1.
*/
import { merge } from 'lodash';
import { KbnServerError } from '@kbn/kibana-utils-plugin/server';
import { errors } from '@elastic/elasticsearch';
import * as indexNotFoundException from '../../../../common/search/test_data/index_not_found_exception.json';
@ -27,6 +28,7 @@ describe('SQL search strategy', () => {
const mockSqlGetAsync = jest.fn();
const mockSqlQuery = jest.fn();
const mockSqlDelete = jest.fn();
const mockSqlClearCursor = jest.fn();
const mockLogger: any = {
debug: () => {},
};
@ -37,6 +39,7 @@ describe('SQL search strategy', () => {
getAsync: mockSqlGetAsync,
query: mockSqlQuery,
deleteAsync: mockSqlDelete,
clearCursor: mockSqlClearCursor,
},
},
},
@ -47,6 +50,7 @@ describe('SQL search strategy', () => {
mockSqlGetAsync.mockClear();
mockSqlQuery.mockClear();
mockSqlDelete.mockClear();
mockSqlClearCursor.mockClear();
});
it('returns a strategy with `search and `cancel`, `extend`', async () => {
@ -230,6 +234,52 @@ describe('SQL search strategy', () => {
expect(err?.message).toBe(errResponse.message);
expect(err?.errBody).toBe(undefined);
});
it('does not close the cursor if the search is incomplete', async () => {
mockSqlGetAsync.mockResolvedValueOnce(
merge({}, mockSqlResponse, {
body: { is_partial: false, is_running: true, cursor: 'cursor' },
})
);
const esSearch = await sqlSearchStrategyProvider(mockLogger);
await esSearch.search({ id: 'foo', params: { query: 'query' } }, {}, mockDeps).toPromise();
expect(mockSqlClearCursor).not.toHaveBeenCalled();
});
it('does not close the cursor if there is a request parameter to keep it', async () => {
mockSqlGetAsync.mockResolvedValueOnce(
merge({}, mockSqlResponse, { body: { cursor: 'cursor' } })
);
const esSearch = await sqlSearchStrategyProvider(mockLogger);
await esSearch
.search({ id: 'foo', params: { query: 'query', keep_cursor: true } }, {}, mockDeps)
.toPromise();
expect(mockSqlClearCursor).not.toHaveBeenCalled();
});
it('closes the cursor when the search is complete', async () => {
mockSqlGetAsync.mockResolvedValueOnce(
merge({}, mockSqlResponse, { body: { cursor: 'cursor' } })
);
const esSearch = await sqlSearchStrategyProvider(mockLogger);
await esSearch.search({ id: 'foo', params: { query: 'query' } }, {}, mockDeps).toPromise();
expect(mockSqlClearCursor).toHaveBeenCalledWith({ cursor: 'cursor' });
});
it('returns the time it took to run a search', async () => {
mockSqlGetAsync.mockResolvedValueOnce(mockSqlResponse);
const esSearch = await sqlSearchStrategyProvider(mockLogger);
await expect(
esSearch.search({ id: 'foo', params: { query: 'query' } }, {}, mockDeps).toPromise()
).resolves.toHaveProperty('took', expect.any(Number));
});
});
describe('cancel', () => {

View file

@ -6,9 +6,10 @@
* Side Public License, v 1.
*/
import type { IncomingHttpHeaders } from 'http';
import type { IScopedClusterClient, Logger } from '@kbn/core/server';
import { catchError, tap } from 'rxjs/operators';
import { SqlGetAsyncRequest, SqlQueryRequest } from '@elastic/elasticsearch/lib/api/types';
import { SqlQueryResponse } from '@elastic/elasticsearch/lib/api/types';
import { getKbnServerError } from '@kbn/kibana-utils-plugin/server';
import type { ISearchStrategy, SearchStrategyDependencies } from '../../types';
import type {
@ -39,6 +40,7 @@ export const sqlSearchStrategyProvider = (
{ esClient }: SearchStrategyDependencies
) {
const client = useInternalUser ? esClient.asInternalUser : esClient.asCurrentUser;
const startTime = Date.now();
// disable search sessions until session task manager supports SQL
// https://github.com/elastic/kibana/issues/127880
@ -46,33 +48,47 @@ export const sqlSearchStrategyProvider = (
const sessionConfig = null;
const search = async () => {
const { keep_cursor: keepCursor, ...params } = request.params ?? {};
let body: SqlQueryResponse;
let headers: IncomingHttpHeaders;
if (id) {
const params: SqlGetAsyncRequest = {
format: request.params?.format ?? 'json',
...getDefaultAsyncGetParams(sessionConfig, options),
id,
};
const { body, headers } = await client.sql.getAsync(params, {
signal: options.abortSignal,
meta: true,
});
return toAsyncKibanaSearchResponse(body, headers?.warning);
({ body, headers } = await client.sql.getAsync(
{
format: params?.format ?? 'json',
...getDefaultAsyncGetParams(sessionConfig, options),
id,
},
{
signal: options.abortSignal,
meta: true,
}
));
} else {
const params: SqlQueryRequest = {
format: request.params?.format ?? 'json',
...getDefaultAsyncSubmitParams(sessionConfig, options),
...request.params,
};
const { headers, body } = await client.sql.query(params, {
signal: options.abortSignal,
meta: true,
});
return toAsyncKibanaSearchResponse(body, headers?.warning);
({ headers, body } = await client.sql.query(
{
format: params.format ?? 'json',
...getDefaultAsyncSubmitParams(sessionConfig, options),
...params,
},
{
signal: options.abortSignal,
meta: true,
}
));
}
if (!body.is_partial && !body.is_running && body.cursor && !keepCursor) {
try {
await client.sql.clearCursor({ cursor: body.cursor });
} catch (error) {
logger.warn(
`sql search: failed to clear cursor=${body.cursor} for async_search_id=${id}: ${error.message}`
);
}
}
return toAsyncKibanaSearchResponse(body, startTime, headers?.warning);
};
const cancel = async () => {

View file

@ -0,0 +1 @@
{"columns":[{"id":"@timestamp","meta":{"type":"date"},"name":"@timestamp"},{"id":"@message","meta":{"type":"string"},"name":"@message"},{"id":"bytes","meta":{"type":"number"},"name":"bytes"}],"meta":{"type":"essql"},"rows":[{"@message":"143.84.142.7 - - [2015-09-20T00:00:00.000Z] \"GET /uploads/steven-hawley.jpg HTTP/1.1\" 200 1623 \"-\" \"Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1; SV1; .NET CLR 1.1.4322)\"","@timestamp":"2015-09-20T00:00:00.000Z","bytes":1623},{"@message":"193.164.192.47 - - [2015-09-20T00:30:34.206Z] \"GET /uploads/michael-foreman.jpg HTTP/1.1\" 200 8537 \"-\" \"Mozilla/5.0 (X11; Linux x86_64; rv:6.0a1) Gecko/20110421 Firefox/6.0a1\"","@timestamp":"2015-09-20T00:30:34.206Z","bytes":8537},{"@message":"176.7.244.68 - - [2015-09-20T00:32:42.058Z] \"GET /uploads/james-pawelczyk.jpg HTTP/1.1\" 200 9196 \"-\" \"Mozilla/5.0 (X11; Linux i686) AppleWebKit/534.24 (KHTML, like Gecko) Chrome/11.0.696.50 Safari/534.24\"","@timestamp":"2015-09-20T00:32:42.058Z","bytes":9196},{"@message":"237.56.90.184 - - [2015-09-20T00:35:21.445Z] \"GET /uploads/david-leestma.jpg HTTP/1.1\" 200 9790 \"-\" \"Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1; SV1; .NET CLR 1.1.4322)\"","@timestamp":"2015-09-20T00:35:21.445Z","bytes":9790},{"@message":"255.56.89.50 - - [2015-09-20T00:43:01.353Z] \"GET /uploads/michael-r-barratt.jpg HTTP/1.1\" 200 9583 \"-\" \"Mozilla/5.0 (X11; Linux x86_64; rv:6.0a1) Gecko/20110421 Firefox/6.0a1\"","@timestamp":"2015-09-20T00:43:01.353Z","bytes":9583}],"type":"datatable"}

View file

@ -0,0 +1 @@
{"columns":[{"id":"@timestamp","meta":{"type":"date"},"name":"@timestamp"},{"id":"@message","meta":{"type":"string"},"name":"@message"},{"id":"bytes","meta":{"type":"number"},"name":"bytes"}],"meta":{"type":"essql"},"rows":[{"@message":"143.84.142.7 - - [2015-09-20T00:00:00.000Z] \"GET /uploads/steven-hawley.jpg HTTP/1.1\" 200 1623 \"-\" \"Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1; SV1; .NET CLR 1.1.4322)\"","@timestamp":"2015-09-20T00:00:00.000Z","bytes":1623},{"@message":"193.164.192.47 - - [2015-09-20T00:30:34.206Z] \"GET /uploads/michael-foreman.jpg HTTP/1.1\" 200 8537 \"-\" \"Mozilla/5.0 (X11; Linux x86_64; rv:6.0a1) Gecko/20110421 Firefox/6.0a1\"","@timestamp":"2015-09-20T00:30:34.206Z","bytes":8537},{"@message":"176.7.244.68 - - [2015-09-20T00:32:42.058Z] \"GET /uploads/james-pawelczyk.jpg HTTP/1.1\" 200 9196 \"-\" \"Mozilla/5.0 (X11; Linux i686) AppleWebKit/534.24 (KHTML, like Gecko) Chrome/11.0.696.50 Safari/534.24\"","@timestamp":"2015-09-20T00:32:42.058Z","bytes":9196},{"@message":"237.56.90.184 - - [2015-09-20T00:35:21.445Z] \"GET /uploads/david-leestma.jpg HTTP/1.1\" 200 9790 \"-\" \"Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1; SV1; .NET CLR 1.1.4322)\"","@timestamp":"2015-09-20T00:35:21.445Z","bytes":9790},{"@message":"255.56.89.50 - - [2015-09-20T00:43:01.353Z] \"GET /uploads/michael-r-barratt.jpg HTTP/1.1\" 200 9583 \"-\" \"Mozilla/5.0 (X11; Linux x86_64; rv:6.0a1) Gecko/20110421 Firefox/6.0a1\"","@timestamp":"2015-09-20T00:43:01.353Z","bytes":9583}],"type":"datatable"}

View file

@ -0,0 +1,128 @@
/*
* 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 expect from '@kbn/expect';
import { ExpectExpression, expectExpressionProvider } from './helpers';
import { FtrProviderContext } from '../../../functional/ftr_provider_context';
export default function ({
getService,
updateBaselines,
}: FtrProviderContext & { updateBaselines: boolean }) {
const supertest = getService('supertest');
let expectExpression: ExpectExpression;
describe('essql', () => {
before(() => {
expectExpression = expectExpressionProvider({ getService, updateBaselines });
});
it('should run a query', async () => {
const expression = `
essql query='select "@message" from "logstash-*"'
`;
const result = await expectExpression('essql', expression).getResponse();
expect(result).to.have.property('type', 'datatable');
expect(result).to.have.property('columns');
expect(result).to.have.property('rows');
});
it('should return a datatable', async () => {
const expression = `
essql query='select "@timestamp", "@message", bytes from "logstash-*" order by "@timestamp"' count=5
`;
await expectExpression('essql', expression).toMatchSnapshot();
});
it('should run an aggregation query', async () => {
const expression = `
essql query='select count(*) as count from "logstash-*"'
`;
const result = await expectExpression('essql_count_aggregation', expression).getResponse();
expect(result?.rows?.[0]?.count).to.be(14004);
});
it('should respect the count parameter', async () => {
const expression = `
essql query='select "@message" from "logstash-*"' count=100
`;
const result = await expectExpression('essql_count_parameter', expression).getResponse();
expect(result).to.have.property('rows');
expect(result.rows).to.have.length(100);
});
it('should interpolate query parameters', async () => {
const expression = `
essql
query='select count(*) as count from "logstash-*" where "@timestamp" >= ? and "@timestamp" <= ?'
parameter="2006-09-21T00:00:00Z"
parameter="2015-09-22T00:00:00Z"
`;
const result = await expectExpression('essql', expression).getResponse();
expect(result?.rows?.[0]?.count).to.be(9375);
});
it('should return the same result on client and server', async () => {
const expression = `
essql query='select "@message" from "logstash-*"' count=10
`;
const client = await expectExpression('essql', expression).getResponse();
await supertest
.post('/api/interpreter_functional/run_expression')
.set('kbn-xsrf', 'anything')
.send({ expression, input: undefined })
.expect(200)
.expect(({ body }) => {
expect(body.rows).to.eql(client.rows);
});
});
it('should support `kibana_context` query on input', async () => {
const expression = `
kibana_context {kql "geo.src: US"}
| essql query='select count(*) as count from "logstash-*"'
`;
const result = await expectExpression('essql_kibana_context_query', expression).getResponse();
expect(result?.rows?.[0]?.count).to.be(1194);
});
it('should support `kibana_context` filters on input', async () => {
const expression = `
kibana_context filters={phraseFilter field={field name="geo.src" type="string"} phrase="US"}
| essql query='select count(*) as count from "logstash-*"'
`;
const result = await expectExpression(
'essql_kibana_context_filters',
expression
).getResponse();
expect(result?.rows?.[0]?.count).to.be(1194);
});
it('should support `kibana_context` time range on input', async () => {
const expression = `
kibana_context timeRange={timerange from="2006-09-21T00:00:00Z" to="2015-09-22T00:00:00Z"}
| essql query='select count(*) as count from "logstash-*"' timeField="@timestamp"
`;
const result = await expectExpression(
'essql_kibana_context_timerange',
expression
).getResponse();
expect(result?.rows?.[0]?.count).to.be(9375);
});
});
}

View file

@ -50,5 +50,6 @@ export default function ({ getService, getPageObjects, loadTestFile }: FtrProvid
loadTestFile(require.resolve('./esaggs_rareterms'));
loadTestFile(require.resolve('./esaggs_topmetrics'));
loadTestFile(require.resolve('./esaggs_histogram'));
loadTestFile(require.resolve('./essql'));
});
}

View file

@ -1,104 +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
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
import {
ExpressionFunctionDefinition,
ExpressionValueFilter,
} from '@kbn/expressions-plugin/common';
import { lastValueFrom } from 'rxjs';
import { searchService } from '../../../public/services';
import { ESSQL_SEARCH_STRATEGY } from '../../../common/lib/constants';
import { EssqlSearchStrategyRequest, EssqlSearchStrategyResponse } from '../../../types';
import { getFunctionHelp } from '../../../i18n';
interface Arguments {
query: string;
parameter: Array<string | number | boolean>;
count: number;
timezone: string;
}
export function essql(): ExpressionFunctionDefinition<
'essql',
ExpressionValueFilter,
Arguments,
any
> {
const { help, args: argHelp } = getFunctionHelp().essql;
return {
name: 'essql',
type: 'datatable',
context: {
types: ['filter'],
},
help,
args: {
query: {
aliases: ['_', 'q'],
types: ['string'],
help: argHelp.query,
},
parameter: {
aliases: ['param'],
types: ['string', 'number', 'boolean'],
multi: true,
help: argHelp.parameter,
},
count: {
types: ['number'],
help: argHelp.count,
default: 1000,
},
timezone: {
aliases: ['tz'],
types: ['string'],
default: 'UTC',
help: argHelp.timezone,
},
},
fn: (input, args, handlers) => {
const search = searchService.getService().search;
const { parameter, ...restOfArgs } = args;
const req = {
...restOfArgs,
params: parameter,
filter: input.and,
};
return lastValueFrom(
search.search<EssqlSearchStrategyRequest, EssqlSearchStrategyResponse>(req, {
strategy: ESSQL_SEARCH_STRATEGY,
})
)
.then((resp: EssqlSearchStrategyResponse) => {
return {
type: 'datatable',
meta: {
type: 'essql',
},
...resp,
};
})
.catch((e) => {
let message = `Unexpected error from Elasticsearch: ${e.message}`;
if (e.err) {
const { type, reason } = e.err.attributes;
if (type === 'parsing_exception') {
message = `Couldn't parse Elasticsearch SQL query. You may need to add double quotes to names containing special characters. Check your query and try again. Error: ${reason}`;
} else {
message = `Unexpected error from Elasticsearch: ${type} - ${reason}`;
}
}
// Re-write the error message before surfacing it up
e.message = message;
throw e;
});
},
};
}

View file

@ -11,6 +11,5 @@ import { markdown } from './markdown';
import { urlparam } from './urlparam';
import { escount } from './escount';
import { esdocs } from './esdocs';
import { essql } from './essql';
export const functions = [location, markdown, urlparam, escount, esdocs, essql, ...commonFunctions];
export const functions = [location, markdown, urlparam, escount, esdocs, ...commonFunctions];

View file

@ -49,7 +49,6 @@ export const TYPE_NULL = '`null`';
export const TYPE_NUMBER = '`number`';
export const TYPE_STRING = '`string`';
export const URL = 'URL';
export const UTC = 'UTC';
export const ZIP = 'ZIP';
export const IMAGE_MODE_CONTAIN = 'contain';
export const IMAGE_MODE_COVER = 'cover';

View file

@ -1,49 +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
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
import { i18n } from '@kbn/i18n';
import { essql } from '../../../canvas_plugin_src/functions/browser/essql';
import { FunctionHelp } from '../function_help';
import { FunctionFactory } from '../../../types';
import { ELASTICSEARCH, SQL, ISO8601, UTC } from '../../constants';
export const help: FunctionHelp<FunctionFactory<typeof essql>> = {
help: i18n.translate('xpack.canvas.functions.essqlHelpText', {
defaultMessage: 'Queries {ELASTICSEARCH} using {ELASTICSEARCH} {SQL}.',
values: {
ELASTICSEARCH,
SQL,
},
}),
args: {
query: i18n.translate('xpack.canvas.functions.essql.args.queryHelpText', {
defaultMessage: 'An {ELASTICSEARCH} {SQL} query.',
values: {
ELASTICSEARCH,
SQL,
},
}),
parameter: i18n.translate('xpack.canvas.functions.essql.args.parameterHelpText', {
defaultMessage: 'A parameter to be passed to the {SQL} query.',
values: {
SQL,
},
}),
count: i18n.translate('xpack.canvas.functions.essql.args.countHelpText', {
defaultMessage:
'The number of documents to retrieve. For better performance, use a smaller data set.',
}),
timezone: i18n.translate('xpack.canvas.functions.essql.args.timezoneHelpText', {
defaultMessage:
'The timezone to use for date operations. Valid {ISO8601} formats and {UTC} offsets both work.',
values: {
ISO8601,
UTC,
},
}),
},
};

View file

@ -30,7 +30,6 @@ import { help as eq } from './dict/eq';
import { help as embeddable } from './dict/embeddable';
import { help as escount } from './dict/escount';
import { help as esdocs } from './dict/esdocs';
import { help as essql } from './dict/essql';
import { help as exactly } from './dict/exactly';
import { help as filterrows } from './dict/filterrows';
import { help as filters } from './dict/filters';
@ -186,7 +185,6 @@ export const getFunctionHelp = (): FunctionHelpDict => ({
embeddable,
escount,
esdocs,
essql,
exactly,
filterrows,
filters,

View file

@ -2466,6 +2466,11 @@
"data.search.esdsl.index.help": "Index Elasticsearch à interroger",
"data.search.esdsl.q.help": "Requête DSL",
"data.search.esdsl.size.help": "Paramètre de taille de lAPI de recherche dElasticsearch",
"data.search.essql.count.help": "Nombre de documents à récupérer. Pour de meilleures performances, utilisez un ensemble de données plus petit.",
"data.search.essql.parameter.help": "Paramètre à transmettre à la requête SQL.",
"data.search.essql.query.help": "Requête Elasticsearch SQL.",
"data.search.essql.timezone.help": "Fuseau horaire à utiliser pour les opérations de date. Les formats ISO8601 et les décalages UTC valides fonctionnent.",
"data.search.essql.help": "Interroge Elasticsearch avec Elasticsearch SQL.",
"data.search.esErrorTitle": "Impossible dextraire les résultats de recherche",
"data.search.functions.cidr.cidr.help": "Spécifier le bloc CIDR",
"data.search.functions.cidr.help": "Créer une plage CIDR",
@ -8494,11 +8499,6 @@
"xpack.canvas.functions.esdocs.args.queryHelpText": "Chaîne de requête {LUCENE}.",
"xpack.canvas.functions.esdocs.args.sortHelpText": "Sens de tri au format {directions}. Par exemple, {example1} ou {example2}.",
"xpack.canvas.functions.esdocsHelpText": "Interrogez {ELASTICSEARCH} pour les documents bruts. Spécifiez les champs que vous souhaitez récupérer, surtout si vous demandez un nombre important de lignes.",
"xpack.canvas.functions.essql.args.countHelpText": "Nombre de documents à récupérer. Pour de meilleures performances, utilisez un ensemble de données plus petit.",
"xpack.canvas.functions.essql.args.parameterHelpText": "Paramètre à transmettre à la requête {SQL}.",
"xpack.canvas.functions.essql.args.queryHelpText": "Requête {ELASTICSEARCH} {SQL}.",
"xpack.canvas.functions.essql.args.timezoneHelpText": "Fuseau horaire à utiliser pour les opérations de date. Les formats {ISO8601} et les décalages {UTC} valides fonctionnent.",
"xpack.canvas.functions.essqlHelpText": "Interroge {ELASTICSEARCH} avec {ELASTICSEARCH} {SQL}.",
"xpack.canvas.functions.exactly.args.columnHelpText": "Colonne ou champ que vous souhaitez filtrer.",
"xpack.canvas.functions.exactly.args.filterGroupHelpText": "Nom du groupe pour le filtre.",
"xpack.canvas.functions.exactly.args.valueHelpText": "Valeur de correspondance exacte, comprenant les espaces blancs et la mise en majuscules.",

View file

@ -2560,6 +2560,11 @@
"data.search.esdsl.index.help": "クエリするElasticsearchインデックス",
"data.search.esdsl.q.help": "クエリDSL",
"data.search.esdsl.size.help": "Elasticsearch 検索 API サイズパラメーター",
"data.search.essql.count.help": "取得するドキュメント数です。パフォーマンスを向上させるには、小さなデータセットを使用します。",
"data.search.essql.parameter.help": "SQLクエリに渡すパラメーター。",
"data.search.essql.query.help": "Elasticsearch SQL クエリです。",
"data.search.essql.timezone.help": "日付操作の際に使用するタイムゾーンです。有効な ISO8601 フォーマットと UTC オフセットの両方が機能します。",
"data.search.essql.help": "Elasticsearch SQL を使用して Elasticsearch にクエリを実行します。",
"data.search.esErrorTitle": "検索結果を取得できません",
"data.search.functions.cidr.cidr.help": "CIDRブロックを指定",
"data.search.functions.cidr.help": "CIDRに基づく範囲を作成",
@ -8586,11 +8591,6 @@
"xpack.canvas.functions.esdocs.args.queryHelpText": "{LUCENE} クエリ文字列です。",
"xpack.canvas.functions.esdocs.args.sortHelpText": "{directions} フォーマットの並べ替え方向です。例:{example1} または {example2}。",
"xpack.canvas.functions.esdocsHelpText": "未加工ドキュメントの {ELASTICSEARCH} をクエリ特に多くの行を問い合わせる場合、取得するフィールドを指定してください。",
"xpack.canvas.functions.essql.args.countHelpText": "取得するドキュメント数です。パフォーマンスを向上させるには、小さなデータセットを使用します。",
"xpack.canvas.functions.essql.args.parameterHelpText": "{SQL}クエリに渡すパラメーター。",
"xpack.canvas.functions.essql.args.queryHelpText": "{ELASTICSEARCH} {SQL} クエリです。",
"xpack.canvas.functions.essql.args.timezoneHelpText": "日付操作の際に使用するタイムゾーンです。有効な {ISO8601} フォーマットと {UTC} オフセットの両方が機能します。",
"xpack.canvas.functions.essqlHelpText": "{ELASTICSEARCH} {SQL} を使用して {ELASTICSEARCH} にクエリを実行します。",
"xpack.canvas.functions.exactly.args.columnHelpText": "フィルタリングする列またはフィールドです。",
"xpack.canvas.functions.exactly.args.filterGroupHelpText": "フィルターのグループ名です。",
"xpack.canvas.functions.exactly.args.valueHelpText": "ホワイトスペースと大文字・小文字を含め、正確に一致させる値です。",

View file

@ -2568,6 +2568,11 @@
"data.search.esdsl.index.help": "要查询的 ElasticSearch 索引",
"data.search.esdsl.q.help": "查询 DSL",
"data.search.esdsl.size.help": "ElasticSearch searchAPI 大小参数",
"data.search.essql.count.help": "要检索的文档数目。要获取更佳的性能,请使用较小的数据集。",
"data.search.essql.parameter.help": "要传递给 SQL 查询的参数。",
"data.search.essql.query.help": "Elasticsearch SQL 查询。",
"data.search.essql.timezone.help": "要用于日期操作的时区。有效的 ISO8601 格式和 UTC 偏移均有效。",
"data.search.essql.help": "使用 Elasticsearch SQL 查询 Elasticsearch。",
"data.search.esErrorTitle": "无法检索搜索结果",
"data.search.functions.cidr.cidr.help": "指定 CIDR 块",
"data.search.functions.cidr.help": "创建基于 CIDR 的范围",
@ -8607,11 +8612,6 @@
"xpack.canvas.functions.esdocs.args.queryHelpText": "{LUCENE} 查询字符串。",
"xpack.canvas.functions.esdocs.args.sortHelpText": "格式为 {directions} 的排序方向。例如 {example1} 或 {example2}。",
"xpack.canvas.functions.esdocsHelpText": "查询 {ELASTICSEARCH} 以获取原始文档。指定要检索的字段,特别是需要大量的行。",
"xpack.canvas.functions.essql.args.countHelpText": "要检索的文档数目。要获取更佳的性能,请使用较小的数据集。",
"xpack.canvas.functions.essql.args.parameterHelpText": "要传递给 {SQL} 查询的参数。",
"xpack.canvas.functions.essql.args.queryHelpText": "{ELASTICSEARCH} {SQL} 查询。",
"xpack.canvas.functions.essql.args.timezoneHelpText": "要用于日期操作的时区。有效的 {ISO8601} 格式和 {UTC} 偏移均有效。",
"xpack.canvas.functions.essqlHelpText": "使用 {ELASTICSEARCH} {SQL} 查询 {ELASTICSEARCH}。",
"xpack.canvas.functions.exactly.args.columnHelpText": "要筛选的列或字段。",
"xpack.canvas.functions.exactly.args.filterGroupHelpText": "筛选的组名称。",
"xpack.canvas.functions.exactly.args.valueHelpText": "要完全匹配的值,包括空格和大写。",