mirror of https://github.com/apache/druid.git
Web console: dynamic query parameters UI (#14921)
* fix nvl in table * add query parameter dialog * pre-wrap in the tables * fix typo
This commit is contained in:
parent
8263f0d1e9
commit
d295b9158f
|
@ -80,6 +80,7 @@ export interface FancyNumericInputProps {
|
||||||
minorStepSize?: number;
|
minorStepSize?: number;
|
||||||
stepSize?: number;
|
stepSize?: number;
|
||||||
majorStepSize?: number;
|
majorStepSize?: number;
|
||||||
|
arbitraryPrecision?: boolean;
|
||||||
}
|
}
|
||||||
|
|
||||||
export const FancyNumericInput = React.memo(function FancyNumericInput(
|
export const FancyNumericInput = React.memo(function FancyNumericInput(
|
||||||
|
@ -103,6 +104,7 @@ export const FancyNumericInput = React.memo(function FancyNumericInput(
|
||||||
|
|
||||||
min,
|
min,
|
||||||
max,
|
max,
|
||||||
|
arbitraryPrecision,
|
||||||
} = props;
|
} = props;
|
||||||
|
|
||||||
const stepSize = props.stepSize || 1;
|
const stepSize = props.stepSize || 1;
|
||||||
|
@ -110,8 +112,11 @@ export const FancyNumericInput = React.memo(function FancyNumericInput(
|
||||||
const majorStepSize = props.majorStepSize || stepSize * 10;
|
const majorStepSize = props.majorStepSize || stepSize * 10;
|
||||||
|
|
||||||
function roundAndClamp(n: number): number {
|
function roundAndClamp(n: number): number {
|
||||||
const inv = 1 / minorStepSize;
|
if (!arbitraryPrecision) {
|
||||||
return clamp(Math.floor(n * inv) / inv, min, max);
|
const inv = 1 / minorStepSize;
|
||||||
|
n = Math.floor(n * inv) / inv;
|
||||||
|
}
|
||||||
|
return clamp(n, min, max);
|
||||||
}
|
}
|
||||||
|
|
||||||
const effectiveValue = value ?? defaultValue;
|
const effectiveValue = value ?? defaultValue;
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
|
|
||||||
.table-cell {
|
.table-cell {
|
||||||
padding: $table-cell-v-padding $table-cell-h-padding;
|
padding: $table-cell-v-padding $table-cell-h-padding;
|
||||||
|
white-space: pre;
|
||||||
|
|
||||||
&.null,
|
&.null,
|
||||||
&.empty {
|
&.empty {
|
||||||
|
|
|
@ -92,7 +92,7 @@ export const KillDatasourceDialog = function KillDatasourceDialog(
|
||||||
format.
|
format.
|
||||||
</p>
|
</p>
|
||||||
<p>
|
<p>
|
||||||
If you have streaming ingestion running make sure that your interval range doe not
|
If you have streaming ingestion running make sure that your interval range does not
|
||||||
overlap with intervals where streaming data is being added - otherwise the kill task
|
overlap with intervals where streaming data is being added - otherwise the kill task
|
||||||
will not start.
|
will not start.
|
||||||
</p>
|
</p>
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import type {
|
import type {
|
||||||
|
QueryParameter,
|
||||||
SqlClusteredByClause,
|
SqlClusteredByClause,
|
||||||
SqlExpression,
|
SqlExpression,
|
||||||
SqlPartitionedByClause,
|
SqlPartitionedByClause,
|
||||||
|
@ -66,6 +67,7 @@ interface IngestionLines {
|
||||||
export interface WorkbenchQueryValue {
|
export interface WorkbenchQueryValue {
|
||||||
queryString: string;
|
queryString: string;
|
||||||
queryContext: QueryContext;
|
queryContext: QueryContext;
|
||||||
|
queryParameters?: QueryParameter[];
|
||||||
engine?: DruidEngine;
|
engine?: DruidEngine;
|
||||||
lastExecution?: LastExecution;
|
lastExecution?: LastExecution;
|
||||||
unlimited?: boolean;
|
unlimited?: boolean;
|
||||||
|
@ -235,6 +237,7 @@ export class WorkbenchQuery {
|
||||||
|
|
||||||
public readonly queryString: string;
|
public readonly queryString: string;
|
||||||
public readonly queryContext: QueryContext;
|
public readonly queryContext: QueryContext;
|
||||||
|
public readonly queryParameters?: QueryParameter[];
|
||||||
public readonly engine?: DruidEngine;
|
public readonly engine?: DruidEngine;
|
||||||
public readonly lastExecution?: LastExecution;
|
public readonly lastExecution?: LastExecution;
|
||||||
public readonly unlimited?: boolean;
|
public readonly unlimited?: boolean;
|
||||||
|
@ -251,6 +254,7 @@ export class WorkbenchQuery {
|
||||||
}
|
}
|
||||||
this.queryString = queryString;
|
this.queryString = queryString;
|
||||||
this.queryContext = value.queryContext;
|
this.queryContext = value.queryContext;
|
||||||
|
this.queryParameters = value.queryParameters;
|
||||||
|
|
||||||
// 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
|
||||||
let possibleEngine: string | undefined = value.engine;
|
let possibleEngine: string | undefined = value.engine;
|
||||||
|
@ -274,6 +278,7 @@ export class WorkbenchQuery {
|
||||||
return {
|
return {
|
||||||
queryString: this.queryString,
|
queryString: this.queryString,
|
||||||
queryContext: this.queryContext,
|
queryContext: this.queryContext,
|
||||||
|
queryParameters: this.queryParameters,
|
||||||
engine: this.engine,
|
engine: this.engine,
|
||||||
unlimited: this.unlimited,
|
unlimited: this.unlimited,
|
||||||
};
|
};
|
||||||
|
@ -297,6 +302,10 @@ export class WorkbenchQuery {
|
||||||
return new WorkbenchQuery({ ...this.valueOf(), queryContext });
|
return new WorkbenchQuery({ ...this.valueOf(), queryContext });
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public changeQueryParameters(queryParameters: QueryParameter[] | undefined): WorkbenchQuery {
|
||||||
|
return new WorkbenchQuery({ ...this.valueOf(), queryParameters });
|
||||||
|
}
|
||||||
|
|
||||||
public changeEngine(engine: DruidEngine | undefined): WorkbenchQuery {
|
public changeEngine(engine: DruidEngine | undefined): WorkbenchQuery {
|
||||||
return new WorkbenchQuery({ ...this.valueOf(), engine });
|
return new WorkbenchQuery({ ...this.valueOf(), engine });
|
||||||
}
|
}
|
||||||
|
@ -425,11 +434,12 @@ export class WorkbenchQuery {
|
||||||
let ret: WorkbenchQuery = this;
|
let ret: WorkbenchQuery = this;
|
||||||
|
|
||||||
// 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
|
||||||
|
const { queryContext } = this;
|
||||||
ret = ret.changeEngine('sql-msq-task').changeQueryContext({
|
ret = ret.changeEngine('sql-msq-task').changeQueryContext({
|
||||||
...this.queryContext,
|
...queryContext,
|
||||||
maxNumTasks: 2,
|
maxNumTasks: 2,
|
||||||
finalizeAggregations: false,
|
finalizeAggregations: queryContext.finalizeAggregations ?? false,
|
||||||
groupByEnableMultiValueUnnesting: false,
|
groupByEnableMultiValueUnnesting: queryContext.groupByEnableMultiValueUnnesting ?? false,
|
||||||
});
|
});
|
||||||
|
|
||||||
// Remove everything pertaining to INSERT INTO / REPLACE INTO from the query string
|
// Remove everything pertaining to INSERT INTO / REPLACE INTO from the query string
|
||||||
|
@ -458,7 +468,7 @@ export class WorkbenchQuery {
|
||||||
prefixLines: number;
|
prefixLines: number;
|
||||||
cancelQueryId?: string;
|
cancelQueryId?: string;
|
||||||
} {
|
} {
|
||||||
const { queryString, queryContext, unlimited, prefixLines } = this;
|
const { queryString, queryContext, queryParameters, unlimited, prefixLines } = this;
|
||||||
const engine = this.getEffectiveEngine();
|
const engine = this.getEffectiveEngine();
|
||||||
|
|
||||||
if (engine === 'native') {
|
if (engine === 'native') {
|
||||||
|
@ -544,6 +554,10 @@ export class WorkbenchQuery {
|
||||||
apiQuery.context.groupByEnableMultiValueUnnesting ??= !ingestQuery;
|
apiQuery.context.groupByEnableMultiValueUnnesting ??= !ingestQuery;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (Array.isArray(queryParameters) && queryParameters.length) {
|
||||||
|
apiQuery.parameters = queryParameters;
|
||||||
|
}
|
||||||
|
|
||||||
return {
|
return {
|
||||||
engine,
|
engine,
|
||||||
query: apiQuery,
|
query: apiQuery,
|
||||||
|
|
|
@ -29,7 +29,6 @@ export * from './local-storage-backed-visibility';
|
||||||
export * from './local-storage-keys';
|
export * from './local-storage-keys';
|
||||||
export * from './object-change';
|
export * from './object-change';
|
||||||
export * from './query-action';
|
export * from './query-action';
|
||||||
export * from './query-cursor';
|
|
||||||
export * from './query-manager';
|
export * from './query-manager';
|
||||||
export * from './query-state';
|
export * from './query-state';
|
||||||
export * from './sample-query';
|
export * from './sample-query';
|
||||||
|
|
|
@ -1,55 +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 { SqlBase, SqlQuery } from '@druid-toolkit/query';
|
|
||||||
import { L } from '@druid-toolkit/query';
|
|
||||||
|
|
||||||
import type { RowColumn } from './general';
|
|
||||||
|
|
||||||
export const EMPTY_LITERAL = L('');
|
|
||||||
|
|
||||||
const CRAZY_STRING = '$.X.@.X.$';
|
|
||||||
const DOT_DOT_DOT_LITERAL = L('...');
|
|
||||||
|
|
||||||
export function prettyPrintSql(b: SqlBase): string {
|
|
||||||
return b
|
|
||||||
.walk(b => {
|
|
||||||
if (b === EMPTY_LITERAL) {
|
|
||||||
return DOT_DOT_DOT_LITERAL;
|
|
||||||
}
|
|
||||||
return b;
|
|
||||||
})
|
|
||||||
.prettyTrim(50)
|
|
||||||
.toString();
|
|
||||||
}
|
|
||||||
|
|
||||||
export function findEmptyLiteralPosition(query: SqlQuery): RowColumn | undefined {
|
|
||||||
const subQueryString = query.walk(b => (b === EMPTY_LITERAL ? L(CRAZY_STRING) : b)).toString();
|
|
||||||
|
|
||||||
const crazyIndex = subQueryString.indexOf(CRAZY_STRING);
|
|
||||||
if (crazyIndex < 0) return;
|
|
||||||
|
|
||||||
const prefix = subQueryString.slice(0, crazyIndex);
|
|
||||||
const lines = prefix.split(/\n/g);
|
|
||||||
const row = lines.length - 1;
|
|
||||||
const lastLine = lines[row];
|
|
||||||
return {
|
|
||||||
row: row,
|
|
||||||
column: lastLine.length,
|
|
||||||
};
|
|
||||||
}
|
|
|
@ -16,6 +16,7 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import type { SqlBase } from '@druid-toolkit/query';
|
||||||
import {
|
import {
|
||||||
SqlColumn,
|
SqlColumn,
|
||||||
SqlExpression,
|
SqlExpression,
|
||||||
|
@ -28,6 +29,10 @@ import {
|
||||||
import type { RowColumn } from './general';
|
import type { RowColumn } from './general';
|
||||||
import { offsetToRowColumn } from './general';
|
import { offsetToRowColumn } from './general';
|
||||||
|
|
||||||
|
export function prettyPrintSql(b: SqlBase): string {
|
||||||
|
return b.prettyTrim(50).toString();
|
||||||
|
}
|
||||||
|
|
||||||
export function timeFormatToSql(timeFormat: string): SqlExpression | undefined {
|
export function timeFormatToSql(timeFormat: string): SqlExpression | undefined {
|
||||||
switch (timeFormat) {
|
switch (timeFormat) {
|
||||||
case 'auto':
|
case 'auto':
|
||||||
|
|
|
@ -89,6 +89,9 @@ export function dataTypeToIcon(dataType: string): IconName {
|
||||||
case 'COMPLEX<IPPREFIX>':
|
case 'COMPLEX<IPPREFIX>':
|
||||||
return IconNames.IP_ADDRESS;
|
return IconNames.IP_ADDRESS;
|
||||||
|
|
||||||
|
case 'COMPLEX<SERIALIZABLEPAIRLONGSTRING>':
|
||||||
|
return IconNames.DOUBLE_CHEVRON_RIGHT;
|
||||||
|
|
||||||
case 'NULL':
|
case 'NULL':
|
||||||
return IconNames.CIRCLE;
|
return IconNames.CIRCLE;
|
||||||
|
|
||||||
|
|
|
@ -383,8 +383,8 @@ export class DatasourcesView extends React.PureComponent<
|
||||||
return `SELECT
|
return `SELECT
|
||||||
${columns.join(',\n')}
|
${columns.join(',\n')}
|
||||||
FROM sys.segments
|
FROM sys.segments
|
||||||
GROUP BY 1
|
GROUP BY datasource
|
||||||
ORDER BY 1`;
|
ORDER BY datasource`;
|
||||||
}
|
}
|
||||||
|
|
||||||
static RUNNING_TASK_SQL = `SELECT
|
static RUNNING_TASK_SQL = `SELECT
|
||||||
|
|
|
@ -16,12 +16,11 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import type { SqlOrderByExpression } from '@druid-toolkit/query';
|
import type { SqlColumn, SqlOrderByExpression } from '@druid-toolkit/query';
|
||||||
import {
|
import {
|
||||||
C,
|
C,
|
||||||
F,
|
F,
|
||||||
SqlCase,
|
SqlCase,
|
||||||
SqlColumn,
|
|
||||||
SqlExpression,
|
SqlExpression,
|
||||||
SqlFunction,
|
SqlFunction,
|
||||||
SqlLiteral,
|
SqlLiteral,
|
||||||
|
@ -80,22 +79,24 @@ function nullableColumn(column: ExpressionMeta) {
|
||||||
}
|
}
|
||||||
|
|
||||||
function nvl(ex: SqlExpression): SqlExpression {
|
function nvl(ex: SqlExpression): SqlExpression {
|
||||||
return SqlFunction.simple('NVL', [ex, NULL_REPLACEMENT]);
|
return SqlFunction.simple('NVL', [ex.cast('VARCHAR'), NULL_REPLACEMENT]);
|
||||||
}
|
}
|
||||||
|
|
||||||
function nullif(ex: SqlExpression): SqlExpression {
|
function joinEquals(c1: SqlColumn, c2: SqlColumn, nullable: boolean): SqlExpression {
|
||||||
return SqlFunction.simple('NULLIF', [ex, NULL_REPLACEMENT]);
|
return c1.applyIf(nullable, nvl).equal(c2.applyIf(nullable, nvl));
|
||||||
}
|
}
|
||||||
|
|
||||||
function toGroupByExpression(
|
function toGroupByExpression(
|
||||||
splitColumn: ExpressionMeta,
|
splitColumn: ExpressionMeta,
|
||||||
nvlIfNeeded: boolean,
|
|
||||||
timeBucket: string,
|
timeBucket: string,
|
||||||
|
compareShiftDuration?: string,
|
||||||
) {
|
) {
|
||||||
const { expression, sqlType, name } = splitColumn;
|
const { expression, sqlType, name } = splitColumn;
|
||||||
return expression
|
return expression
|
||||||
.applyIf(sqlType === 'TIMESTAMP', e => SqlFunction.simple('TIME_FLOOR', [e, timeBucket]))
|
.applyIf(sqlType === 'TIMESTAMP' && compareShiftDuration, e =>
|
||||||
.applyIf(nvlIfNeeded && nullableColumn(splitColumn), nvl)
|
F.timeShift(e, compareShiftDuration!, 1),
|
||||||
|
)
|
||||||
|
.applyIf(sqlType === 'TIMESTAMP', e => F.timeFloor(e, timeBucket))
|
||||||
.as(name);
|
.as(name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -143,16 +144,6 @@ function toShowColumnExpression(
|
||||||
return ex.as(showColumn.name);
|
return ex.as(showColumn.name);
|
||||||
}
|
}
|
||||||
|
|
||||||
function shiftTime(ex: SqlQuery, period: string): SqlQuery {
|
|
||||||
return ex.walk(q => {
|
|
||||||
if (q instanceof SqlColumn && q.getName() === '__time') {
|
|
||||||
return SqlFunction.simple('TIME_SHIFT', [q, period, 1]);
|
|
||||||
} else {
|
|
||||||
return q;
|
|
||||||
}
|
|
||||||
}) as SqlQuery;
|
|
||||||
}
|
|
||||||
|
|
||||||
interface QueryAndHints {
|
interface QueryAndHints {
|
||||||
query: SqlQuery;
|
query: SqlQuery;
|
||||||
groupHints: string[];
|
groupHints: string[];
|
||||||
|
@ -327,7 +318,7 @@ function TableModule(props: TableModuleProps) {
|
||||||
|
|
||||||
const mainQuery = getInitQuery(table, where)
|
const mainQuery = getInitQuery(table, where)
|
||||||
.applyForEach(splitColumns, (q, splitColumn) =>
|
.applyForEach(splitColumns, (q, splitColumn) =>
|
||||||
q.addSelect(toGroupByExpression(splitColumn, hasCompare, timeBucket), {
|
q.addSelect(toGroupByExpression(splitColumn, timeBucket), {
|
||||||
addToGroupBy: 'end',
|
addToGroupBy: 'end',
|
||||||
}),
|
}),
|
||||||
)
|
)
|
||||||
|
@ -381,26 +372,20 @@ function TableModule(props: TableModuleProps) {
|
||||||
`compare${i}`,
|
`compare${i}`,
|
||||||
getInitQuery(table, where)
|
getInitQuery(table, where)
|
||||||
.applyForEach(splitColumns, (q, splitColumn) =>
|
.applyForEach(splitColumns, (q, splitColumn) =>
|
||||||
q.addSelect(toGroupByExpression(splitColumn, true, timeBucket), {
|
q.addSelect(toGroupByExpression(splitColumn, timeBucket, comparePeriod), {
|
||||||
addToGroupBy: 'end',
|
addToGroupBy: 'end',
|
||||||
}),
|
}),
|
||||||
)
|
)
|
||||||
.applyForEach(metrics, (q, metric) =>
|
.applyForEach(metrics, (q, metric) =>
|
||||||
q.addSelect(metric.expression.as(metric.name)),
|
q.addSelect(metric.expression.as(metric.name)),
|
||||||
)
|
),
|
||||||
.apply(q => shiftTime(q, comparePeriod)),
|
|
||||||
),
|
),
|
||||||
),
|
),
|
||||||
),
|
),
|
||||||
)
|
)
|
||||||
.changeSelectExpressions(
|
.changeSelectExpressions(
|
||||||
splitColumns
|
splitColumns
|
||||||
.map(splitColumn =>
|
.map(splitColumn => main.column(splitColumn.name).as(splitColumn.name))
|
||||||
main
|
|
||||||
.column(splitColumn.name)
|
|
||||||
.applyIf(nullableColumn(splitColumn), nullif)
|
|
||||||
.as(splitColumn.name),
|
|
||||||
)
|
|
||||||
.concat(
|
.concat(
|
||||||
showColumns.map(showColumn => main.column(showColumn.name).as(showColumn.name)),
|
showColumns.map(showColumn => main.column(showColumn.name).as(showColumn.name)),
|
||||||
metrics.map(metric => main.column(metric.name).as(metric.name)),
|
metrics.map(metric => main.column(metric.name).as(metric.name)),
|
||||||
|
@ -432,7 +417,11 @@ function TableModule(props: TableModuleProps) {
|
||||||
T(`compare${i}`),
|
T(`compare${i}`),
|
||||||
SqlExpression.and(
|
SqlExpression.and(
|
||||||
...splitColumns.map(splitColumn =>
|
...splitColumns.map(splitColumn =>
|
||||||
main.column(splitColumn.name).equal(T(`compare${i}`).column(splitColumn.name)),
|
joinEquals(
|
||||||
|
main.column(splitColumn.name),
|
||||||
|
T(`compare${i}`).column(splitColumn.name),
|
||||||
|
nullableColumn(splitColumn),
|
||||||
|
),
|
||||||
),
|
),
|
||||||
),
|
),
|
||||||
),
|
),
|
||||||
|
|
|
@ -19,11 +19,11 @@
|
||||||
import { MenuItem } from '@blueprintjs/core';
|
import { MenuItem } from '@blueprintjs/core';
|
||||||
import { IconNames } from '@blueprintjs/icons';
|
import { IconNames } from '@blueprintjs/icons';
|
||||||
import type { SqlExpression, SqlQuery } from '@druid-toolkit/query';
|
import type { SqlExpression, SqlQuery } from '@druid-toolkit/query';
|
||||||
import { C, F, N, SqlJoinPart, T } from '@druid-toolkit/query';
|
import { C, F, N, SqlJoinPart, SqlPlaceholder, T } from '@druid-toolkit/query';
|
||||||
import type { JSX } from 'react';
|
import type { JSX } from 'react';
|
||||||
import React from 'react';
|
import React from 'react';
|
||||||
|
|
||||||
import { EMPTY_LITERAL, prettyPrintSql } from '../../../../../utils';
|
import { prettyPrintSql } from '../../../../../utils';
|
||||||
import { getJoinColumns } from '../../column-tree';
|
import { getJoinColumns } from '../../column-tree';
|
||||||
|
|
||||||
export interface StringMenuItemsProps {
|
export interface StringMenuItemsProps {
|
||||||
|
@ -53,9 +53,9 @@ export const StringMenuItems = React.memo(function StringMenuItems(props: String
|
||||||
return (
|
return (
|
||||||
<MenuItem icon={IconNames.FILTER} text="Filter">
|
<MenuItem icon={IconNames.FILTER} text="Filter">
|
||||||
{filterMenuItem(column.isNotNull())}
|
{filterMenuItem(column.isNotNull())}
|
||||||
{filterMenuItem(column.equal(EMPTY_LITERAL), false)}
|
{filterMenuItem(column.equal(SqlPlaceholder.PLACEHOLDER), false)}
|
||||||
{filterMenuItem(column.like(EMPTY_LITERAL), false)}
|
{filterMenuItem(column.like(SqlPlaceholder.PLACEHOLDER), false)}
|
||||||
{filterMenuItem(F('REGEXP_LIKE', column, EMPTY_LITERAL), false)}
|
{filterMenuItem(F('REGEXP_LIKE', column, SqlPlaceholder.PLACEHOLDER), false)}
|
||||||
</MenuItem>
|
</MenuItem>
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -136,7 +136,7 @@ export const StringMenuItems = React.memo(function StringMenuItems(props: String
|
||||||
<MenuItem icon={IconNames.FUNCTION} text="Aggregate">
|
<MenuItem icon={IconNames.FUNCTION} text="Aggregate">
|
||||||
{aggregateMenuItem(F.countDistinct(column), `dist_${columnName}`)}
|
{aggregateMenuItem(F.countDistinct(column), `dist_${columnName}`)}
|
||||||
{aggregateMenuItem(
|
{aggregateMenuItem(
|
||||||
F.count().addWhereExpression(column.equal(EMPTY_LITERAL)),
|
F.count().addWhereExpression(column.equal(SqlPlaceholder.PLACEHOLDER)),
|
||||||
`filtered_dist_${columnName}`,
|
`filtered_dist_${columnName}`,
|
||||||
false,
|
false,
|
||||||
)}
|
)}
|
||||||
|
|
|
@ -0,0 +1,457 @@
|
||||||
|
// Jest Snapshot v1, https://goo.gl/fbAQLP
|
||||||
|
|
||||||
|
exports[`QueryParametersDialog matches snapshot 1`] = `
|
||||||
|
<Blueprint4.Dialog
|
||||||
|
canOutsideClickClose={true}
|
||||||
|
className="query-parameters-dialog"
|
||||||
|
isOpen={true}
|
||||||
|
onClose={[Function]}
|
||||||
|
title="Dynamic query parameters"
|
||||||
|
>
|
||||||
|
<div
|
||||||
|
className="bp4-dialog-body"
|
||||||
|
>
|
||||||
|
<p>
|
||||||
|
Druid SQL supports dynamic parameters using question mark
|
||||||
|
<Unknown>
|
||||||
|
?
|
||||||
|
</Unknown>
|
||||||
|
syntax, where parameters are bound positionally to ? placeholders at execution time.
|
||||||
|
</p>
|
||||||
|
<Blueprint4.FormGroup
|
||||||
|
label="Parameter in position 1"
|
||||||
|
>
|
||||||
|
<Blueprint4.ControlGroup
|
||||||
|
fill={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Popover2
|
||||||
|
boundary="clippingParents"
|
||||||
|
captureDismiss={false}
|
||||||
|
content={
|
||||||
|
<Blueprint4.Menu>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="tick"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="VARCHAR"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="TIMESTAMP"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="BIGINT"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="DOUBLE"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="FLOAT"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Menu>
|
||||||
|
}
|
||||||
|
defaultIsOpen={false}
|
||||||
|
disabled={false}
|
||||||
|
fill={false}
|
||||||
|
hasBackdrop={false}
|
||||||
|
hoverCloseDelay={300}
|
||||||
|
hoverOpenDelay={150}
|
||||||
|
inheritDarkTheme={true}
|
||||||
|
interactionKind="click"
|
||||||
|
matchTargetWidth={false}
|
||||||
|
minimal={true}
|
||||||
|
openOnTargetFocus={true}
|
||||||
|
position="bottom-left"
|
||||||
|
positioningStrategy="absolute"
|
||||||
|
shouldReturnFocusOnClose={false}
|
||||||
|
targetTagName="span"
|
||||||
|
transitionDuration={300}
|
||||||
|
usePortal={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Button
|
||||||
|
rightIcon="caret-down"
|
||||||
|
text="VARCHAR"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Popover2>
|
||||||
|
<Blueprint4.InputGroup
|
||||||
|
fill={true}
|
||||||
|
onChange={[Function]}
|
||||||
|
placeholder="Parameter value"
|
||||||
|
value="Hello world"
|
||||||
|
/>
|
||||||
|
<Blueprint4.Button
|
||||||
|
icon="trash"
|
||||||
|
onClick={[Function]}
|
||||||
|
/>
|
||||||
|
</Blueprint4.ControlGroup>
|
||||||
|
</Blueprint4.FormGroup>
|
||||||
|
<Blueprint4.FormGroup
|
||||||
|
label="Parameter in position 2"
|
||||||
|
>
|
||||||
|
<Blueprint4.ControlGroup
|
||||||
|
fill={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Popover2
|
||||||
|
boundary="clippingParents"
|
||||||
|
captureDismiss={false}
|
||||||
|
content={
|
||||||
|
<Blueprint4.Menu>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="VARCHAR"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="tick"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="TIMESTAMP"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="BIGINT"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="DOUBLE"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="FLOAT"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Menu>
|
||||||
|
}
|
||||||
|
defaultIsOpen={false}
|
||||||
|
disabled={false}
|
||||||
|
fill={false}
|
||||||
|
hasBackdrop={false}
|
||||||
|
hoverCloseDelay={300}
|
||||||
|
hoverOpenDelay={150}
|
||||||
|
inheritDarkTheme={true}
|
||||||
|
interactionKind="click"
|
||||||
|
matchTargetWidth={false}
|
||||||
|
minimal={true}
|
||||||
|
openOnTargetFocus={true}
|
||||||
|
position="bottom-left"
|
||||||
|
positioningStrategy="absolute"
|
||||||
|
shouldReturnFocusOnClose={false}
|
||||||
|
targetTagName="span"
|
||||||
|
transitionDuration={300}
|
||||||
|
usePortal={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Button
|
||||||
|
rightIcon="caret-down"
|
||||||
|
text="TIMESTAMP"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Popover2>
|
||||||
|
<Blueprint4.InputGroup
|
||||||
|
fill={true}
|
||||||
|
onChange={[Function]}
|
||||||
|
placeholder="2022-01-01 00:00:00"
|
||||||
|
value="2022-02-02 01:02:03"
|
||||||
|
/>
|
||||||
|
<Blueprint4.Button
|
||||||
|
icon="trash"
|
||||||
|
onClick={[Function]}
|
||||||
|
/>
|
||||||
|
</Blueprint4.ControlGroup>
|
||||||
|
</Blueprint4.FormGroup>
|
||||||
|
<Blueprint4.FormGroup
|
||||||
|
label="Parameter in position 3"
|
||||||
|
>
|
||||||
|
<Blueprint4.ControlGroup
|
||||||
|
fill={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Popover2
|
||||||
|
boundary="clippingParents"
|
||||||
|
captureDismiss={false}
|
||||||
|
content={
|
||||||
|
<Blueprint4.Menu>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="VARCHAR"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="TIMESTAMP"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="tick"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="BIGINT"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="DOUBLE"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="FLOAT"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Menu>
|
||||||
|
}
|
||||||
|
defaultIsOpen={false}
|
||||||
|
disabled={false}
|
||||||
|
fill={false}
|
||||||
|
hasBackdrop={false}
|
||||||
|
hoverCloseDelay={300}
|
||||||
|
hoverOpenDelay={150}
|
||||||
|
inheritDarkTheme={true}
|
||||||
|
interactionKind="click"
|
||||||
|
matchTargetWidth={false}
|
||||||
|
minimal={true}
|
||||||
|
openOnTargetFocus={true}
|
||||||
|
position="bottom-left"
|
||||||
|
positioningStrategy="absolute"
|
||||||
|
shouldReturnFocusOnClose={false}
|
||||||
|
targetTagName="span"
|
||||||
|
transitionDuration={300}
|
||||||
|
usePortal={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Button
|
||||||
|
rightIcon="caret-down"
|
||||||
|
text="BIGINT"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Popover2>
|
||||||
|
<Memo(FancyNumericInput)
|
||||||
|
arbitraryPrecision={false}
|
||||||
|
fill={true}
|
||||||
|
onValueChange={[Function]}
|
||||||
|
value={42}
|
||||||
|
/>
|
||||||
|
<Blueprint4.Button
|
||||||
|
icon="trash"
|
||||||
|
onClick={[Function]}
|
||||||
|
/>
|
||||||
|
</Blueprint4.ControlGroup>
|
||||||
|
</Blueprint4.FormGroup>
|
||||||
|
<Blueprint4.FormGroup
|
||||||
|
label="Parameter in position 4"
|
||||||
|
>
|
||||||
|
<Blueprint4.ControlGroup
|
||||||
|
fill={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Popover2
|
||||||
|
boundary="clippingParents"
|
||||||
|
captureDismiss={false}
|
||||||
|
content={
|
||||||
|
<Blueprint4.Menu>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="VARCHAR"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="TIMESTAMP"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="BIGINT"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="tick"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="DOUBLE"
|
||||||
|
/>
|
||||||
|
<Blueprint4.MenuItem
|
||||||
|
active={false}
|
||||||
|
disabled={false}
|
||||||
|
icon="blank"
|
||||||
|
multiline={false}
|
||||||
|
onClick={[Function]}
|
||||||
|
popoverProps={Object {}}
|
||||||
|
selected={false}
|
||||||
|
shouldDismissPopover={true}
|
||||||
|
text="FLOAT"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Menu>
|
||||||
|
}
|
||||||
|
defaultIsOpen={false}
|
||||||
|
disabled={false}
|
||||||
|
fill={false}
|
||||||
|
hasBackdrop={false}
|
||||||
|
hoverCloseDelay={300}
|
||||||
|
hoverOpenDelay={150}
|
||||||
|
inheritDarkTheme={true}
|
||||||
|
interactionKind="click"
|
||||||
|
matchTargetWidth={false}
|
||||||
|
minimal={true}
|
||||||
|
openOnTargetFocus={true}
|
||||||
|
position="bottom-left"
|
||||||
|
positioningStrategy="absolute"
|
||||||
|
shouldReturnFocusOnClose={false}
|
||||||
|
targetTagName="span"
|
||||||
|
transitionDuration={300}
|
||||||
|
usePortal={true}
|
||||||
|
>
|
||||||
|
<Blueprint4.Button
|
||||||
|
rightIcon="caret-down"
|
||||||
|
text="DOUBLE"
|
||||||
|
/>
|
||||||
|
</Blueprint4.Popover2>
|
||||||
|
<Memo(FancyNumericInput)
|
||||||
|
arbitraryPrecision={true}
|
||||||
|
fill={true}
|
||||||
|
onValueChange={[Function]}
|
||||||
|
value={1.618}
|
||||||
|
/>
|
||||||
|
<Blueprint4.Button
|
||||||
|
icon="trash"
|
||||||
|
onClick={[Function]}
|
||||||
|
/>
|
||||||
|
</Blueprint4.ControlGroup>
|
||||||
|
</Blueprint4.FormGroup>
|
||||||
|
<Blueprint4.Button
|
||||||
|
icon="plus"
|
||||||
|
onClick={[Function]}
|
||||||
|
text="Add parameter"
|
||||||
|
/>
|
||||||
|
</div>
|
||||||
|
<div
|
||||||
|
className="bp4-dialog-footer"
|
||||||
|
>
|
||||||
|
<div
|
||||||
|
className="bp4-dialog-footer-actions"
|
||||||
|
>
|
||||||
|
<Blueprint4.Button
|
||||||
|
onClick={[Function]}
|
||||||
|
text="Close"
|
||||||
|
/>
|
||||||
|
<Blueprint4.Button
|
||||||
|
intent="primary"
|
||||||
|
onClick={[Function]}
|
||||||
|
text="Save"
|
||||||
|
/>
|
||||||
|
</div>
|
||||||
|
</div>
|
||||||
|
</Blueprint4.Dialog>
|
||||||
|
`;
|
|
@ -0,0 +1,28 @@
|
||||||
|
/*
|
||||||
|
* 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';
|
||||||
|
|
||||||
|
.query-parameters-dialog {
|
||||||
|
.#{$bp-ns}-dialog-body {
|
||||||
|
position: relative;
|
||||||
|
min-height: 50vh;
|
||||||
|
overflow: auto;
|
||||||
|
max-height: 80vh;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,42 @@
|
||||||
|
/*
|
||||||
|
* 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 React from 'react';
|
||||||
|
|
||||||
|
import { shallow } from '../../../utils/shallow-renderer';
|
||||||
|
|
||||||
|
import { QueryParametersDialog } from './query-parameters-dialog';
|
||||||
|
|
||||||
|
describe('QueryParametersDialog', () => {
|
||||||
|
it('matches snapshot', () => {
|
||||||
|
const comp = shallow(
|
||||||
|
<QueryParametersDialog
|
||||||
|
queryParameters={[
|
||||||
|
{ type: 'VARCHAR', value: 'Hello world' },
|
||||||
|
{ type: 'TIMESTAMP', value: '2022-02-02 01:02:03' },
|
||||||
|
{ type: 'BIGINT', value: 42 },
|
||||||
|
{ type: 'DOUBLE', value: 1.618 },
|
||||||
|
]}
|
||||||
|
onQueryParametersChange={() => {}}
|
||||||
|
onClose={() => {}}
|
||||||
|
/>,
|
||||||
|
);
|
||||||
|
|
||||||
|
expect(comp).toMatchSnapshot();
|
||||||
|
});
|
||||||
|
});
|
|
@ -0,0 +1,152 @@
|
||||||
|
/*
|
||||||
|
* 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,
|
||||||
|
Classes,
|
||||||
|
Code,
|
||||||
|
ControlGroup,
|
||||||
|
Dialog,
|
||||||
|
FormGroup,
|
||||||
|
InputGroup,
|
||||||
|
Intent,
|
||||||
|
Menu,
|
||||||
|
MenuItem,
|
||||||
|
Position,
|
||||||
|
} from '@blueprintjs/core';
|
||||||
|
import { IconNames } from '@blueprintjs/icons';
|
||||||
|
import { Popover2 } from '@blueprintjs/popover2';
|
||||||
|
import type { QueryParameter } from '@druid-toolkit/query';
|
||||||
|
import { isEmptyArray } from '@druid-toolkit/query';
|
||||||
|
import React, { useState } from 'react';
|
||||||
|
|
||||||
|
import { FancyNumericInput } from '../../../components/fancy-numeric-input/fancy-numeric-input';
|
||||||
|
import { deepSet, oneOf, tickIcon, without } from '../../../utils';
|
||||||
|
|
||||||
|
import './query-parameters-dialog.scss';
|
||||||
|
|
||||||
|
const TYPES = ['VARCHAR', 'TIMESTAMP', 'BIGINT', 'DOUBLE', 'FLOAT'];
|
||||||
|
|
||||||
|
interface QueryParametersDialogProps {
|
||||||
|
queryParameters: QueryParameter[] | undefined;
|
||||||
|
onQueryParametersChange(parameters: QueryParameter[] | undefined): void;
|
||||||
|
onClose(): void;
|
||||||
|
}
|
||||||
|
|
||||||
|
export const QueryParametersDialog = React.memo(function QueryParametersDialog(
|
||||||
|
props: QueryParametersDialogProps,
|
||||||
|
) {
|
||||||
|
const { queryParameters, onQueryParametersChange, onClose } = props;
|
||||||
|
const [currentQueryParameters, setCurrentQueryParameters] = useState(queryParameters || []);
|
||||||
|
|
||||||
|
function onSave() {
|
||||||
|
onQueryParametersChange(
|
||||||
|
isEmptyArray(currentQueryParameters) ? undefined : currentQueryParameters,
|
||||||
|
);
|
||||||
|
onClose();
|
||||||
|
}
|
||||||
|
|
||||||
|
return (
|
||||||
|
<Dialog
|
||||||
|
className="query-parameters-dialog"
|
||||||
|
isOpen
|
||||||
|
onClose={onClose}
|
||||||
|
title="Dynamic query parameters"
|
||||||
|
>
|
||||||
|
<div className={Classes.DIALOG_BODY}>
|
||||||
|
<p>
|
||||||
|
Druid SQL supports dynamic parameters using question mark <Code>?</Code> syntax, where
|
||||||
|
parameters are bound positionally to ? placeholders at execution time.
|
||||||
|
</p>
|
||||||
|
{currentQueryParameters.map((queryParameter, i) => {
|
||||||
|
const { type, value } = queryParameter;
|
||||||
|
|
||||||
|
function onValueChange(v: string | number) {
|
||||||
|
setCurrentQueryParameters(deepSet(currentQueryParameters, `${i}.value`, v));
|
||||||
|
}
|
||||||
|
|
||||||
|
return (
|
||||||
|
<FormGroup key={i} label={`Parameter in position ${i + 1}`}>
|
||||||
|
<ControlGroup fill>
|
||||||
|
<Popover2
|
||||||
|
minimal
|
||||||
|
position={Position.BOTTOM_LEFT}
|
||||||
|
content={
|
||||||
|
<Menu>
|
||||||
|
{TYPES.map(t => (
|
||||||
|
<MenuItem
|
||||||
|
key={t}
|
||||||
|
icon={tickIcon(t === type)}
|
||||||
|
text={t}
|
||||||
|
onClick={() => {
|
||||||
|
setCurrentQueryParameters(
|
||||||
|
deepSet(currentQueryParameters, `${i}.type`, t),
|
||||||
|
);
|
||||||
|
}}
|
||||||
|
/>
|
||||||
|
))}
|
||||||
|
</Menu>
|
||||||
|
}
|
||||||
|
>
|
||||||
|
<Button text={type} rightIcon={IconNames.CARET_DOWN} />
|
||||||
|
</Popover2>
|
||||||
|
{oneOf(type, 'BIGINT', 'DOUBLE', 'FLOAT') ? (
|
||||||
|
<FancyNumericInput
|
||||||
|
value={Number(value)}
|
||||||
|
onValueChange={onValueChange}
|
||||||
|
fill
|
||||||
|
arbitraryPrecision={type !== 'BIGINT'}
|
||||||
|
/>
|
||||||
|
) : (
|
||||||
|
<InputGroup
|
||||||
|
value={String(value)}
|
||||||
|
onChange={(e: any) => onValueChange(e.target.value)}
|
||||||
|
placeholder={type === 'TIMESTAMP' ? '2022-01-01 00:00:00' : 'Parameter value'}
|
||||||
|
fill
|
||||||
|
/>
|
||||||
|
)}
|
||||||
|
<Button
|
||||||
|
icon={IconNames.TRASH}
|
||||||
|
onClick={() => {
|
||||||
|
setCurrentQueryParameters(without(currentQueryParameters, queryParameter));
|
||||||
|
}}
|
||||||
|
/>
|
||||||
|
</ControlGroup>
|
||||||
|
</FormGroup>
|
||||||
|
);
|
||||||
|
})}
|
||||||
|
<Button
|
||||||
|
icon={IconNames.PLUS}
|
||||||
|
text="Add parameter"
|
||||||
|
intent={currentQueryParameters.length ? undefined : Intent.PRIMARY}
|
||||||
|
onClick={() => {
|
||||||
|
setCurrentQueryParameters(
|
||||||
|
currentQueryParameters.concat({ type: 'VARCHAR', value: '' }),
|
||||||
|
);
|
||||||
|
}}
|
||||||
|
/>
|
||||||
|
</div>
|
||||||
|
<div className={Classes.DIALOG_FOOTER}>
|
||||||
|
<div className={Classes.DIALOG_FOOTER_ACTIONS}>
|
||||||
|
<Button text="Close" onClick={onClose} />
|
||||||
|
<Button text="Save" intent={Intent.PRIMARY} onClick={onSave} />
|
||||||
|
</div>
|
||||||
|
</div>
|
||||||
|
</Dialog>
|
||||||
|
);
|
||||||
|
});
|
|
@ -57,6 +57,7 @@ import {
|
||||||
} from '../../../druid-models';
|
} from '../../../druid-models';
|
||||||
import { deepGet, deepSet, pluralIfNeeded, tickIcon } from '../../../utils';
|
import { deepGet, deepSet, pluralIfNeeded, tickIcon } from '../../../utils';
|
||||||
import { MaxTasksButton } from '../max-tasks-button/max-tasks-button';
|
import { MaxTasksButton } from '../max-tasks-button/max-tasks-button';
|
||||||
|
import { QueryParametersDialog } from '../query-parameters-dialog/query-parameters-dialog';
|
||||||
|
|
||||||
import './run-panel.scss';
|
import './run-panel.scss';
|
||||||
|
|
||||||
|
@ -97,6 +98,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
|
||||||
const { query, onQueryChange, onRun, moreMenu, running, 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 [editParametersDialogOpen, setEditParametersDialogOpen] = useState(false);
|
||||||
const [customTimezoneDialogOpen, setCustomTimezoneDialogOpen] = useState(false);
|
const [customTimezoneDialogOpen, setCustomTimezoneDialogOpen] = useState(false);
|
||||||
const [indexSpecDialogSpec, setIndexSpecDialogSpec] = useState<IndexSpec | undefined>();
|
const [indexSpecDialogSpec, setIndexSpecDialogSpec] = useState<IndexSpec | undefined>();
|
||||||
|
|
||||||
|
@ -104,6 +106,7 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
|
||||||
const ingestMode = query.isIngestQuery();
|
const ingestMode = query.isIngestQuery();
|
||||||
const queryContext = query.queryContext;
|
const queryContext = query.queryContext;
|
||||||
const numContextKeys = Object.keys(queryContext).length;
|
const numContextKeys = Object.keys(queryContext).length;
|
||||||
|
const queryParameters = query.queryParameters;
|
||||||
|
|
||||||
const maxParseExceptions = getMaxParseExceptions(queryContext);
|
const maxParseExceptions = getMaxParseExceptions(queryContext);
|
||||||
const finalizeAggregations = getFinalizeAggregations(queryContext);
|
const finalizeAggregations = getFinalizeAggregations(queryContext);
|
||||||
|
@ -238,6 +241,12 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
|
||||||
onClick={() => setEditContextDialogOpen(true)}
|
onClick={() => setEditContextDialogOpen(true)}
|
||||||
label={pluralIfNeeded(numContextKeys, 'key')}
|
label={pluralIfNeeded(numContextKeys, 'key')}
|
||||||
/>
|
/>
|
||||||
|
<MenuItem
|
||||||
|
icon={IconNames.HELP}
|
||||||
|
text="Define parameters"
|
||||||
|
onClick={() => setEditParametersDialogOpen(true)}
|
||||||
|
label={queryParameters ? pluralIfNeeded(queryParameters.length, 'parameter') : ''}
|
||||||
|
/>
|
||||||
{effectiveEngine !== 'native' && (
|
{effectiveEngine !== 'native' && (
|
||||||
<MenuItem
|
<MenuItem
|
||||||
icon={IconNames.GLOBE_NETWORK}
|
icon={IconNames.GLOBE_NETWORK}
|
||||||
|
@ -455,6 +464,15 @@ export const RunPanel = React.memo(function RunPanel(props: RunPanelProps) {
|
||||||
}}
|
}}
|
||||||
/>
|
/>
|
||||||
)}
|
)}
|
||||||
|
{editParametersDialogOpen && (
|
||||||
|
<QueryParametersDialog
|
||||||
|
queryParameters={queryParameters}
|
||||||
|
onQueryParametersChange={p => onQueryChange(query.changeQueryParameters(p))}
|
||||||
|
onClose={() => {
|
||||||
|
setEditParametersDialogOpen(false);
|
||||||
|
}}
|
||||||
|
/>
|
||||||
|
)}
|
||||||
{customTimezoneDialogOpen && (
|
{customTimezoneDialogOpen && (
|
||||||
<StringInputDialog
|
<StringInputDialog
|
||||||
title="Custom timezone"
|
title="Custom timezone"
|
||||||
|
|
Loading…
Reference in New Issue