Web console: improving the helper queries by allowing for running inline helper queries (#14801)

* remove helper queries

* fix tests

* take care of zero queries also

* switch to better place
This commit is contained in:
Vadim Ogievetsky 2023-08-16 23:50:43 -07:00 committed by GitHub
parent 5d4ac64178
commit dc2ae1e99c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 904 additions and 1480 deletions

View File

@ -36,7 +36,7 @@ import { waitTillWebConsoleReady } from './util/setup';
jest.setTimeout(5 * 60 * 1000); jest.setTimeout(5 * 60 * 1000);
const ALL_SORTS_OF_CHARS = '<>|!@#$%^&`\'".,:;\\*()[]{}Россия 한국 中国!?~'; const ALL_SORTS_OF_CHARS = '<>|!@#$%^&`\'".,:;\\*()[]{}Україна 한국 中国!?~';
describe('Tutorial: Loading a file', () => { describe('Tutorial: Loading a file', () => {
let browser: playwright.Browser; let browser: playwright.Browser;

View File

@ -39,7 +39,7 @@ export const QueryErrorPane = React.memo(function QueryErrorPane(props: QueryErr
return <div className="query-error-pane">{error.message}</div>; return <div className="query-error-pane">{error.message}</div>;
} }
const { position, suggestion } = error; const { startRowColumn, suggestion } = error;
let suggestionElement: JSX.Element | undefined; let suggestionElement: JSX.Element | undefined;
if (suggestion && queryString && onQueryStringChange) { if (suggestion && queryString && onQueryStringChange) {
const newQuery = suggestion.fn(queryString); const newQuery = suggestion.fn(queryString);
@ -69,14 +69,14 @@ export const QueryErrorPane = React.memo(function QueryErrorPane(props: QueryErr
)} )}
{error.errorMessageWithoutExpectation && ( {error.errorMessageWithoutExpectation && (
<p> <p>
{position ? ( {startRowColumn ? (
<HighlightText <HighlightText
text={error.errorMessageWithoutExpectation} text={error.errorMessageWithoutExpectation}
find={/\(line \[\d+], column \[\d+]\)/} find={/\(line \[\d+], column \[\d+]\)/}
replace={found => ( replace={found => (
<a <a
onClick={() => { onClick={() => {
moveCursorTo(position); moveCursorTo(startRowColumn);
}} }}
> >
{found} {found}

View File

@ -41,4 +41,3 @@ export * from './time/time';
export * from './timestamp-spec/timestamp-spec'; export * from './timestamp-spec/timestamp-spec';
export * from './transform-spec/transform-spec'; export * from './transform-spec/transform-spec';
export * from './workbench-query/workbench-query'; export * from './workbench-query/workbench-query';
export * from './workbench-query/workbench-query-part';

View File

@ -1,261 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 type { SqlValues, SqlWithQuery } from '@druid-toolkit/query';
import { SqlExpression, SqlQuery, T } from '@druid-toolkit/query';
import Hjson from 'hjson';
import * as JSONBig from 'json-bigint-native';
import type { ColumnMetadata } from '../../utils';
import { compact, filterMap, generate8HexId } from '../../utils';
import type { LastExecution } from '../execution/execution';
import { validateLastExecution } from '../execution/execution';
import { fitExternalConfigPattern } from '../external-config/external-config';
// -----------------------------
export interface WorkbenchQueryPartValue {
id: string;
queryName?: string;
queryString: string;
collapsed?: boolean;
lastExecution?: LastExecution;
}
export class WorkbenchQueryPart {
static blank() {
return new WorkbenchQueryPart({
id: generate8HexId(),
queryString: '',
});
}
static fromQuery(query: SqlQuery | SqlValues, queryName?: string, collapsed?: boolean) {
return this.fromQueryString(query.changeParens([]).toString(), queryName, collapsed);
}
static fromQueryString(queryString: string, queryName?: string, collapsed?: boolean) {
return new WorkbenchQueryPart({
id: generate8HexId(),
queryName,
queryString,
collapsed,
});
}
static isTaskEngineNeeded(queryString: string): boolean {
return /EXTERN\s*\(|(?:INSERT|REPLACE)\s+INTO/im.test(queryString);
}
static getIngestDatasourceFromQueryFragment(queryFragment: string): string | undefined {
// Assuming the queryFragment is no parsable find the prefix that look like:
// REPLACE<space>INTO<space><whatever><space>SELECT<space or EOF>
const matchInsertReplaceIndex = queryFragment.match(/(?:INSERT|REPLACE)\s+INTO/i)?.index;
if (typeof matchInsertReplaceIndex !== 'number') return;
const queryStartingWithInsertOrReplace = queryFragment.substring(matchInsertReplaceIndex);
const matchEnd = queryStartingWithInsertOrReplace.match(/\b(?:SELECT|WITH)\b|$/i);
const fragmentQuery = SqlQuery.maybeParse(
queryStartingWithInsertOrReplace.substring(0, matchEnd?.index) + ' SELECT * FROM t',
);
if (!fragmentQuery) return;
return fragmentQuery.getIngestTable()?.getName();
}
public readonly id: string;
public readonly queryName?: string;
public readonly queryString: string;
public readonly collapsed: boolean;
public readonly lastExecution?: LastExecution;
public readonly parsedQuery?: SqlQuery;
constructor(value: WorkbenchQueryPartValue) {
this.id = value.id;
this.queryName = value.queryName;
this.queryString = value.queryString;
this.collapsed = Boolean(value.collapsed);
this.lastExecution = validateLastExecution(value.lastExecution);
try {
this.parsedQuery = SqlQuery.parse(this.queryString);
} catch {}
}
public valueOf(): WorkbenchQueryPartValue {
return {
id: this.id,
queryName: this.queryName,
queryString: this.queryString,
collapsed: this.collapsed,
lastExecution: this.lastExecution,
};
}
public changeId(id: string): WorkbenchQueryPart {
return new WorkbenchQueryPart({ ...this.valueOf(), id });
}
public changeQueryName(queryName: string): WorkbenchQueryPart {
return new WorkbenchQueryPart({ ...this.valueOf(), queryName });
}
public changeQueryString(queryString: string): WorkbenchQueryPart {
return new WorkbenchQueryPart({ ...this.valueOf(), queryString });
}
public changeCollapsed(collapsed: boolean): WorkbenchQueryPart {
return new WorkbenchQueryPart({ ...this.valueOf(), collapsed });
}
public changeLastExecution(lastExecution: LastExecution | undefined): WorkbenchQueryPart {
return new WorkbenchQueryPart({ ...this.valueOf(), lastExecution });
}
public clear(): WorkbenchQueryPart {
return new WorkbenchQueryPart({
...this.valueOf(),
queryString: '',
});
}
public isEmptyQuery(): boolean {
return this.queryString.trim() === '';
}
public isJsonLike(): boolean {
return this.queryString.trim().startsWith('{');
}
public issueWithJson(): string | undefined {
try {
Hjson.parse(this.queryString);
} catch (e) {
return e.message;
}
return;
}
public isSqlInJson(): boolean {
try {
const query = Hjson.parse(this.queryString);
return typeof query.query === 'string';
} catch {
return false;
}
}
public getSqlString(): string {
if (this.isJsonLike()) {
const query = Hjson.parse(this.queryString);
return typeof query.query === 'string' ? query.query : '';
} else {
return this.queryString;
}
}
public prettyPrintJson(): WorkbenchQueryPart {
let parsed: unknown;
try {
parsed = Hjson.parse(this.queryString);
} catch {
return this;
}
return this.changeQueryString(JSONBig.stringify(parsed, undefined, 2));
}
public getIngestDatasource(): string | undefined {
const { queryString, parsedQuery } = this;
if (parsedQuery) {
return parsedQuery.getIngestTable()?.getName();
}
if (this.isJsonLike()) return;
return WorkbenchQueryPart.getIngestDatasourceFromQueryFragment(queryString);
}
public getInlineMetadata(): ColumnMetadata[] {
const { queryName, parsedQuery } = this;
if (queryName && parsedQuery) {
try {
return fitExternalConfigPattern(parsedQuery).signature.map(columnDeclaration => ({
COLUMN_NAME: columnDeclaration.getColumnName(),
DATA_TYPE: columnDeclaration.columnType.getEffectiveType(),
TABLE_NAME: queryName,
TABLE_SCHEMA: 'druid',
}));
} catch {
return filterMap(parsedQuery.getSelectExpressionsArray(), ex => {
const outputName = ex.getOutputName();
if (!outputName) return;
return {
COLUMN_NAME: outputName,
DATA_TYPE: 'UNKNOWN',
TABLE_NAME: queryName,
TABLE_SCHEMA: 'druid',
};
});
}
}
return [];
}
public isTaskEngineNeeded(): boolean {
return WorkbenchQueryPart.isTaskEngineNeeded(this.queryString);
}
public extractCteHelpers(): WorkbenchQueryPart[] | undefined {
let flatQuery: SqlQuery;
try {
// We need to do our own parsing here because this.parseQuery necessarily must be a SqlQuery
// object, and we might have a SqlWithQuery here.
flatQuery = (SqlExpression.parse(this.queryString) as SqlWithQuery).flattenWith();
} catch {
return;
}
const possibleNewParts = flatQuery.getWithParts().map(({ table, columns, query }) => {
if (columns) return;
return WorkbenchQueryPart.fromQuery(query, table.name, true);
});
if (!possibleNewParts.length) return;
const newParts = compact(possibleNewParts);
if (newParts.length !== possibleNewParts.length) return;
return newParts.concat(this.changeQueryString(flatQuery.changeWithParts(undefined).toString()));
}
public toWithPart(): string {
const { queryName, queryString } = this;
return `${T(queryName || 'q')} AS (\n${queryString}\n)`;
}
public duplicate(): WorkbenchQueryPart {
return this.changeId(generate8HexId()).changeLastExecution(undefined);
}
public addPreviewLimit(): WorkbenchQueryPart {
const { parsedQuery } = this;
if (!parsedQuery || parsedQuery.hasLimit()) return this;
return this.changeQueryString(parsedQuery.changeLimitValue(10000).toString());
}
}

View File

@ -19,7 +19,6 @@
import { sane } from '@druid-toolkit/query'; import { sane } from '@druid-toolkit/query';
import { WorkbenchQuery } from './workbench-query'; import { WorkbenchQuery } from './workbench-query';
import { WorkbenchQueryPart } from './workbench-query-part';
describe('WorkbenchQuery', () => { describe('WorkbenchQuery', () => {
beforeAll(() => { beforeAll(() => {
@ -107,12 +106,6 @@ describe('WorkbenchQuery', () => {
describe('.fromString', () => { describe('.fromString', () => {
const tabString = sane` const tabString = sane`
===== Helper: q =====
SELECT *
FROM wikipedia
===== Query ===== ===== Query =====
SELECT * FROM q SELECT * FROM q
@ -207,6 +200,7 @@ describe('WorkbenchQuery', () => {
expect(apiQuery).toEqual({ expect(apiQuery).toEqual({
cancelQueryId: 'deadbeef-9fb0-499c-8475-ea461e96a4fd', cancelQueryId: 'deadbeef-9fb0-499c-8475-ea461e96a4fd',
engine: 'native', engine: 'native',
prefixLines: 0,
query: { query: {
aggregations: [ aggregations: [
{ {
@ -258,6 +252,7 @@ describe('WorkbenchQuery', () => {
expect(apiQuery).toEqual({ expect(apiQuery).toEqual({
cancelQueryId: 'lol', cancelQueryId: 'lol',
engine: 'native', engine: 'native',
prefixLines: 0,
query: { query: {
aggregations: [ aggregations: [
{ {
@ -302,7 +297,7 @@ describe('WorkbenchQuery', () => {
sqlTypesHeader: true, sqlTypesHeader: true,
typesHeader: true, typesHeader: true,
}, },
sqlPrefixLines: 0, prefixLines: 0,
}); });
}); });
@ -328,7 +323,7 @@ describe('WorkbenchQuery', () => {
sqlTypesHeader: true, sqlTypesHeader: true,
typesHeader: true, typesHeader: true,
}, },
sqlPrefixLines: 0, prefixLines: 0,
}); });
}); });
@ -368,7 +363,7 @@ describe('WorkbenchQuery', () => {
sqlTypesHeader: true, sqlTypesHeader: true,
typesHeader: true, typesHeader: true,
}, },
sqlPrefixLines: 0, prefixLines: 0,
}); });
}); });
@ -407,7 +402,7 @@ describe('WorkbenchQuery', () => {
sqlTypesHeader: true, sqlTypesHeader: true,
typesHeader: true, typesHeader: true,
}, },
sqlPrefixLines: 0, prefixLines: 0,
}); });
}); });
@ -435,7 +430,7 @@ describe('WorkbenchQuery', () => {
sqlTypesHeader: true, sqlTypesHeader: true,
typesHeader: true, typesHeader: true,
}, },
sqlPrefixLines: 0, prefixLines: 0,
}); });
}); });
@ -536,130 +531,6 @@ describe('WorkbenchQuery', () => {
}); });
}); });
describe('#extractCteHelpers', () => {
it('works', () => {
const sql = sane`
REPLACE INTO task_statuses OVERWRITE ALL
WITH
task_statuses AS (
SELECT * FROM
TABLE(
EXTERN(
'{"type":"local","baseDir":"/Users/vadim/Desktop/","filter":"task_statuses.json"}',
'{"type":"json"}',
'[{"name":"id","type":"string"},{"name":"status","type":"string"},{"name":"duration","type":"long"},{"name":"errorMsg","type":"string"},{"name":"created_date","type":"string"}]'
)
)
)
(
--PLACE INTO task_statuses OVERWRITE ALL
SELECT
id,
status,
duration,
errorMsg,
created_date
FROM task_statuses
--RTITIONED BY ALL
)
PARTITIONED BY ALL
`;
expect(WorkbenchQuery.blank().changeQueryString(sql).extractCteHelpers().getQueryString())
.toEqual(sane`
REPLACE INTO task_statuses OVERWRITE ALL
SELECT
id,
status,
duration,
errorMsg,
created_date
FROM task_statuses
PARTITIONED BY ALL
`);
});
});
describe('#materializeHelpers', () => {
it('works', () => {
expect(
WorkbenchQuery.blank()
.changeQueryParts([
new WorkbenchQueryPart({
id: 'aaa',
queryName: 'kttm_data',
queryString: sane`
SELECT * FROM TABLE(
EXTERN(
'{"type":"http","uris":["https://static.imply.io/example-data/kttm-v2/kttm-v2-2019-08-25.json.gz"]}',
'{"type":"json"}'
)
) EXTEND ("timestamp" VARCHAR, "agent_type" VARCHAR)
`,
}),
new WorkbenchQueryPart({
id: 'bbb',
queryName: 'country_lookup',
queryString: sane`
SELECT * FROM TABLE(
EXTERN(
'{"type":"http","uris":["https://static.imply.io/example-data/lookup/countries.tsv"]}',
'{"type":"tsv","findColumnsFromHeader":true}'
)
) EXTEND ("Country" VARCHAR, "Capital" VARCHAR, "ISO3" VARCHAR, "ISO2" VARCHAR))
`,
}),
new WorkbenchQueryPart({
id: 'ccc',
queryName: 'x',
queryString: sane`
SELECT
os,
CONCAT(country, ' (', country_lookup.ISO3, ')') AS "country",
COUNT(DISTINCT session) AS "unique_sessions"
FROM kttm_data
LEFT JOIN country_lookup ON country_lookup.Country = kttm_data.country
GROUP BY 1, 2
ORDER BY 3 DESC
LIMIT 10
`,
}),
])
.materializeHelpers()
.getQueryString(),
).toEqual(sane`
WITH
"kttm_data" AS (
SELECT * FROM TABLE(
EXTERN(
'{"type":"http","uris":["https://static.imply.io/example-data/kttm-v2/kttm-v2-2019-08-25.json.gz"]}',
'{"type":"json"}'
)
) EXTEND ("timestamp" VARCHAR, "agent_type" VARCHAR)
),
"country_lookup" AS (
SELECT * FROM TABLE(
EXTERN(
'{"type":"http","uris":["https://static.imply.io/example-data/lookup/countries.tsv"]}',
'{"type":"tsv","findColumnsFromHeader":true}'
)
) EXTEND ("Country" VARCHAR, "Capital" VARCHAR, "ISO3" VARCHAR, "ISO2" VARCHAR))
)
(
SELECT
os,
CONCAT(country, ' (', country_lookup.ISO3, ')') AS "country",
COUNT(DISTINCT session) AS "unique_sessions"
FROM kttm_data
LEFT JOIN country_lookup ON country_lookup.Country = kttm_data.country
GROUP BY 1, 2
ORDER BY 3 DESC
LIMIT 10
)
`);
});
});
describe('#getIssue', () => { describe('#getIssue', () => {
it('works', () => { it('works', () => {
expect( expect(

View File

@ -20,7 +20,6 @@ import type {
SqlClusteredByClause, SqlClusteredByClause,
SqlExpression, SqlExpression,
SqlPartitionedByClause, SqlPartitionedByClause,
SqlQuery,
} from '@druid-toolkit/query'; } from '@druid-toolkit/query';
import { import {
C, C,
@ -28,17 +27,18 @@ import {
SqlLiteral, SqlLiteral,
SqlOrderByClause, SqlOrderByClause,
SqlOrderByExpression, SqlOrderByExpression,
SqlTable, SqlQuery,
} from '@druid-toolkit/query'; } from '@druid-toolkit/query';
import Hjson from 'hjson'; import Hjson from 'hjson';
import * as JSONBig from 'json-bigint-native'; import * as JSONBig from 'json-bigint-native';
import { v4 as uuidv4 } from 'uuid'; import { v4 as uuidv4 } from 'uuid';
import type { ColumnMetadata, RowColumn } from '../../utils'; import type { RowColumn } from '../../utils';
import { deleteKeys, generate8HexId } from '../../utils'; import { deleteKeys } from '../../utils';
import type { DruidEngine } from '../druid-engine/druid-engine'; import type { DruidEngine } from '../druid-engine/druid-engine';
import { validDruidEngine } from '../druid-engine/druid-engine'; import { validDruidEngine } from '../druid-engine/druid-engine';
import type { LastExecution } from '../execution/execution'; import type { LastExecution } from '../execution/execution';
import { validateLastExecution } from '../execution/execution';
import type { ExternalConfig } from '../external-config/external-config'; import type { ExternalConfig } from '../external-config/external-config';
import { import {
externalConfigToIngestQueryPattern, externalConfigToIngestQueryPattern,
@ -46,7 +46,7 @@ import {
} from '../ingest-query-pattern/ingest-query-pattern'; } from '../ingest-query-pattern/ingest-query-pattern';
import type { QueryContext } from '../query-context/query-context'; import type { QueryContext } from '../query-context/query-context';
import { WorkbenchQueryPart } from './workbench-query-part'; const ISSUE_MARKER = '--:ISSUE:';
export interface TabEntry { export interface TabEntry {
id: string; id: string;
@ -64,10 +64,15 @@ interface IngestionLines {
// ----------------------------- // -----------------------------
export interface WorkbenchQueryValue { export interface WorkbenchQueryValue {
queryParts: WorkbenchQueryPart[]; queryString: string;
queryContext: QueryContext; queryContext: QueryContext;
engine?: DruidEngine; engine?: DruidEngine;
lastExecution?: LastExecution;
unlimited?: boolean; unlimited?: boolean;
prefixLines?: number;
// Legacy
queryParts?: any[];
} }
export class WorkbenchQuery { export class WorkbenchQuery {
@ -75,8 +80,8 @@ export class WorkbenchQuery {
static blank(): WorkbenchQuery { static blank(): WorkbenchQuery {
return new WorkbenchQuery({ return new WorkbenchQuery({
queryString: '',
queryContext: {}, queryContext: {},
queryParts: [WorkbenchQueryPart.blank()],
}); });
} }
@ -87,19 +92,15 @@ export class WorkbenchQuery {
partitionedByHint: string | undefined, partitionedByHint: string | undefined,
): WorkbenchQuery { ): WorkbenchQuery {
return new WorkbenchQuery({ return new WorkbenchQuery({
queryContext: {}, queryString: ingestQueryPatternToQuery(
queryParts: [ externalConfigToIngestQueryPattern(
WorkbenchQueryPart.fromQueryString( externalConfig,
ingestQueryPatternToQuery( isArrays,
externalConfigToIngestQueryPattern( timeExpression,
externalConfig, partitionedByHint,
isArrays,
timeExpression,
partitionedByHint,
),
).toString(),
), ),
], ).toString(),
queryContext: {},
}); });
} }
@ -118,39 +119,21 @@ export class WorkbenchQuery {
} }
} }
const queryParts: WorkbenchQueryPart[] = []; let queryString = '';
let queryContext: QueryContext = {}; let queryContext: QueryContext = {};
for (let i = 0; i < headers.length; i++) { for (let i = 0; i < headers.length; i++) {
const header = headers[i]; const header = headers[i];
const body = bodies[i]; const body = bodies[i];
if (header === 'Context') { if (header === 'Context') {
queryContext = JSONBig.parse(body); queryContext = JSONBig.parse(body);
} else if (header.startsWith('Helper:')) {
queryParts.push(
new WorkbenchQueryPart({
id: generate8HexId(),
queryName: header.replace(/^Helper:/, '').trim(),
queryString: body,
collapsed: true,
}),
);
} else { } else {
queryParts.push( queryString = body;
new WorkbenchQueryPart({
id: generate8HexId(),
queryString: body,
}),
);
} }
} }
if (!queryParts.length) {
queryParts.push(WorkbenchQueryPart.blank());
}
return new WorkbenchQuery({ return new WorkbenchQuery({
queryString,
queryContext, queryContext,
queryParts,
}); });
} }
@ -229,20 +212,44 @@ export class WorkbenchQuery {
return { row: Number(m[1]) - 1, column: Number(m[2]) - 1 }; return { row: Number(m[1]) - 1, column: Number(m[2]) - 1 };
} }
public readonly queryParts: WorkbenchQueryPart[]; static isTaskEngineNeeded(queryString: string): boolean {
return /EXTERN\s*\(|(?:INSERT|REPLACE)\s+INTO/im.test(queryString);
}
static getIngestDatasourceFromQueryFragment(queryFragment: string): string | undefined {
// Assuming the queryFragment is no parsable find the prefix that look like:
// REPLACE<space>INTO<space><whatever><space>SELECT<space or EOF>
const matchInsertReplaceIndex = queryFragment.match(/(?:INSERT|REPLACE)\s+INTO/i)?.index;
if (typeof matchInsertReplaceIndex !== 'number') return;
const queryStartingWithInsertOrReplace = queryFragment.substring(matchInsertReplaceIndex);
const matchEnd = queryStartingWithInsertOrReplace.match(/\b(?:SELECT|WITH)\b|$/i);
const fragmentQuery = SqlQuery.maybeParse(
queryStartingWithInsertOrReplace.substring(0, matchEnd?.index) + ' SELECT * FROM t',
);
if (!fragmentQuery) return;
return fragmentQuery.getIngestTable()?.getName();
}
public readonly queryString: string;
public readonly queryContext: QueryContext; public readonly queryContext: QueryContext;
public readonly engine?: DruidEngine; public readonly engine?: DruidEngine;
public readonly lastExecution?: LastExecution;
public readonly unlimited?: boolean; public readonly unlimited?: boolean;
public readonly prefixLines?: number;
public readonly parsedQuery?: SqlQuery;
constructor(value: WorkbenchQueryValue) { constructor(value: WorkbenchQueryValue) {
let queryParts = value.queryParts; let queryString = value.queryString;
if (!Array.isArray(queryParts) || !queryParts.length) { // Back compat to read legacy workbench query
queryParts = [WorkbenchQueryPart.blank()]; if (typeof queryString === 'undefined' && Array.isArray(value.queryParts)) {
const lastQueryPart = value.queryParts[value.queryParts.length - 1];
queryString = lastQueryPart.queryString || '';
} }
if (!(queryParts instanceof WorkbenchQueryPart)) { this.queryString = queryString;
queryParts = queryParts.map(p => new WorkbenchQueryPart(p));
}
this.queryParts = queryParts;
this.queryContext = value.queryContext; this.queryContext = value.queryContext;
// Start back compat code for the engine names that might be coming from local storage // Start back compat code for the engine names that might be coming from local storage
@ -255,13 +262,17 @@ export class WorkbenchQuery {
// End bac compat code // End bac compat code
this.engine = validDruidEngine(possibleEngine) ? possibleEngine : undefined; this.engine = validDruidEngine(possibleEngine) ? possibleEngine : undefined;
this.lastExecution = validateLastExecution(value.lastExecution);
if (value.unlimited) this.unlimited = true; if (value.unlimited) this.unlimited = true;
this.prefixLines = value.prefixLines;
this.parsedQuery = SqlQuery.maybeParse(this.queryString);
} }
public valueOf(): WorkbenchQueryValue { public valueOf(): WorkbenchQueryValue {
return { return {
queryParts: this.queryParts, queryString: this.queryString,
queryContext: this.queryContext, queryContext: this.queryContext,
engine: this.engine, engine: this.engine,
unlimited: this.unlimited, unlimited: this.unlimited,
@ -269,21 +280,17 @@ export class WorkbenchQuery {
} }
public toString(): string { public toString(): string {
const { queryParts, queryContext } = this; const { queryString, queryContext } = this;
return queryParts return [
.slice(0, queryParts.length - 1) `===== Query =====`,
.flatMap(part => [`===== Helper: ${part.queryName} =====`, part.queryString]) queryString,
.concat([ `===== Context =====`,
`===== Query =====`, JSONBig.stringify(queryContext, undefined, 2),
this.getLastPart().queryString, ].join('\n\n');
`===== Context =====`,
JSONBig.stringify(queryContext, undefined, 2),
])
.join('\n\n');
} }
public changeQueryParts(queryParts: WorkbenchQueryPart[]): WorkbenchQuery { public changeQueryString(queryString: string): WorkbenchQuery {
return new WorkbenchQuery({ ...this.valueOf(), queryParts }); return new WorkbenchQuery({ ...this.valueOf(), queryString });
} }
public changeQueryContext(queryContext: QueryContext): WorkbenchQuery { public changeQueryContext(queryContext: QueryContext): WorkbenchQuery {
@ -294,20 +301,28 @@ export class WorkbenchQuery {
return new WorkbenchQuery({ ...this.valueOf(), engine }); return new WorkbenchQuery({ ...this.valueOf(), engine });
} }
public changeLastExecution(lastExecution: LastExecution | undefined): WorkbenchQuery {
return new WorkbenchQuery({ ...this.valueOf(), lastExecution });
}
public changeUnlimited(unlimited: boolean): WorkbenchQuery { public changeUnlimited(unlimited: boolean): WorkbenchQuery {
return new WorkbenchQuery({ ...this.valueOf(), unlimited }); return new WorkbenchQuery({ ...this.valueOf(), unlimited });
} }
public changePrefixLines(prefixLines: number): WorkbenchQuery {
return new WorkbenchQuery({ ...this.valueOf(), prefixLines });
}
public isTaskEngineNeeded(): boolean { public isTaskEngineNeeded(): boolean {
return this.queryParts.some(part => part.isTaskEngineNeeded()); return WorkbenchQuery.isTaskEngineNeeded(this.queryString);
} }
public getEffectiveEngine(): DruidEngine { public getEffectiveEngine(): DruidEngine {
const { engine } = this; const { engine } = this;
if (engine) return engine; if (engine) return engine;
const enabledEngines = WorkbenchQuery.getQueryEngines(); const enabledEngines = WorkbenchQuery.getQueryEngines();
if (this.getLastPart().isJsonLike()) { if (this.isJsonLike()) {
if (this.getLastPart().isSqlInJson()) { if (this.isSqlInJson()) {
if (enabledEngines.includes('sql-native')) return 'sql-native'; if (enabledEngines.includes('sql-native')) return 'sql-native';
} else { } else {
if (enabledEngines.includes('native')) return 'native'; if (enabledEngines.includes('native')) return 'native';
@ -318,61 +333,60 @@ export class WorkbenchQuery {
return enabledEngines[0] || 'sql-native'; return enabledEngines[0] || 'sql-native';
} }
private getLastPart(): WorkbenchQueryPart {
const { queryParts } = this;
return queryParts[queryParts.length - 1];
}
public getId(): string {
return this.getLastPart().id;
}
public getIds(): string[] {
return this.queryParts.map(queryPart => queryPart.id);
}
public getQueryName(): string {
return this.getLastPart().queryName || '';
}
public getQueryString(): string { public getQueryString(): string {
return this.getLastPart().queryString; return this.queryString;
}
public getCollapsed(): boolean {
return this.getLastPart().collapsed;
} }
public getLastExecution(): LastExecution | undefined { public getLastExecution(): LastExecution | undefined {
return this.getLastPart().lastExecution; return this.lastExecution;
} }
public getParsedQuery(): SqlQuery | undefined { public getParsedQuery(): SqlQuery | undefined {
return this.getLastPart().parsedQuery; return this.parsedQuery;
} }
public isEmptyQuery(): boolean { public isEmptyQuery(): boolean {
return this.getLastPart().isEmptyQuery(); return this.queryString.trim() === '';
} }
public getIssue(): string | undefined { public getIssue(): string | undefined {
const lastPart = this.getLastPart(); if (this.isJsonLike()) {
if (lastPart.isJsonLike()) { return this.issueWithJson();
return lastPart.issueWithJson();
} }
return; return;
} }
public isJsonLike(): boolean {
return this.queryString.trim().startsWith('{');
}
public issueWithJson(): string | undefined {
try {
Hjson.parse(this.queryString);
} catch (e) {
return e.message;
}
return;
}
public isSqlInJson(): boolean {
try {
const query = Hjson.parse(this.queryString);
return typeof query.query === 'string';
} catch {
return false;
}
}
public canPrettify(): boolean { public canPrettify(): boolean {
const lastPart = this.getLastPart(); return this.isJsonLike();
return lastPart.isJsonLike();
} }
public prettify(): WorkbenchQuery { public prettify(): WorkbenchQuery {
const lastPart = this.getLastPart(); const queryString = this.getQueryString();
let parsed; let parsed;
try { try {
parsed = Hjson.parse(lastPart.queryString); parsed = Hjson.parse(queryString);
} catch { } catch {
return this; return this;
} }
@ -381,96 +395,31 @@ export class WorkbenchQuery {
public getIngestDatasource(): string | undefined { public getIngestDatasource(): string | undefined {
if (this.getEffectiveEngine() !== 'sql-msq-task') return; if (this.getEffectiveEngine() !== 'sql-msq-task') return;
return this.getLastPart().getIngestDatasource();
const { queryString, parsedQuery } = this;
if (parsedQuery) {
return parsedQuery.getIngestTable()?.getName();
}
if (this.isJsonLike()) return;
return WorkbenchQuery.getIngestDatasourceFromQueryFragment(queryString);
} }
public isIngestQuery(): boolean { public isIngestQuery(): boolean {
return Boolean(this.getIngestDatasource()); return Boolean(this.getIngestDatasource());
} }
private changeLastQueryPart(lastQueryPart: WorkbenchQueryPart): WorkbenchQuery {
const { queryParts } = this;
return this.changeQueryParts(queryParts.slice(0, queryParts.length - 1).concat(lastQueryPart));
}
public changeQueryName(queryName: string): WorkbenchQuery {
return this.changeLastQueryPart(this.getLastPart().changeQueryName(queryName));
}
public changeQueryString(queryString: string): WorkbenchQuery {
return this.changeLastQueryPart(this.getLastPart().changeQueryString(queryString));
}
public changeCollapsed(collapsed: boolean): WorkbenchQuery {
return this.changeLastQueryPart(this.getLastPart().changeCollapsed(collapsed));
}
public changeLastExecution(lastExecution: LastExecution | undefined): WorkbenchQuery {
return this.changeLastQueryPart(this.getLastPart().changeLastExecution(lastExecution));
}
public clear(): WorkbenchQuery {
return new WorkbenchQuery({
queryParts: [],
queryContext: {},
});
}
public toggleUnlimited(): WorkbenchQuery { public toggleUnlimited(): WorkbenchQuery {
const { unlimited } = this; const { unlimited } = this;
return this.changeUnlimited(!unlimited); return this.changeUnlimited(!unlimited);
} }
public hasHelperQueries(): boolean {
return this.queryParts.length > 1;
}
public materializeHelpers(): WorkbenchQuery {
if (!this.hasHelperQueries()) return this;
const { query } = this.getApiQuery();
const queryString = query.query;
if (typeof queryString !== 'string') return this;
const lastPart = this.getLastPart();
return this.changeQueryParts([
new WorkbenchQueryPart({
id: lastPart.id,
queryName: lastPart.queryName,
queryString,
}),
]);
}
public extractCteHelpers(): WorkbenchQuery {
const { queryParts } = this;
let changed = false;
const newParts = queryParts.flatMap(queryPart => {
const helpers = queryPart.extractCteHelpers();
if (helpers) changed = true;
return helpers || [queryPart];
});
return changed ? this.changeQueryParts(newParts) : this;
}
public makePreview(): WorkbenchQuery { public makePreview(): WorkbenchQuery {
if (!this.isIngestQuery()) return this; if (!this.isIngestQuery()) return this;
let ret: WorkbenchQuery = this; let ret: WorkbenchQuery = this;
// Limit all the helper queries
const parsedQuery = this.getParsedQuery();
if (parsedQuery) {
const fromExpression = parsedQuery.getFirstFromExpression();
if (fromExpression instanceof SqlTable) {
const firstTable = fromExpression.getName();
ret = ret.changeQueryParts(
this.queryParts.map(queryPart =>
queryPart.queryName === firstTable ? queryPart.addPreviewLimit() : queryPart,
),
);
}
}
// Explicitly select MSQ, adjust the context, set maxNumTasks to the lowest possible and add in ingest mode flags // Explicitly select MSQ, adjust the context, set maxNumTasks to the lowest possible and add in ingest mode flags
ret = ret.changeEngine('sql-msq-task').changeQueryContext({ ret = ret.changeEngine('sql-msq-task').changeQueryContext({
...this.queryContext, ...this.queryContext,
@ -480,8 +429,8 @@ export class WorkbenchQuery {
}); });
// Remove everything pertaining to INSERT INTO / REPLACE INTO from the query string // Remove everything pertaining to INSERT INTO / REPLACE INTO from the query string
const newQueryString = parsedQuery const newQueryString = this.parsedQuery
? parsedQuery ? this.parsedQuery
.changeInsertClause(undefined) .changeInsertClause(undefined)
.changeReplaceClause(undefined) .changeReplaceClause(undefined)
.changePartitionedByClause(undefined) .changePartitionedByClause(undefined)
@ -502,18 +451,16 @@ export class WorkbenchQuery {
public getApiQuery(makeQueryId: () => string = uuidv4): { public getApiQuery(makeQueryId: () => string = uuidv4): {
engine: DruidEngine; engine: DruidEngine;
query: Record<string, any>; query: Record<string, any>;
sqlPrefixLines?: number; prefixLines: number;
cancelQueryId?: string; cancelQueryId?: string;
} { } {
const { queryParts, queryContext, unlimited } = this; const { queryString, queryContext, unlimited, prefixLines } = this;
if (!queryParts.length) throw new Error(`should not get here`);
const engine = this.getEffectiveEngine(); const engine = this.getEffectiveEngine();
const lastQueryPart = this.getLastPart();
if (engine === 'native') { if (engine === 'native') {
let query: any; let query: any;
try { try {
query = Hjson.parse(lastQueryPart.queryString); query = Hjson.parse(queryString);
} catch (e) { } catch (e) {
throw new Error( throw new Error(
`You have selected the 'native' engine but the query you entered could not be parsed as JSON: ${e.message}`, `You have selected the 'native' engine but the query you entered could not be parsed as JSON: ${e.message}`,
@ -530,24 +477,21 @@ export class WorkbenchQuery {
return { return {
engine, engine,
query, query,
prefixLines: prefixLines || 0,
cancelQueryId, cancelQueryId,
}; };
} }
const prefixParts = queryParts
.slice(0, queryParts.length - 1)
.filter(part => !part.getIngestDatasource());
let apiQuery: Record<string, any> = {}; let apiQuery: Record<string, any> = {};
if (lastQueryPart.isJsonLike()) { if (this.isJsonLike()) {
try { try {
apiQuery = Hjson.parse(lastQueryPart.queryString); apiQuery = Hjson.parse(queryString);
} catch (e) { } catch (e) {
throw new Error(`The query you entered could not be parsed as JSON: ${e.message}`); throw new Error(`The query you entered could not be parsed as JSON: ${e.message}`);
} }
} else { } else {
apiQuery = { apiQuery = {
query: lastQueryPart.queryString, query: queryString,
resultFormat: 'array', resultFormat: 'array',
header: true, header: true,
typesHeader: true, typesHeader: true,
@ -555,42 +499,13 @@ export class WorkbenchQuery {
}; };
} }
let queryPrepend = ''; const issueIndex = String(apiQuery.query).indexOf(ISSUE_MARKER);
let queryAppend = ''; if (issueIndex !== -1) {
const issueComment = String(apiQuery.query)
if (prefixParts.length) { .slice(issueIndex + ISSUE_MARKER.length)
const { insertReplaceLine, overwriteLine, partitionedByLine, clusteredByLine } = .split('\n')[0];
WorkbenchQuery.getIngestionLines(apiQuery.query);
if (insertReplaceLine) {
queryPrepend += insertReplaceLine + '\n';
if (overwriteLine) {
queryPrepend += overwriteLine + '\n';
}
apiQuery.query = WorkbenchQuery.commentOutIngestParts(apiQuery.query);
if (clusteredByLine) {
queryAppend = '\n' + clusteredByLine + queryAppend;
}
if (partitionedByLine) {
queryAppend = '\n' + partitionedByLine + queryAppend;
}
}
queryPrepend += 'WITH\n' + prefixParts.map(p => p.toWithPart()).join(',\n') + '\n(\n';
queryAppend = '\n)' + queryAppend;
}
let prefixLines = 0;
if (queryPrepend) {
prefixLines = queryPrepend.split('\n').length - 1;
apiQuery.query = queryPrepend + apiQuery.query + queryAppend;
}
const m = /--:ISSUE:(.+)(?:\n|$)/.exec(apiQuery.query);
if (m) {
throw new Error( throw new Error(
`This query contains an ISSUE comment: ${m[1] `This query contains an ISSUE comment: ${issueComment
.trim() .trim()
.replace( .replace(
/\.$/, /\.$/,
@ -628,55 +543,8 @@ export class WorkbenchQuery {
return { return {
engine, engine,
query: apiQuery, query: apiQuery,
sqlPrefixLines: prefixLines, prefixLines: prefixLines || 0,
cancelQueryId, cancelQueryId,
}; };
} }
public getInlineMetadata(): ColumnMetadata[] {
const { queryParts } = this;
if (!queryParts.length) return [];
return queryParts.slice(0, queryParts.length - 1).flatMap(p => p.getInlineMetadata());
}
public getPrefix(index: number): WorkbenchQuery {
return this.changeQueryParts(this.queryParts.slice(0, index + 1));
}
public getPrefixQueries(): WorkbenchQuery[] {
return this.queryParts.slice(0, this.queryParts.length - 1).map((_, i) => this.getPrefix(i));
}
public applyUpdate(newQuery: WorkbenchQuery, index: number): WorkbenchQuery {
return newQuery.changeQueryParts(newQuery.queryParts.concat(this.queryParts.slice(index + 1)));
}
public duplicate(): WorkbenchQuery {
return this.changeQueryParts(this.queryParts.map(part => part.duplicate()));
}
public duplicateLast(): WorkbenchQuery {
const { queryParts } = this;
const last = this.getLastPart();
return this.changeQueryParts(queryParts.concat(last.duplicate()));
}
public addBlank(): WorkbenchQuery {
const { queryParts } = this;
const last = this.getLastPart();
return this.changeQueryParts(
queryParts.slice(0, queryParts.length - 1).concat(
last
.changeQueryName(last.queryName || 'q')
.changeCollapsed(true)
.changeLastExecution(undefined),
WorkbenchQueryPart.blank(),
),
);
}
public remove(index: number): WorkbenchQuery {
const { queryParts } = this;
return this.changeQueryParts(queryParts.filter((_, i) => i !== index));
}
} }

View File

@ -40,9 +40,7 @@ export class AceEditorStateCache {
session.setUndoManager(state.undoManager); session.setUndoManager(state.undoManager);
} }
static deleteStates(ids: string[]): void { static deleteState(id: string): void {
for (const id of ids) { delete AceEditorStateCache.states[id];
delete AceEditorStateCache.states[id];
}
} }
} }

View File

@ -33,10 +33,4 @@ export class ExecutionStateCache {
static deleteState(id: string): void { static deleteState(id: string): void {
delete ExecutionStateCache.cache[id]; delete ExecutionStateCache.cache[id];
} }
static deleteStates(ids: string[]): void {
for (const id of ids) {
delete ExecutionStateCache.cache[id];
}
}
} }

View File

@ -20,7 +20,7 @@ import type { QueryResult } from '@druid-toolkit/query';
export interface WorkbenchRunningPromise { export interface WorkbenchRunningPromise {
promise: Promise<QueryResult>; promise: Promise<QueryResult>;
sqlPrefixLines: number | undefined; prefixLines: number;
} }
export class WorkbenchRunningPromises { export class WorkbenchRunningPromises {
@ -41,10 +41,4 @@ export class WorkbenchRunningPromises {
static deletePromise(id: string): void { static deletePromise(id: string): void {
delete WorkbenchRunningPromises.promises[id]; delete WorkbenchRunningPromises.promises[id];
} }
static deletePromises(ids: string[]): void {
for (const id of ids) {
delete WorkbenchRunningPromises.promises[id];
}
}
} }

View File

@ -21,10 +21,10 @@ import { sane } from '@druid-toolkit/query';
import { DruidError, getDruidErrorMessage } from './druid-query'; import { DruidError, getDruidErrorMessage } from './druid-query';
describe('DruidQuery', () => { describe('DruidQuery', () => {
describe('DruidError.parsePosition', () => { describe('DruidError.extractStartRowColumn', () => {
it('works for single error 1', () => { it('works for single error 1', () => {
expect( expect(
DruidError.extractPosition({ DruidError.extractStartRowColumn({
sourceType: 'sql', sourceType: 'sql',
line: '2', line: '2',
column: '12', column: '12',
@ -39,7 +39,7 @@ describe('DruidQuery', () => {
it('works for range', () => { it('works for range', () => {
expect( expect(
DruidError.extractPosition({ DruidError.extractStartRowColumn({
sourceType: 'sql', sourceType: 'sql',
line: '1', line: '1',
column: '16', column: '16',
@ -51,8 +51,37 @@ describe('DruidQuery', () => {
).toEqual({ ).toEqual({
row: 0, row: 0,
column: 15, column: 15,
endRow: 0, });
endColumn: 16, });
});
describe('DruidError.extractEndRowColumn', () => {
it('works for single error 1', () => {
expect(
DruidError.extractEndRowColumn({
sourceType: 'sql',
line: '2',
column: '12',
token: "AS \\'l\\'",
expected: '...',
}),
).toBeUndefined();
});
it('works for range', () => {
expect(
DruidError.extractEndRowColumn({
sourceType: 'sql',
line: '1',
column: '16',
endLine: '1',
endColumn: '17',
token: "AS \\'l\\'",
expected: '...',
}),
).toEqual({
row: 0,
column: 16,
}); });
}); });
}); });

View File

@ -22,8 +22,8 @@ import axios from 'axios';
import { Api } from '../singletons'; import { Api } from '../singletons';
import type { RowColumn } from './general';
import { assemble } from './general'; import { assemble } from './general';
import type { RowColumn } from './query-cursor';
const CANCELED_MESSAGE = 'Query canceled by user.'; const CANCELED_MESSAGE = 'Query canceled by user.';
@ -109,20 +109,28 @@ export function getDruidErrorMessage(e: any): string {
} }
export class DruidError extends Error { export class DruidError extends Error {
static extractPosition(context: Record<string, any> | undefined): RowColumn | undefined { static extractStartRowColumn(
context: Record<string, any> | undefined,
offsetLines = 0,
): RowColumn | undefined {
if (context?.sourceType !== 'sql' || !context.line || !context.column) return; if (context?.sourceType !== 'sql' || !context.line || !context.column) return;
const rowColumn: RowColumn = { return {
row: Number(context.line) - 1, row: Number(context.line) - 1 + offsetLines,
column: Number(context.column) - 1, column: Number(context.column) - 1,
}; };
}
if (context.endLine && context.endColumn) { static extractEndRowColumn(
rowColumn.endRow = Number(context.endLine) - 1; context: Record<string, any> | undefined,
rowColumn.endColumn = Number(context.endColumn) - 1; offsetLines = 0,
} ): RowColumn | undefined {
if (context?.sourceType !== 'sql' || !context.endLine || !context.endColumn) return;
return rowColumn; return {
row: Number(context.endLine) - 1 + offsetLines,
column: Number(context.endColumn) - 1,
};
} }
static positionToIndex(str: string, line: number, column: number): number { static positionToIndex(str: string, line: number, column: number): number {
@ -256,7 +264,8 @@ export class DruidError extends Error {
public errorMessage?: string; public errorMessage?: string;
public errorMessageWithoutExpectation?: string; public errorMessageWithoutExpectation?: string;
public expectation?: string; public expectation?: string;
public position?: RowColumn; public startRowColumn?: RowColumn;
public endRowColumn?: RowColumn;
public suggestion?: QuerySuggestion; public suggestion?: QuerySuggestion;
// Deprecated // Deprecated
@ -264,7 +273,7 @@ export class DruidError extends Error {
public errorClass?: string; public errorClass?: string;
public host?: string; public host?: string;
constructor(e: any, skipLines = 0) { constructor(e: any, offsetLines = 0) {
super(axios.isCancel(e) ? CANCELED_MESSAGE : getDruidErrorMessage(e)); super(axios.isCancel(e) ? CANCELED_MESSAGE : getDruidErrorMessage(e));
if (axios.isCancel(e)) { if (axios.isCancel(e)) {
this.canceled = true; this.canceled = true;
@ -286,14 +295,15 @@ export class DruidError extends Error {
Object.assign(this, druidErrorResponse); Object.assign(this, druidErrorResponse);
if (this.errorMessage) { if (this.errorMessage) {
if (skipLines) { if (offsetLines) {
this.errorMessage = this.errorMessage.replace( this.errorMessage = this.errorMessage.replace(
/line \[(\d+)],/g, /line \[(\d+)],/g,
(_, c) => `line [${Number(c) - skipLines}],`, (_, c) => `line [${Number(c) + offsetLines}],`,
); );
} }
this.position = DruidError.extractPosition(this.context); this.startRowColumn = DruidError.extractStartRowColumn(this.context, offsetLines);
this.endRowColumn = DruidError.extractEndRowColumn(this.context, offsetLines);
this.suggestion = DruidError.getSuggestion(this.errorMessage); this.suggestion = DruidError.getSuggestion(this.errorMessage);
const expectationIndex = this.errorMessage.indexOf('Was expecting one of'); const expectationIndex = this.errorMessage.indexOf('Was expecting one of');

View File

@ -175,15 +175,24 @@ describe('general', () => {
describe('offsetToRowColumn', () => { describe('offsetToRowColumn', () => {
it('works', () => { it('works', () => {
expect(offsetToRowColumn('Hello\nThis is a test\nstring.', -6)).toBeUndefined(); const str = 'Hello\nThis is a test\nstring.';
expect(offsetToRowColumn('Hello\nThis is a test\nstring.', 666)).toBeUndefined(); expect(offsetToRowColumn(str, -6)).toBeUndefined();
expect(offsetToRowColumn('Hello\nThis is a test\nstring.', 3)).toEqual({ expect(offsetToRowColumn(str, 666)).toBeUndefined();
column: 3, expect(offsetToRowColumn(str, 3)).toEqual({
row: 0, row: 0,
});
expect(offsetToRowColumn('Hello\nThis is a test\nstring.', 24)).toEqual({
column: 3, column: 3,
});
expect(offsetToRowColumn(str, 5)).toEqual({
row: 0,
column: 5,
});
expect(offsetToRowColumn(str, 24)).toEqual({
row: 2, row: 2,
column: 3,
});
expect(offsetToRowColumn(str, str.length)).toEqual({
row: 2,
column: 7,
}); });
}); });
}); });

View File

@ -156,7 +156,7 @@ function identity<T>(x: T): T {
export function lookupBy<T, Q = T>( export function lookupBy<T, Q = T>(
array: readonly T[], array: readonly T[],
keyFn: (x: T, index: number) => string = String, keyFn: (x: T, index: number) => string | number = String,
valueFn?: (x: T, index: number) => Q, valueFn?: (x: T, index: number) => Q,
): Record<string, Q> { ): Record<string, Q> {
if (!valueFn) valueFn = identity as any; if (!valueFn) valueFn = identity as any;
@ -521,17 +521,19 @@ export function generate8HexId(): string {
return (Math.random() * 1e10).toString(16).replace('.', '').slice(0, 8); return (Math.random() * 1e10).toString(16).replace('.', '').slice(0, 8);
} }
export function offsetToRowColumn( export interface RowColumn {
str: string, row: number;
offset: number, column: number;
): { row: number; column: number } | undefined { }
export function offsetToRowColumn(str: string, offset: number): RowColumn | undefined {
// Ensure offset is within the string length // Ensure offset is within the string length
if (offset < 0 || offset > str.length) return; if (offset < 0 || offset > str.length) return;
const lines = str.split('\n'); const lines = str.split('\n');
for (let row = 0; row < lines.length; row++) { for (let row = 0; row < lines.length; row++) {
const line = lines[row]; const line = lines[row];
if (offset < line.length) { if (offset <= line.length) {
return { return {
row, row,
column: offset, column: offset,

View File

@ -19,6 +19,8 @@
import type { SqlBase, SqlQuery } from '@druid-toolkit/query'; import type { SqlBase, SqlQuery } from '@druid-toolkit/query';
import { L } from '@druid-toolkit/query'; import { L } from '@druid-toolkit/query';
import type { RowColumn } from './general';
export const EMPTY_LITERAL = L(''); export const EMPTY_LITERAL = L('');
const CRAZY_STRING = '$.X.@.X.$'; const CRAZY_STRING = '$.X.@.X.$';
@ -36,13 +38,6 @@ export function prettyPrintSql(b: SqlBase): string {
.toString(); .toString();
} }
export interface RowColumn {
row: number;
column: number;
endRow?: number;
endColumn?: number;
}
export function findEmptyLiteralPosition(query: SqlQuery): RowColumn | undefined { export function findEmptyLiteralPosition(query: SqlQuery): RowColumn | undefined {
const subQueryString = query.walk(b => (b === EMPTY_LITERAL ? L(CRAZY_STRING) : b)).toString(); const subQueryString = query.walk(b => (b === EMPTY_LITERAL ? L(CRAZY_STRING) : b)).toString();
@ -54,7 +49,7 @@ export function findEmptyLiteralPosition(query: SqlQuery): RowColumn | undefined
const row = lines.length - 1; const row = lines.length - 1;
const lastLine = lines[row]; const lastLine = lines[row];
return { return {
row, row: row,
column: lastLine.length, column: lastLine.length,
}; };
} }

View File

@ -0,0 +1,349 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 { sane } from '@druid-toolkit/query';
import { findAllSqlQueriesInText, findSqlQueryPrefix } from './sql';
describe('sql', () => {
describe('getSqlQueryPrefix', () => {
it('works when whole query parses', () => {
expect(
findSqlQueryPrefix(sane`
SELECT *
FROM wikipedia
`),
).toMatchInlineSnapshot(`
"SELECT *
FROM wikipedia"
`);
});
it('works when there are two queries', () => {
expect(
findSqlQueryPrefix(sane`
SELECT *
FROM wikipedia
SELECT *
FROM w2
`),
).toMatchInlineSnapshot(`
"SELECT *
FROM wikipedia"
`);
});
it('works when there are extra closing parens', () => {
expect(
findSqlQueryPrefix(sane`
SELECT *
FROM wikipedia)) lololol
`),
).toMatchInlineSnapshot(`
"SELECT *
FROM wikipedia"
`);
});
});
describe('findAllSqlQueriesInText', () => {
it('works with separate queries', () => {
const text = sane`
SELECT *
FROM wikipedia
SELECT *
FROM w2
LIMIT 5
SELECT
`;
const found = findAllSqlQueriesInText(text);
expect(found).toMatchInlineSnapshot(`
Array [
Object {
"endOffset": 23,
"endRowColumn": Object {
"column": 14,
"row": 1,
},
"sql": "SELECT *
FROM wikipedia",
"startOffset": 0,
"startRowColumn": Object {
"column": 0,
"row": 0,
},
},
Object {
"endOffset": 49,
"endRowColumn": Object {
"column": 7,
"row": 5,
},
"sql": "SELECT *
FROM w2
LIMIT 5",
"startOffset": 25,
"startRowColumn": Object {
"column": 0,
"row": 3,
},
},
]
`);
});
it('works with simple query inside', () => {
const text = sane`
SELECT
"channel",
COUNT(*) AS "Count"
FROM (SELECT * FROM "wikipedia")
GROUP BY 1
ORDER BY 2 DESC
`;
const found = findAllSqlQueriesInText(text);
expect(found).toMatchInlineSnapshot(`
Array [
Object {
"endOffset": 101,
"endRowColumn": Object {
"column": 15,
"row": 5,
},
"sql": "SELECT
\\"channel\\",
COUNT(*) AS \\"Count\\"
FROM (SELECT * FROM \\"wikipedia\\")
GROUP BY 1
ORDER BY 2 DESC",
"startOffset": 0,
"startRowColumn": Object {
"column": 0,
"row": 0,
},
},
Object {
"endOffset": 73,
"endRowColumn": Object {
"column": 31,
"row": 3,
},
"sql": "SELECT * FROM \\"wikipedia\\"",
"startOffset": 48,
"startRowColumn": Object {
"column": 6,
"row": 3,
},
},
]
`);
});
it('works with CTE query', () => {
const text = sane`
WITH w1 AS (
SELECT channel, page FROM "wikipedia"
)
SELECT
page,
COUNT(*) AS "cnt"
FROM w1
GROUP BY 1
ORDER BY 2 DESC
`;
const found = findAllSqlQueriesInText(text);
expect(found).toMatchInlineSnapshot(`
Array [
Object {
"endOffset": 124,
"endRowColumn": Object {
"column": 15,
"row": 8,
},
"sql": "WITH w1 AS (
SELECT channel, page FROM \\"wikipedia\\"
)
SELECT
page,
COUNT(*) AS \\"cnt\\"
FROM w1
GROUP BY 1
ORDER BY 2 DESC",
"startOffset": 0,
"startRowColumn": Object {
"column": 0,
"row": 0,
},
},
Object {
"endOffset": 52,
"endRowColumn": Object {
"column": 39,
"row": 1,
},
"sql": "SELECT channel, page FROM \\"wikipedia\\"",
"startOffset": 15,
"startRowColumn": Object {
"column": 2,
"row": 1,
},
},
Object {
"endOffset": 124,
"endRowColumn": Object {
"column": 15,
"row": 8,
},
"sql": "SELECT
page,
COUNT(*) AS \\"cnt\\"
FROM w1
GROUP BY 1
ORDER BY 2 DESC",
"startOffset": 55,
"startRowColumn": Object {
"column": 0,
"row": 3,
},
},
]
`);
});
it('works with replace query', () => {
const text = sane`
REPLACE INTO "wikipedia" OVERWRITE ALL
WITH "ext" AS (SELECT *
FROM TABLE(
EXTERN(
'{"type":"http","uris":["https://druid.apache.org/data/wikipedia.json.gz"]}',
'{"type":"json"}'
)
) EXTEND ("isRobot" VARCHAR, "channel" VARCHAR, "timestamp" VARCHAR))
SELECT
TIME_PARSE("timestamp") AS "__time",
"isRobot",
"channel"
FROM "ext"
PARTITIONED BY DAY
`;
const found = findAllSqlQueriesInText(text);
expect(found).toMatchInlineSnapshot(`
Array [
Object {
"endOffset": 363,
"endRowColumn": Object {
"column": 18,
"row": 13,
},
"sql": "REPLACE INTO \\"wikipedia\\" OVERWRITE ALL
WITH \\"ext\\" AS (SELECT *
FROM TABLE(
EXTERN(
'{\\"type\\":\\"http\\",\\"uris\\":[\\"https://druid.apache.org/data/wikipedia.json.gz\\"]}',
'{\\"type\\":\\"json\\"}'
)
) EXTEND (\\"isRobot\\" VARCHAR, \\"channel\\" VARCHAR, \\"timestamp\\" VARCHAR))
SELECT
TIME_PARSE(\\"timestamp\\") AS \\"__time\\",
\\"isRobot\\",
\\"channel\\"
FROM \\"ext\\"
PARTITIONED BY DAY",
"startOffset": 0,
"startRowColumn": Object {
"column": 0,
"row": 0,
},
},
Object {
"endOffset": 344,
"endRowColumn": Object {
"column": 10,
"row": 12,
},
"sql": "WITH \\"ext\\" AS (SELECT *
FROM TABLE(
EXTERN(
'{\\"type\\":\\"http\\",\\"uris\\":[\\"https://druid.apache.org/data/wikipedia.json.gz\\"]}',
'{\\"type\\":\\"json\\"}'
)
) EXTEND (\\"isRobot\\" VARCHAR, \\"channel\\" VARCHAR, \\"timestamp\\" VARCHAR))
SELECT
TIME_PARSE(\\"timestamp\\") AS \\"__time\\",
\\"isRobot\\",
\\"channel\\"
FROM \\"ext\\"",
"startOffset": 39,
"startRowColumn": Object {
"column": 0,
"row": 1,
},
},
Object {
"endOffset": 261,
"endRowColumn": Object {
"column": 68,
"row": 7,
},
"sql": "SELECT *
FROM TABLE(
EXTERN(
'{\\"type\\":\\"http\\",\\"uris\\":[\\"https://druid.apache.org/data/wikipedia.json.gz\\"]}',
'{\\"type\\":\\"json\\"}'
)
) EXTEND (\\"isRobot\\" VARCHAR, \\"channel\\" VARCHAR, \\"timestamp\\" VARCHAR)",
"startOffset": 54,
"startRowColumn": Object {
"column": 15,
"row": 1,
},
},
Object {
"endOffset": 344,
"endRowColumn": Object {
"column": 10,
"row": 12,
},
"sql": "SELECT
TIME_PARSE(\\"timestamp\\") AS \\"__time\\",
\\"isRobot\\",
\\"channel\\"
FROM \\"ext\\"",
"startOffset": 263,
"startRowColumn": Object {
"column": 0,
"row": 8,
},
},
]
`);
});
});
});

View File

@ -16,7 +16,17 @@
* limitations under the License. * limitations under the License.
*/ */
import { SqlColumn, SqlExpression, SqlFunction, SqlLiteral, SqlStar } from '@druid-toolkit/query'; import {
SqlColumn,
SqlExpression,
SqlFunction,
SqlLiteral,
SqlQuery,
SqlStar,
} from '@druid-toolkit/query';
import type { RowColumn } from './general';
import { offsetToRowColumn } from './general';
export function timeFormatToSql(timeFormat: string): SqlExpression | undefined { export function timeFormatToSql(timeFormat: string): SqlExpression | undefined {
switch (timeFormat) { switch (timeFormat) {
@ -60,3 +70,72 @@ export function convertToGroupByExpression(ex: SqlExpression): SqlExpression | u
return newEx.as((ex.getOutputName() || 'grouped').replace(/^[a-z]+_/i, '')); return newEx.as((ex.getOutputName() || 'grouped').replace(/^[a-z]+_/i, ''));
} }
function extractQueryPrefix(text: string): string {
let q = SqlQuery.parse(text);
// The parser will parse a SELECT query with a partitionedByClause and clusteredByClause but that is not valid, remove them from the query
if (!q.getIngestTable() && (q.partitionedByClause || q.clusteredByClause)) {
q = q.changePartitionedByClause(undefined).changeClusteredByClause(undefined);
}
return q.toString().trimEnd();
}
export function findSqlQueryPrefix(text: string): string | undefined {
try {
return extractQueryPrefix(text);
} catch (e) {
const startOffset = e.location?.start?.offset;
if (typeof startOffset !== 'number') return;
const prefix = text.slice(0, startOffset);
// Try to trim to where the error came from
try {
return extractQueryPrefix(prefix);
} catch {
// Try to trim out last word
try {
return extractQueryPrefix(prefix.replace(/\s*\w+$/, ''));
} catch {
return;
}
}
}
}
export interface QuerySlice {
startOffset: number;
startRowColumn: RowColumn;
endOffset: number;
endRowColumn: RowColumn;
sql: string;
}
export function findAllSqlQueriesInText(text: string): QuerySlice[] {
const found: QuerySlice[] = [];
let remainingText = text;
let offset = 0;
let m: RegExpExecArray | null = null;
do {
m = /SELECT|WITH|INSERT|REPLACE/i.exec(remainingText);
if (m) {
const sql = findSqlQueryPrefix(remainingText.slice(m.index));
const advanceBy = m.index + m[0].length; // Skip the initial word
if (sql) {
const endIndex = m.index + sql.length;
found.push({
startOffset: offset + m.index,
startRowColumn: offsetToRowColumn(text, offset + m.index)!,
endOffset: offset + endIndex,
endRowColumn: offsetToRowColumn(text, offset + endIndex)!,
sql,
});
}
remainingText = remainingText.slice(advanceBy);
offset += advanceBy;
}
} while (m);
return found;
}

View File

@ -164,7 +164,6 @@ export const ColumnEditor = React.memo(function ColumnEditor(props: ColumnEditor
</FormGroup> </FormGroup>
<FormGroup label="SQL expression"> <FormGroup label="SQL expression">
<FlexibleQueryInput <FlexibleQueryInput
autoHeight={false}
showGutter={false} showGutter={false}
placeholder="expression" placeholder="expression"
queryString={effectiveExpressionString} queryString={effectiveExpressionString}

View File

@ -57,7 +57,6 @@ export const ExpressionEditorDialog = React.memo(function ExpressionEditorDialog
<div className={Classes.DIALOG_BODY}> <div className={Classes.DIALOG_BODY}>
<FormGroup> <FormGroup>
<FlexibleQueryInput <FlexibleQueryInput
autoHeight={false}
showGutter={false} showGutter={false}
placeholder="expression" placeholder="expression"
queryString={formula} queryString={formula}

View File

@ -58,7 +58,7 @@ import {
ingestQueryPatternToQuery, ingestQueryPatternToQuery,
possibleDruidFormatForValues, possibleDruidFormatForValues,
TIME_COLUMN, TIME_COLUMN,
WorkbenchQueryPart, WorkbenchQuery,
} from '../../../druid-models'; } from '../../../druid-models';
import { import {
executionBackgroundResultStatusCheck, executionBackgroundResultStatusCheck,
@ -483,8 +483,7 @@ export const SchemaStep = function SchemaStep(props: SchemaStepProps) {
const [previewResultState] = useQueryManager<string, QueryResult, Execution>({ const [previewResultState] = useQueryManager<string, QueryResult, Execution>({
query: previewQueryString, query: previewQueryString,
processQuery: async (previewQueryString, cancelToken) => { processQuery: async (previewQueryString, cancelToken) => {
const taskEngine = WorkbenchQueryPart.isTaskEngineNeeded(previewQueryString); if (WorkbenchQuery.isTaskEngineNeeded(previewQueryString)) {
if (taskEngine) {
return extractResult( return extractResult(
await submitTaskQuery({ await submitTaskQuery({
query: previewQueryString, query: previewQueryString,
@ -872,7 +871,6 @@ export const SchemaStep = function SchemaStep(props: SchemaStepProps) {
))} ))}
{effectiveMode === 'sql' && ( {effectiveMode === 'sql' && (
<FlexibleQueryInput <FlexibleQueryInput
autoHeight={false}
queryString={queryString} queryString={queryString}
onQueryStringChange={onQueryStringChange} onQueryStringChange={onQueryStringChange}
columnMetadata={undefined} columnMetadata={undefined}

View File

@ -23,7 +23,7 @@ const BASE_QUERY = WorkbenchQuery.blank();
export function getDemoQueries(): TabEntry[] { export function getDemoQueries(): TabEntry[] {
function makeDemoQuery(queryString: string): WorkbenchQuery { function makeDemoQuery(queryString: string): WorkbenchQuery {
return BASE_QUERY.duplicate().changeQueryString(queryString.trim()); return BASE_QUERY.changeQueryString(queryString.trim());
} }
return [ return [

View File

@ -95,7 +95,6 @@ export const ExecutionDetailsPane = React.memo(function ExecutionDetailsPane(
? String(execution.sqlQuery) ? String(execution.sqlQuery)
: JSONBig.stringify(execution.nativeQuery, undefined, 2) : JSONBig.stringify(execution.nativeQuery, undefined, 2)
} }
autoHeight={false}
/> />
); );

View File

@ -12,7 +12,7 @@ exports[`FlexibleQueryInput matches snapshot 1`] = `
class="flexible-query-input" class="flexible-query-input"
> >
<div <div
class="ace-container" class="ace-container query-idle"
> >
<div <div
class=" ace_editor ace_hidpi ace-tm placeholder-padding no-background ace_focus" class=" ace_editor ace_hidpi ace-tm placeholder-padding no-background ace_focus"

View File

@ -16,6 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
@import '../../../variables';
.flexible-query-input { .flexible-query-input {
position: relative; position: relative;
@ -24,4 +26,54 @@
width: 100%; width: 100%;
height: 100%; height: 100%;
} }
.sub-query-highlight {
position: absolute;
background: $gray1;
}
.sub-query-gutter-marker {
cursor: pointer;
&:before {
content: '';
position: absolute;
top: 3px;
left: 2px;
width: 12px;
height: 12px;
background: $blue3;
color: white;
line-height: 12px;
text-align: center;
border-radius: 2px;
}
&:hover:before {
background: $blue2;
}
&:hover:after {
content: 'Run';
position: absolute;
top: 0;
left: 16px;
right: 0;
background: #383d57;
text-align: left;
animation: sharpFadeIn 1s;
}
}
}
@keyframes sharpFadeIn {
0% {
opacity: 0;
}
90% {
opacity: 0;
}
100% {
opacity: 1;
}
} }

View File

@ -24,11 +24,7 @@ import { FlexibleQueryInput } from './flexible-query-input';
describe('FlexibleQueryInput', () => { describe('FlexibleQueryInput', () => {
it('matches snapshot', () => { it('matches snapshot', () => {
const sqlControl = ( const sqlControl = (
<FlexibleQueryInput <FlexibleQueryInput queryString="hello world" onQueryStringChange={() => {}} />
queryString="hello world"
autoHeight={false}
onQueryStringChange={() => {}}
/>
); );
const { container } = render(sqlControl); const { container } = render(sqlControl);

View File

@ -16,11 +16,14 @@
* limitations under the License. * limitations under the License.
*/ */
import { Intent } from '@blueprintjs/core';
import { IconNames } from '@blueprintjs/icons';
import { ResizeSensor2 } from '@blueprintjs/popover2'; import { ResizeSensor2 } from '@blueprintjs/popover2';
import { C, T } from '@druid-toolkit/query'; import { C, dedupe, T } from '@druid-toolkit/query';
import type { Ace } from 'ace-builds'; import type { Ace } from 'ace-builds';
import ace from 'ace-builds'; import ace from 'ace-builds';
import classNames from 'classnames'; import classNames from 'classnames';
import debounce from 'lodash.debounce';
import escape from 'lodash.escape'; import escape from 'lodash.escape';
import React from 'react'; import React from 'react';
import AceEditor from 'react-ace'; import AceEditor from 'react-ace';
@ -32,16 +35,16 @@ import {
SQL_KEYWORDS, SQL_KEYWORDS,
} from '../../../../lib/keywords'; } from '../../../../lib/keywords';
import { SQL_DATA_TYPES, SQL_FUNCTIONS } from '../../../../lib/sql-docs'; import { SQL_DATA_TYPES, SQL_FUNCTIONS } from '../../../../lib/sql-docs';
import { AppToaster } from '../../../singletons';
import { AceEditorStateCache } from '../../../singletons/ace-editor-state-cache'; import { AceEditorStateCache } from '../../../singletons/ace-editor-state-cache';
import type { ColumnMetadata, RowColumn } from '../../../utils'; import type { ColumnMetadata, QuerySlice, RowColumn } from '../../../utils';
import { uniq } from '../../../utils'; import { findAllSqlQueriesInText, findMap, uniq } from '../../../utils';
import './flexible-query-input.scss'; import './flexible-query-input.scss';
const langTools = ace.require('ace/ext/language_tools'); const langTools = ace.require('ace/ext/language_tools');
const V_PADDING = 10; const V_PADDING = 10;
const SCROLLBAR = 20;
const COMPLETER = { const COMPLETER = {
insertMatch: (editor: any, data: Ace.Completion) => { insertMatch: (editor: any, data: Ace.Completion) => {
@ -58,8 +61,8 @@ interface ItemDescription {
export interface FlexibleQueryInputProps { export interface FlexibleQueryInputProps {
queryString: string; queryString: string;
onQueryStringChange?: (newQueryString: string) => void; onQueryStringChange?: (newQueryString: string) => void;
autoHeight: boolean; runQuerySlice?: (querySlice: QuerySlice) => void;
minRows?: number; running?: boolean;
showGutter?: boolean; showGutter?: boolean;
placeholder?: string; placeholder?: string;
columnMetadata?: readonly ColumnMetadata[]; columnMetadata?: readonly ColumnMetadata[];
@ -85,6 +88,8 @@ export class FlexibleQueryInput extends React.PureComponent<
FlexibleQueryInputState FlexibleQueryInputState
> { > {
private aceEditor: Ace.Editor | undefined; private aceEditor: Ace.Editor | undefined;
private lastFoundQueries: QuerySlice[] = [];
private highlightFoundQuery: { row: number; marker: number } | undefined;
static replaceDefaultAutoCompleter(): void { static replaceDefaultAutoCompleter(): void {
if (!langTools) return; if (!langTools) return;
@ -260,6 +265,14 @@ export class FlexibleQueryInput extends React.PureComponent<
}, },
}); });
} }
this.markQueries();
}
componentDidUpdate(prevProps: Readonly<FlexibleQueryInputProps>) {
if (this.props.queryString !== prevProps.queryString) {
this.markQueriesDebounced();
}
} }
componentWillUnmount() { componentWillUnmount() {
@ -267,8 +280,42 @@ export class FlexibleQueryInput extends React.PureComponent<
if (editorStateId && this.aceEditor) { if (editorStateId && this.aceEditor) {
AceEditorStateCache.saveState(editorStateId, this.aceEditor); AceEditorStateCache.saveState(editorStateId, this.aceEditor);
} }
delete this.aceEditor;
} }
private findAllQueriesByLine() {
const { queryString } = this.props;
const found = dedupe(findAllSqlQueriesInText(queryString), ({ startRowColumn }) =>
String(startRowColumn.row),
);
if (found.length <= 1) return []; // Do not highlight a single query or no queries
// Do not report the first query if it is basically the main query minus whitespace
const firstQuery = found[0].sql;
if (firstQuery === queryString.trim()) return found.slice(1);
return found;
}
private readonly markQueries = () => {
if (!this.props.runQuerySlice) return;
const { aceEditor } = this;
if (!aceEditor) return;
const session = aceEditor.getSession();
this.lastFoundQueries = this.findAllQueriesByLine();
session.clearBreakpoints();
this.lastFoundQueries.forEach(({ startRowColumn }) => {
// session.addGutterDecoration(startRowColumn.row, `sub-query-gutter-marker query-${i}`);
session.setBreakpoint(
startRowColumn.row,
`sub-query-gutter-marker query-${startRowColumn.row}`,
);
});
};
private readonly markQueriesDebounced = debounce(this.markQueries, 900, { trailing: true });
private readonly handleAceContainerResize = (entries: ResizeObserverEntry[]) => { private readonly handleAceContainerResize = (entries: ResizeObserverEntry[]) => {
if (entries.length !== 1) return; if (entries.length !== 1) return;
this.setState({ editorHeight: entries[0].contentRect.height }); this.setState({ editorHeight: entries[0].contentRect.height });
@ -285,35 +332,16 @@ export class FlexibleQueryInput extends React.PureComponent<
if (!aceEditor) return; if (!aceEditor) return;
aceEditor.focus(); // Grab the focus aceEditor.focus(); // Grab the focus
aceEditor.getSelection().moveCursorTo(rowColumn.row, rowColumn.column); aceEditor.getSelection().moveCursorTo(rowColumn.row, rowColumn.column);
if (rowColumn.endRow && rowColumn.endColumn) { // If we had an end we could also do
aceEditor // aceEditor.getSelection().selectToPosition({ row: endRow, column: endColumn });
.getSelection()
.selectToPosition({ row: rowColumn.endRow, column: rowColumn.endColumn });
}
} }
renderAce() { renderAce() {
const { const { queryString, onQueryStringChange, showGutter, placeholder, editorStateId } = this.props;
queryString,
onQueryStringChange,
autoHeight,
minRows,
showGutter,
placeholder,
editorStateId,
} = this.props;
const { editorHeight } = this.state; const { editorHeight } = this.state;
const jsonMode = queryString.trim().startsWith('{'); const jsonMode = queryString.trim().startsWith('{');
let height: number;
if (autoHeight) {
height =
Math.max(queryString.split('\n').length, minRows ?? 2) * 18 + 2 * V_PADDING + SCROLLBAR;
} else {
height = editorHeight;
}
return ( return (
<AceEditor <AceEditor
mode={jsonMode ? 'hjson' : 'dsql'} mode={jsonMode ? 'hjson' : 'dsql'}
@ -327,7 +355,7 @@ export class FlexibleQueryInput extends React.PureComponent<
focus focus
fontSize={13} fontSize={13}
width="100%" width="100%"
height={height + 'px'} height={editorHeight + 'px'}
showGutter={showGutter} showGutter={showGutter}
showPrintMargin={false} showPrintMargin={false}
value={queryString} value={queryString}
@ -359,18 +387,83 @@ export class FlexibleQueryInput extends React.PureComponent<
} }
render() { render() {
const { autoHeight } = this.props; const { runQuerySlice, running } = this.props;
// Set the key in the AceEditor to force a rebind and prevent an error that happens otherwise // Set the key in the AceEditor to force a rebind and prevent an error that happens otherwise
return ( return (
<div className="flexible-query-input"> <div className="flexible-query-input">
{autoHeight ? ( <ResizeSensor2 onResize={this.handleAceContainerResize}>
this.renderAce() <div
) : ( className={classNames('ace-container', running ? 'query-running' : 'query-idle')}
<ResizeSensor2 onResize={this.handleAceContainerResize}> onClick={e => {
<div className="ace-container">{this.renderAce()}</div> if (!runQuerySlice) return;
</ResizeSensor2> const classes = [...(e.target as any).classList];
)} if (!classes.includes('sub-query-gutter-marker')) return;
const row = findMap(classes, c => {
const m = /^query-(\d+)$/.exec(c);
return m ? Number(m[1]) : undefined;
});
if (typeof row === 'undefined') return;
// Gutter query marker clicked on line ${row}
const slice = this.lastFoundQueries.find(
({ startRowColumn }) => startRowColumn.row === row,
);
if (!slice) return;
if (running) {
AppToaster.show({
icon: IconNames.WARNING_SIGN,
intent: Intent.WARNING,
message: `Another query is currently running`,
});
return;
}
runQuerySlice(slice);
}}
onMouseOver={e => {
if (!runQuerySlice) return;
const aceEditor = this.aceEditor;
if (!aceEditor) return;
const classes = [...(e.target as any).classList];
if (!classes.includes('sub-query-gutter-marker')) return;
const row = findMap(classes, c => {
const m = /^query-(\d+)$/.exec(c);
return m ? Number(m[1]) : undefined;
});
if (typeof row === 'undefined' || this.highlightFoundQuery?.row === row) return;
const slice = this.lastFoundQueries.find(
({ startRowColumn }) => startRowColumn.row === row,
);
if (!slice) return;
const marker = aceEditor
.getSession()
.addMarker(
new ace.Range(
slice.startRowColumn.row,
slice.startRowColumn.column,
slice.endRowColumn.row,
slice.endRowColumn.column,
),
'sub-query-highlight',
'text',
);
this.highlightFoundQuery = { row, marker };
}}
onMouseOut={() => {
if (!this.highlightFoundQuery) return;
const aceEditor = this.aceEditor;
if (!aceEditor) return;
aceEditor.getSession().removeMarker(this.highlightFoundQuery.marker);
this.highlightFoundQuery = undefined;
}}
>
{this.renderAce()}
</div>
</ResizeSensor2>
</div> </div>
); );
} }

View File

@ -1,100 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 '../../../variables';
.helper-query {
position: relative;
@include card-like;
.query-top-bar {
position: relative;
height: 36px;
display: flex;
align-items: center;
gap: 4px;
white-space: nowrap;
.corner {
position: absolute;
top: 50%;
right: 3px;
transform: translate(0, -50%);
@include card-background;
}
}
.flexible-query-input {
border-top: 1px solid rgba($dark-gray1, 0.5);
border-bottom: 1px solid rgba($dark-gray1, 0.5);
}
.query-control-bar {
position: relative;
width: 100%;
height: 30px;
display: flex;
gap: 10px;
align-items: center;
.execution-timer-panel,
.execution-summary-panel {
position: absolute;
top: 0;
right: 0;
}
}
.init-pane {
text-align: center;
flex: 1;
border-top: 1px solid rgba($dark-gray1, 0.5);
p {
position: relative;
top: 38%;
font-size: 15px;
}
}
.output-pane {
overflow: hidden;
position: relative;
height: 254px;
border-top: 1px solid rgba($dark-gray1, 0.5);
> * {
position: absolute;
width: 100%;
height: 100%;
}
.error-container {
position: relative;
.execution-error-pane {
position: absolute;
top: 5px;
left: 5px;
right: 5px;
height: 150px;
width: auto;
}
}
}
}

View File

@ -1,473 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 { Button, ButtonGroup, InputGroup, Intent, Menu, MenuItem } from '@blueprintjs/core';
import { IconNames } from '@blueprintjs/icons';
import { Popover2 } from '@blueprintjs/popover2';
import type { QueryResult } from '@druid-toolkit/query';
import { QueryRunner, SqlQuery } from '@druid-toolkit/query';
import axios from 'axios';
import type { JSX } from 'react';
import React, { useCallback, useEffect, useRef, useState } from 'react';
import { useStore } from 'zustand';
import { Loader, QueryErrorPane } from '../../../components';
import type { DruidEngine, LastExecution, QueryContext } from '../../../druid-models';
import {
Execution,
fitExternalConfigPattern,
summarizeExternalConfig,
WorkbenchQuery,
} from '../../../druid-models';
import {
executionBackgroundStatusCheck,
maybeGetClusterCapacity,
reattachTaskExecution,
submitTaskQuery,
} from '../../../helpers';
import { usePermanentCallback, useQueryManager } from '../../../hooks';
import { Api, AppToaster } from '../../../singletons';
import { ExecutionStateCache } from '../../../singletons/execution-state-cache';
import { WorkbenchHistory } from '../../../singletons/workbench-history';
import type { WorkbenchRunningPromise } from '../../../singletons/workbench-running-promises';
import { WorkbenchRunningPromises } from '../../../singletons/workbench-running-promises';
import type { ColumnMetadata, QueryAction, RowColumn } from '../../../utils';
import { DruidError, QueryManager } from '../../../utils';
import { CapacityAlert } from '../capacity-alert/capacity-alert';
import type { ExecutionDetailsTab } from '../execution-details-pane/execution-details-pane';
import { ExecutionErrorPane } from '../execution-error-pane/execution-error-pane';
import { ExecutionProgressPane } from '../execution-progress-pane/execution-progress-pane';
import { ExecutionStagesPane } from '../execution-stages-pane/execution-stages-pane';
import { ExecutionSummaryPanel } from '../execution-summary-panel/execution-summary-panel';
import { ExecutionTimerPanel } from '../execution-timer-panel/execution-timer-panel';
import { FlexibleQueryInput } from '../flexible-query-input/flexible-query-input';
import { IngestSuccessPane } from '../ingest-success-pane/ingest-success-pane';
import { metadataStateStore } from '../metadata-state-store';
import { ResultTablePane } from '../result-table-pane/result-table-pane';
import { RunPanel } from '../run-panel/run-panel';
import { workStateStore } from '../work-state-store';
import './helper-query.scss';
const queryRunner = new QueryRunner({
inflateDateStrategy: 'none',
});
export interface HelperQueryProps {
query: WorkbenchQuery;
mandatoryQueryContext: QueryContext | undefined;
columnMetadata: readonly ColumnMetadata[] | undefined;
onQueryChange(newQuery: WorkbenchQuery): void;
onQueryTab(newQuery: WorkbenchQuery, tabName?: string): void;
onDelete(): void;
onDetails(id: string, initTab?: ExecutionDetailsTab): void;
queryEngines: DruidEngine[];
clusterCapacity: number | undefined;
goToTask(taskId: string): void;
}
export const HelperQuery = React.memo(function HelperQuery(props: HelperQueryProps) {
const {
query,
columnMetadata,
mandatoryQueryContext,
onQueryChange,
onQueryTab,
onDelete,
onDetails,
queryEngines,
clusterCapacity,
goToTask,
} = props;
const [alertElement, setAlertElement] = useState<JSX.Element | undefined>();
// Store the cancellation function for natively run queries allowing us to trigger it only when the user explicitly clicks "cancel" (vs changing tab)
const nativeQueryCancelFnRef = useRef<() => void>();
const handleQueryStringChange = usePermanentCallback((queryString: string) => {
onQueryChange(query.changeQueryString(queryString));
});
const parsedQuery = query.getParsedQuery();
const handleQueryAction = usePermanentCallback((queryAction: QueryAction) => {
if (!(parsedQuery instanceof SqlQuery)) return;
onQueryChange(query.changeQueryString(parsedQuery.apply(queryAction).toString()));
if (shouldAutoRun()) {
setTimeout(() => void handleRun(false), 20);
}
});
function shouldAutoRun(): boolean {
if (query.getEffectiveEngine() !== 'sql-native') return false;
const queryDuration = executionState.data?.result?.queryDuration;
return Boolean(queryDuration && queryDuration < 10000);
}
const queryInputRef = useRef<FlexibleQueryInput | null>(null);
const id = query.getId();
const [executionState, queryManager] = useQueryManager<
WorkbenchQuery | WorkbenchRunningPromise | LastExecution,
Execution,
Execution,
DruidError
>({
initQuery: ExecutionStateCache.getState(id)
? undefined
: WorkbenchRunningPromises.getPromise(id) || query.getLastExecution(),
initState: ExecutionStateCache.getState(id),
processQuery: async (q, cancelToken) => {
if (q instanceof WorkbenchQuery) {
ExecutionStateCache.deleteState(id);
const { engine, query, sqlPrefixLines, cancelQueryId } = q.getApiQuery();
switch (engine) {
case 'sql-msq-task':
return await submitTaskQuery({
query,
prefixLines: sqlPrefixLines,
cancelToken,
preserveOnTermination: true,
onSubmitted: id => {
onQueryChange(props.query.changeLastExecution({ engine, id }));
},
});
case 'native':
case 'sql-native': {
if (cancelQueryId) {
void cancelToken.promise
.then(cancel => {
if (cancel.message === QueryManager.TERMINATION_MESSAGE) return;
return Api.instance.delete(
`/druid/v2${engine === 'sql-native' ? '/sql' : ''}/${Api.encodePath(
cancelQueryId,
)}`,
);
})
.catch(() => {});
}
onQueryChange(props.query.changeLastExecution(undefined));
let result: QueryResult;
try {
const resultPromise = queryRunner.runQuery({
query,
extraQueryContext: mandatoryQueryContext,
cancelToken: new axios.CancelToken(cancelFn => {
nativeQueryCancelFnRef.current = cancelFn;
}),
});
WorkbenchRunningPromises.storePromise(id, { promise: resultPromise, sqlPrefixLines });
result = await resultPromise;
nativeQueryCancelFnRef.current = undefined;
} catch (e) {
nativeQueryCancelFnRef.current = undefined;
throw new DruidError(e, sqlPrefixLines);
}
return Execution.fromResult(engine, result);
}
}
} else if (WorkbenchRunningPromises.isWorkbenchRunningPromise(q)) {
let result: QueryResult;
try {
result = await q.promise;
} catch (e) {
WorkbenchRunningPromises.deletePromise(id);
throw new DruidError(e, q.sqlPrefixLines);
}
WorkbenchRunningPromises.deletePromise(id);
return Execution.fromResult('sql-native', result);
} else {
switch (q.engine) {
case 'sql-msq-task':
return await reattachTaskExecution({
id: q.id,
cancelToken,
preserveOnTermination: true,
});
default:
throw new Error(`Can not reattach on ${q.engine}`);
}
}
},
backgroundStatusCheck: executionBackgroundStatusCheck,
swallowBackgroundError: Api.isNetworkError,
});
useEffect(() => {
if (!executionState.data) return;
ExecutionStateCache.storeState(id, executionState);
// eslint-disable-next-line react-hooks/exhaustive-deps
}, [executionState.data, executionState.error]);
const incrementWorkVersion = useStore(
workStateStore,
useCallback(state => state.increment, []),
);
useEffect(() => {
incrementWorkVersion();
// eslint-disable-next-line react-hooks/exhaustive-deps
}, [executionState.loading, Boolean(executionState.intermediate)]);
const execution = executionState.data;
const incrementMetadataVersion = useStore(
metadataStateStore,
useCallback(state => state.increment, []),
);
useEffect(() => {
if (execution?.isSuccessfulInsert()) {
incrementMetadataVersion();
}
// eslint-disable-next-line react-hooks/exhaustive-deps
}, [Boolean(execution?.isSuccessfulInsert())]);
function moveToPosition(position: RowColumn) {
const currentQueryInput = queryInputRef.current;
if (!currentQueryInput) return;
currentQueryInput.goToPosition(position);
}
const handleRun = usePermanentCallback(async (preview: boolean) => {
const queryIssue = query.getIssue();
if (queryIssue) {
const position = WorkbenchQuery.getRowColumnFromIssue(queryIssue);
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
timeout: 90000,
message: queryIssue,
action: position
? {
text: 'Go to issue',
onClick: () => moveToPosition(position),
}
: undefined,
});
return;
}
if (query.getEffectiveEngine() !== 'sql-msq-task') {
WorkbenchHistory.addQueryToHistory(query);
queryManager.runQuery(query);
return;
}
const effectiveQuery = preview
? query.makePreview()
: query.setMaxNumTasksIfUnset(clusterCapacity);
const capacityInfo = await maybeGetClusterCapacity();
const effectiveMaxNumTasks = effectiveQuery.queryContext.maxNumTasks ?? 2;
if (capacityInfo && capacityInfo.availableTaskSlots < effectiveMaxNumTasks) {
setAlertElement(
<CapacityAlert
maxNumTasks={effectiveMaxNumTasks}
capacityInfo={capacityInfo}
onRun={() => {
queryManager.runQuery(effectiveQuery);
}}
onClose={() => {
setAlertElement(undefined);
}}
/>,
);
} else {
queryManager.runQuery(effectiveQuery);
}
});
const collapsed = query.getCollapsed();
const insertDatasource = query.getIngestDatasource();
const statsTaskId: string | undefined = execution?.id;
let extraInfo: string | undefined;
if (collapsed && parsedQuery instanceof SqlQuery) {
try {
extraInfo = summarizeExternalConfig(fitExternalConfigPattern(parsedQuery));
} catch {}
}
const onUserCancel = (message?: string) => {
queryManager.cancelCurrent(message);
nativeQueryCancelFnRef.current?.();
};
return (
<div className="helper-query">
<div className="query-top-bar">
<Button
icon={collapsed ? IconNames.CARET_RIGHT : IconNames.CARET_DOWN}
minimal
onClick={() => onQueryChange(query.changeCollapsed(!collapsed))}
/>
{insertDatasource ? (
`<insert query : ${insertDatasource}>`
) : (
<>
{collapsed ? (
<span className="query-name">{query.getQueryName()}</span>
) : (
<InputGroup
className="query-name"
value={query.getQueryName()}
onChange={(e: any) => {
onQueryChange(query.changeQueryName(e.target.value));
}}
/>
)}
<span className="as-label">AS</span>
{extraInfo && <span className="extra-info">{extraInfo}</span>}
</>
)}
<ButtonGroup className="corner">
<Popover2
content={
<Menu>
<MenuItem
icon={IconNames.DUPLICATE}
text="Duplicate"
onClick={() => onQueryChange(query.duplicateLast())}
/>
</Menu>
}
>
<Button icon={IconNames.MORE} minimal />
</Popover2>
<Button
icon={IconNames.CROSS}
minimal
onClick={() => {
ExecutionStateCache.deleteState(id);
WorkbenchRunningPromises.deletePromise(id);
onDelete();
}}
/>
</ButtonGroup>
</div>
{!collapsed && (
<>
<FlexibleQueryInput
ref={queryInputRef}
autoHeight
queryString={query.getQueryString()}
onQueryStringChange={handleQueryStringChange}
columnMetadata={
columnMetadata ? columnMetadata.concat(query.getInlineMetadata()) : undefined
}
editorStateId={query.getId()}
/>
<div className="query-control-bar">
<RunPanel
query={query}
onQueryChange={onQueryChange}
onRun={handleRun}
loading={executionState.loading}
small
queryEngines={queryEngines}
clusterCapacity={clusterCapacity}
/>
{executionState.isLoading() && (
<ExecutionTimerPanel
execution={executionState.intermediate}
onCancel={() => queryManager.cancelCurrent()}
/>
)}
{(execution || executionState.error) && (
<ExecutionSummaryPanel
execution={execution}
onExecutionDetail={() => onDetails(statsTaskId!)}
onReset={() => {
queryManager.reset();
onQueryChange(props.query.changeLastExecution(undefined));
ExecutionStateCache.deleteState(id);
}}
/>
)}
</div>
{!executionState.isInit() && (
<div className="output-pane">
{execution &&
(execution.result ? (
<ResultTablePane
runeMode={execution.engine === 'native'}
queryResult={execution.result}
onQueryAction={handleQueryAction}
initPageSize={5}
/>
) : execution.isSuccessfulInsert() ? (
<IngestSuccessPane
execution={execution}
onDetails={onDetails}
onQueryTab={onQueryTab}
/>
) : execution.error ? (
<div className="error-container">
<ExecutionErrorPane execution={execution} />
{execution.stages && (
<ExecutionStagesPane
execution={execution}
onErrorClick={() => onDetails(statsTaskId!, 'error')}
onWarningClick={() => onDetails(statsTaskId!, 'warnings')}
goToTask={goToTask}
/>
)}
</div>
) : (
<div>Unknown query execution state</div>
))}
{executionState.error && (
<QueryErrorPane
error={executionState.error}
moveCursorTo={position => {
moveToPosition(position);
}}
queryString={query.getQueryString()}
onQueryStringChange={handleQueryStringChange}
/>
)}
{executionState.isLoading() &&
(executionState.intermediate ? (
<ExecutionProgressPane
execution={executionState.intermediate}
intermediateError={executionState.intermediateError}
goToTask={goToTask}
onCancel={onUserCancel}
/>
) : (
<Loader cancelText="Cancel query" onCancel={onUserCancel} />
))}
</div>
)}
</>
)}
{alertElement}
</div>
);
});

View File

@ -49,39 +49,20 @@ $vertical-gap: 6px;
width: 100%; width: 100%;
top: 0; top: 0;
bottom: 30px + $vertical-gap; bottom: 30px + $vertical-gap;
overflow: auto; @include card-like;
overflow: hidden;
.helper-query { .flexible-query-input {
margin-top: $vertical-gap; height: 100%;
} }
.main-query { .corner {
position: relative; position: absolute;
@include card-like; top: 0;
min-height: 100%; right: 0;
overflow: hidden; @include card-background;
z-index: 1;
&.single { padding: 3px;
height: 100%;
.flexible-query-input {
height: 100%;
}
}
&.multi {
min-height: calc(100% - 18px);
margin-top: $vertical-gap;
}
.corner {
position: absolute;
top: 0;
right: 0;
@include card-background;
z-index: 1;
padding: 3px;
}
} }
} }

View File

@ -16,13 +16,11 @@
* limitations under the License. * limitations under the License.
*/ */
import { Button, Code, Intent, Menu, MenuItem } from '@blueprintjs/core'; import { Code, Intent } from '@blueprintjs/core';
import { IconNames } from '@blueprintjs/icons'; import { IconNames } from '@blueprintjs/icons';
import { Popover2 } from '@blueprintjs/popover2';
import type { QueryResult } from '@druid-toolkit/query'; import type { QueryResult } from '@druid-toolkit/query';
import { QueryRunner, SqlQuery } from '@druid-toolkit/query'; import { QueryRunner, SqlQuery } from '@druid-toolkit/query';
import axios from 'axios'; import axios from 'axios';
import classNames from 'classnames';
import type { JSX } from 'react'; import type { JSX } from 'react';
import React, { useCallback, useEffect, useRef, useState } from 'react'; import React, { useCallback, useEffect, useRef, useState } from 'react';
import SplitterLayout from 'react-splitter-layout'; import SplitterLayout from 'react-splitter-layout';
@ -43,7 +41,7 @@ import { ExecutionStateCache } from '../../../singletons/execution-state-cache';
import { WorkbenchHistory } from '../../../singletons/workbench-history'; import { WorkbenchHistory } from '../../../singletons/workbench-history';
import type { WorkbenchRunningPromise } from '../../../singletons/workbench-running-promises'; import type { WorkbenchRunningPromise } from '../../../singletons/workbench-running-promises';
import { WorkbenchRunningPromises } from '../../../singletons/workbench-running-promises'; import { WorkbenchRunningPromises } from '../../../singletons/workbench-running-promises';
import type { ColumnMetadata, QueryAction, RowColumn } from '../../../utils'; import type { ColumnMetadata, QueryAction, QuerySlice, RowColumn } from '../../../utils';
import { import {
DruidError, DruidError,
localStorageGet, localStorageGet,
@ -59,7 +57,6 @@ import { ExecutionStagesPane } from '../execution-stages-pane/execution-stages-p
import { ExecutionSummaryPanel } from '../execution-summary-panel/execution-summary-panel'; import { ExecutionSummaryPanel } from '../execution-summary-panel/execution-summary-panel';
import { ExecutionTimerPanel } from '../execution-timer-panel/execution-timer-panel'; import { ExecutionTimerPanel } from '../execution-timer-panel/execution-timer-panel';
import { FlexibleQueryInput } from '../flexible-query-input/flexible-query-input'; import { FlexibleQueryInput } from '../flexible-query-input/flexible-query-input';
import { HelperQuery } from '../helper-query/helper-query';
import { IngestSuccessPane } from '../ingest-success-pane/ingest-success-pane'; import { IngestSuccessPane } from '../ingest-success-pane/ingest-success-pane';
import { metadataStateStore } from '../metadata-state-store'; import { metadataStateStore } from '../metadata-state-store';
import { ResultTablePane } from '../result-table-pane/result-table-pane'; import { ResultTablePane } from '../result-table-pane/result-table-pane';
@ -74,6 +71,7 @@ const queryRunner = new QueryRunner({
export interface QueryTabProps { export interface QueryTabProps {
query: WorkbenchQuery; query: WorkbenchQuery;
id: string;
mandatoryQueryContext: QueryContext | undefined; mandatoryQueryContext: QueryContext | undefined;
columnMetadata: readonly ColumnMetadata[] | undefined; columnMetadata: readonly ColumnMetadata[] | undefined;
onQueryChange(newQuery: WorkbenchQuery): void; onQueryChange(newQuery: WorkbenchQuery): void;
@ -88,6 +86,7 @@ export interface QueryTabProps {
export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) { export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
const { const {
query, query,
id,
columnMetadata, columnMetadata,
mandatoryQueryContext, mandatoryQueryContext,
onQueryChange, onQueryChange,
@ -145,7 +144,6 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
const queryInputRef = useRef<FlexibleQueryInput | null>(null); const queryInputRef = useRef<FlexibleQueryInput | null>(null);
const id = query.getId();
const [executionState, queryManager] = useQueryManager< const [executionState, queryManager] = useQueryManager<
WorkbenchQuery | WorkbenchRunningPromise | LastExecution, WorkbenchQuery | WorkbenchRunningPromise | LastExecution,
Execution, Execution,
@ -159,13 +157,13 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
processQuery: async (q, cancelToken) => { processQuery: async (q, cancelToken) => {
if (q instanceof WorkbenchQuery) { if (q instanceof WorkbenchQuery) {
ExecutionStateCache.deleteState(id); ExecutionStateCache.deleteState(id);
const { engine, query, sqlPrefixLines, cancelQueryId } = q.getApiQuery(); const { engine, query, prefixLines, cancelQueryId } = q.getApiQuery();
switch (engine) { switch (engine) {
case 'sql-msq-task': case 'sql-msq-task':
return await submitTaskQuery({ return await submitTaskQuery({
query, query,
prefixLines: sqlPrefixLines, prefixLines,
cancelToken, cancelToken,
preserveOnTermination: true, preserveOnTermination: true,
onSubmitted: id => { onSubmitted: id => {
@ -199,13 +197,13 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
nativeQueryCancelFnRef.current = cancelFn; nativeQueryCancelFnRef.current = cancelFn;
}), }),
}); });
WorkbenchRunningPromises.storePromise(id, { promise: resultPromise, sqlPrefixLines }); WorkbenchRunningPromises.storePromise(id, { promise: resultPromise, prefixLines });
result = await resultPromise; result = await resultPromise;
nativeQueryCancelFnRef.current = undefined; nativeQueryCancelFnRef.current = undefined;
} catch (e) { } catch (e) {
nativeQueryCancelFnRef.current = undefined; nativeQueryCancelFnRef.current = undefined;
throw new DruidError(e, sqlPrefixLines); throw new DruidError(e, prefixLines);
} }
return Execution.fromResult(engine, result); return Execution.fromResult(engine, result);
@ -217,7 +215,7 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
result = await q.promise; result = await q.promise;
} catch (e) { } catch (e) {
WorkbenchRunningPromises.deletePromise(id); WorkbenchRunningPromises.deletePromise(id);
throw new DruidError(e, q.sqlPrefixLines); throw new DruidError(e, q.prefixLines);
} }
WorkbenchRunningPromises.deletePromise(id); WorkbenchRunningPromises.deletePromise(id);
@ -274,7 +272,7 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
currentQueryInput.goToPosition(position); currentQueryInput.goToPosition(position);
} }
const handleRun = usePermanentCallback(async (preview: boolean) => { const handleRun = usePermanentCallback(async (preview: boolean, querySlice?: QuerySlice) => {
const queryIssue = query.getIssue(); const queryIssue = query.getIssue();
if (queryIssue) { if (queryIssue) {
const position = WorkbenchQuery.getRowColumnFromIssue(queryIssue); const position = WorkbenchQuery.getRowColumnFromIssue(queryIssue);
@ -294,15 +292,22 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
return; return;
} }
if (query.getEffectiveEngine() !== 'sql-msq-task') { let effectiveQuery = query;
WorkbenchHistory.addQueryToHistory(query); if (querySlice) {
queryManager.runQuery(query); effectiveQuery = effectiveQuery
.changeQueryString(querySlice.sql)
.changePrefixLines(querySlice.startRowColumn.row);
}
if (effectiveQuery.getEffectiveEngine() !== 'sql-msq-task') {
WorkbenchHistory.addQueryToHistory(effectiveQuery);
queryManager.runQuery(effectiveQuery);
return; return;
} }
const effectiveQuery = preview effectiveQuery = preview
? query.makePreview() ? effectiveQuery.makePreview()
: query.setMaxNumTasksIfUnset(clusterCapacity); : effectiveQuery.setMaxNumTasksIfUnset(clusterCapacity);
const capacityInfo = await maybeGetClusterCapacity(); const capacityInfo = await maybeGetClusterCapacity();
@ -327,9 +332,6 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
const statsTaskId: string | undefined = execution?.id; const statsTaskId: string | undefined = execution?.id;
const queryPrefixes = query.getPrefixQueries();
const extractedCtes = query.extractCteHelpers();
const onUserCancel = (message?: string) => { const onUserCancel = (message?: string) => {
queryManager.cancelCurrent(message); queryManager.cancelCurrent(message);
nativeQueryCancelFnRef.current?.(); nativeQueryCancelFnRef.current?.();
@ -347,81 +349,22 @@ export const QueryTab = React.memo(function QueryTab(props: QueryTabProps) {
> >
<div className="top-section"> <div className="top-section">
<div className="query-section"> <div className="query-section">
{queryPrefixes.map((queryPrefix, i) => ( <FlexibleQueryInput
<HelperQuery ref={queryInputRef}
key={queryPrefix.getId()} queryString={query.getQueryString()}
query={queryPrefix} onQueryStringChange={handleQueryStringChange}
mandatoryQueryContext={mandatoryQueryContext} runQuerySlice={slice => void handleRun(false, slice)}
columnMetadata={columnMetadata} running={executionState.loading}
onQueryChange={newQuery => { columnMetadata={columnMetadata}
onQueryChange(query.applyUpdate(newQuery, i)); editorStateId={id}
}} />
onQueryTab={onQueryTab}
onDelete={() => {
onQueryChange(query.remove(i));
}}
onDetails={onDetails}
queryEngines={queryEngines}
clusterCapacity={clusterCapacity}
goToTask={goToTask}
/>
))}
<div className={classNames('main-query', queryPrefixes.length ? 'multi' : 'single')}>
<FlexibleQueryInput
ref={queryInputRef}
autoHeight={Boolean(queryPrefixes.length)}
minRows={10}
queryString={query.getQueryString()}
onQueryStringChange={handleQueryStringChange}
columnMetadata={
columnMetadata ? columnMetadata.concat(query.getInlineMetadata()) : undefined
}
editorStateId={query.getId()}
/>
<div className="corner">
<Popover2
content={
<Menu>
<MenuItem
icon={IconNames.ARROW_UP}
text="Save as helper query"
onClick={() => {
onQueryChange(query.addBlank());
}}
/>
{extractedCtes !== query && (
<MenuItem
icon={IconNames.DOCUMENT_SHARE}
text="Extract WITH clauses into helper queries"
onClick={() => onQueryChange(extractedCtes)}
/>
)}
{query.hasHelperQueries() && (
<MenuItem
icon={IconNames.DOCUMENT_OPEN}
text="Materialize helper queries"
onClick={() => onQueryChange(query.materializeHelpers())}
/>
)}
<MenuItem
icon={IconNames.DUPLICATE}
text="Duplicate as helper query"
onClick={() => onQueryChange(query.duplicateLast())}
/>
</Menu>
}
>
<Button icon={IconNames.LIST} minimal />
</Popover2>
</div>
</div>
</div> </div>
<div className="run-bar"> <div className="run-bar">
<RunPanel <RunPanel
query={query} query={query}
onQueryChange={onQueryChange} onQueryChange={onQueryChange}
onRun={handleRun} onRun={handleRun}
loading={executionState.loading} running={executionState.loading}
queryEngines={queryEngines} queryEngines={queryEngines}
clusterCapacity={clusterCapacity} clusterCapacity={clusterCapacity}
moreMenu={runMoreMenu} moreMenu={runMoreMenu}

View File

@ -85,7 +85,7 @@ const NAMED_TIMEZONES: string[] = [
export interface RunPanelProps { export interface RunPanelProps {
query: WorkbenchQuery; query: WorkbenchQuery;
onQueryChange(query: WorkbenchQuery): void; onQueryChange(query: WorkbenchQuery): void;
loading: boolean; running: boolean;
small?: boolean; small?: boolean;
onRun(preview: boolean): void | Promise<void>; onRun(preview: boolean): void | Promise<void>;
queryEngines: DruidEngine[]; queryEngines: DruidEngine[];
@ -94,7 +94,7 @@ export interface RunPanelProps {
} }
export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) { export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
const { query, onQueryChange, onRun, moreMenu, loading, small, queryEngines, clusterCapacity } = const { query, onQueryChange, onRun, moreMenu, running, small, queryEngines, clusterCapacity } =
props; props;
const [editContextDialogOpen, setEditContextDialogOpen] = useState(false); const [editContextDialogOpen, setEditContextDialogOpen] = useState(false);
const [customTimezoneDialogOpen, setCustomTimezoneDialogOpen] = useState(false); const [customTimezoneDialogOpen, setCustomTimezoneDialogOpen] = useState(false);
@ -201,7 +201,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
<div className="run-panel"> <div className="run-panel">
<Button <Button
className={effectiveEngine === 'native' ? 'rune-button' : undefined} className={effectiveEngine === 'native' ? 'rune-button' : undefined}
disabled={loading} disabled={running}
icon={IconNames.CARET_RIGHT} icon={IconNames.CARET_RIGHT}
onClick={() => void onRun(false)} onClick={() => void onRun(false)}
text="Run" text="Run"
@ -211,7 +211,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
/> />
{ingestMode && ( {ingestMode && (
<Button <Button
disabled={loading} disabled={running}
icon={IconNames.EYE_OPEN} icon={IconNames.EYE_OPEN}
onClick={() => void onRun(true)} onClick={() => void onRun(true)}
text="Preview" text="Preview"

View File

@ -64,10 +64,10 @@ import { WorkbenchHistoryDialog } from './workbench-history-dialog/workbench-his
import './workbench-view.scss'; import './workbench-view.scss';
function cleanupTabEntry(tabEntry: TabEntry): void { function cleanupTabEntry(tabEntry: TabEntry): void {
const discardedIds = tabEntry.query.getIds(); const discardedId = tabEntry.id;
WorkbenchRunningPromises.deletePromises(discardedIds); WorkbenchRunningPromises.deletePromise(discardedId);
ExecutionStateCache.deleteStates(discardedIds); ExecutionStateCache.deleteState(discardedId);
AceEditorStateCache.deleteStates(discardedIds); AceEditorStateCache.deleteState(discardedId);
} }
function externalDataTabId(tabId: string | undefined): boolean { function externalDataTabId(tabId: string | undefined): boolean {
@ -496,7 +496,7 @@ export class WorkbenchView extends React.PureComponent<WorkbenchViewProps, Workb
const newTabEntry: TabEntry = { const newTabEntry: TabEntry = {
id, id,
tabName: tabEntry.tabName + ' (copy)', tabName: tabEntry.tabName + ' (copy)',
query: tabEntry.query.duplicate(), query: tabEntry.query,
}; };
this.handleQueriesChange( this.handleQueriesChange(
tabEntries.slice(0, i + 1).concat(newTabEntry, tabEntries.slice(i + 1)), tabEntries.slice(0, i + 1).concat(newTabEntry, tabEntries.slice(i + 1)),
@ -639,6 +639,7 @@ export class WorkbenchView extends React.PureComponent<WorkbenchViewProps, Workb
<QueryTab <QueryTab
key={currentTabEntry.id} key={currentTabEntry.id}
query={currentTabEntry.query} query={currentTabEntry.query}
id={currentTabEntry.id}
mandatoryQueryContext={mandatoryQueryContext} mandatoryQueryContext={mandatoryQueryContext}
columnMetadata={columnMetadataState.getSomeData()} columnMetadata={columnMetadataState.getSomeData()}
onQueryChange={this.handleQueryChange} onQueryChange={this.handleQueryChange}