Web console: add ability to issue auxiliary queries to speed up data views (#16952)

* Add ability to issue auxiliary queries

* readonly supervisor

* return

* update snapshot

* fix classes
This commit is contained in:
Vadim Ogievetsky 2024-08-27 13:38:30 -07:00 committed by GitHub
parent 0caf383102
commit 21dcf804eb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
19 changed files with 387 additions and 240 deletions

View File

@ -33,7 +33,7 @@
}
}
.#{$bp-ns}-text-muted .#{$bp-ns}-popover2-target {
.#{$bp-ns}-text-muted .#{$bp-ns}-popover-target {
margin-top: 0;
}
@ -48,7 +48,7 @@
}
}
.#{$bp-ns}-popover2-content {
.#{$bp-ns}-popover-content {
.code-block {
white-space: pre;
overflow: auto;

View File

@ -21,7 +21,7 @@
.formatted-input {
position: relative;
& > .#{$bp-ns}-popover2-target {
& > .#{$bp-ns}-popover-target {
position: absolute;
width: 0;
right: 0;

View File

@ -111,7 +111,7 @@
width: 100%;
}
.#{$bp-ns}-popover2-target {
.#{$bp-ns}-popover-target {
width: 100%;
}
}

View File

@ -24,7 +24,7 @@
overflow: hidden;
text-overflow: ellipsis;
&.#{$ns}-popover2-target {
&.#{$ns}-popover-target {
display: block; // extra nesting for stronger CSS selectors
}
}

View File

@ -381,6 +381,14 @@ export function findMap<T, Q>(
return filterMap(xs, f)[0];
}
export function changeByIndex<T>(
xs: readonly T[],
i: number,
f: (x: T, i: number) => T | undefined,
): T[] {
return filterMap(xs, (x, j) => (j === i ? f(x, i) : x));
}
export function compact<T>(xs: (T | undefined | false | null | '')[]): T[] {
return xs.filter(Boolean) as T[];
}

View File

@ -24,14 +24,12 @@ export * from './druid-lookup';
export * from './druid-query';
export * from './formatter';
export * from './general';
export * from './intermediate-query-state';
export * from './local-storage-backed-visibility';
export * from './local-storage-keys';
export * from './null-mode-detection';
export * from './object-change';
export * from './query-action';
export * from './query-manager';
export * from './query-state';
export * from './sanitizers';
export * from './sql';
export * from './table-helpers';

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
export * from './intermediate-query-state';
export * from './query-manager';
export * from './query-state';
export * from './result-with-auxiliary-work';

View File

@ -20,9 +20,11 @@ import type { Canceler, CancelToken } from 'axios';
import axios from 'axios';
import debounce from 'lodash.debounce';
import { wait } from './general';
import { wait } from '../general';
import { IntermediateQueryState } from './intermediate-query-state';
import { QueryState } from './query-state';
import { ResultWithAuxiliaryWork } from './result-with-auxiliary-work';
export interface QueryManagerOptions<Q, R, I = never, E extends Error = Error> {
initState?: QueryState<R, E, I>;
@ -30,12 +32,12 @@ export interface QueryManagerOptions<Q, R, I = never, E extends Error = Error> {
query: Q,
cancelToken: CancelToken,
setIntermediateQuery: (intermediateQuery: any) => void,
) => Promise<R | IntermediateQueryState<I>>;
) => Promise<R | IntermediateQueryState<I> | ResultWithAuxiliaryWork<R>>;
backgroundStatusCheck?: (
state: I,
query: Q,
cancelToken: CancelToken,
) => Promise<R | IntermediateQueryState<I>>;
) => Promise<R | IntermediateQueryState<I> | ResultWithAuxiliaryWork<R>>;
onStateChange?: (queryResolve: QueryState<R, E, I>) => void;
debounceIdle?: number;
debounceLoading?: number;
@ -55,13 +57,13 @@ export class QueryManager<Q, R, I = never, E extends Error = Error> {
query: Q,
cancelToken: CancelToken,
setIntermediateQuery: (intermediateQuery: any) => void,
) => Promise<R | IntermediateQueryState<I>>;
) => Promise<R | IntermediateQueryState<I> | ResultWithAuxiliaryWork<R>>;
private readonly backgroundStatusCheck?: (
state: I,
query: Q,
cancelToken: CancelToken,
) => Promise<R | IntermediateQueryState<I>>;
) => Promise<R | IntermediateQueryState<I> | ResultWithAuxiliaryWork<R>>;
private readonly onStateChange?: (queryResolve: QueryState<R, E, I>) => void;
private readonly backgroundStatusCheckInitDelay: number;
@ -120,7 +122,7 @@ export class QueryManager<Q, R, I = never, E extends Error = Error> {
});
const query = this.lastQuery;
let data: R | IntermediateQueryState<I>;
let data: R | IntermediateQueryState<I> | ResultWithAuxiliaryWork<R>;
try {
data = await this.processQuery(query, cancelToken, (intermediateQuery: any) => {
this.lastIntermediateQuery = intermediateQuery;
@ -147,6 +149,7 @@ export class QueryManager<Q, R, I = never, E extends Error = Error> {
);
}
cancelToken.throwIfRequested();
if (this.currentQueryId !== myQueryId) return;
this.setState(
new QueryState<R, E, I>({
@ -166,6 +169,7 @@ export class QueryManager<Q, R, I = never, E extends Error = Error> {
if (delay) {
await wait(delay);
cancelToken.throwIfRequested();
if (this.currentQueryId !== myQueryId) return;
}
data = await this.backgroundStatusCheck(data.state, query, cancelToken);
@ -189,12 +193,54 @@ export class QueryManager<Q, R, I = never, E extends Error = Error> {
backgroundChecks++;
}
if (this.currentQueryId !== myQueryId) return;
if (data instanceof ResultWithAuxiliaryWork && !data.auxiliaryQueries.length) {
data = data.result;
}
const lastData = this.state.getSomeData();
if (data instanceof ResultWithAuxiliaryWork) {
const auxiliaryQueries = data.auxiliaryQueries;
const numAuxiliaryQueries = auxiliaryQueries.length;
data = data.result;
this.setState(
new QueryState<R, E>({
data,
auxiliaryLoading: true,
lastData,
}),
);
try {
for (let i = 0; i < numAuxiliaryQueries; i++) {
cancelToken.throwIfRequested();
if (this.currentQueryId !== myQueryId) return;
data = await auxiliaryQueries[i](data, cancelToken);
if (this.currentQueryId !== myQueryId) return;
if (i < numAuxiliaryQueries - 1) {
// Update data in intermediate state
this.setState(
new QueryState<R, E>({
data,
auxiliaryLoading: true,
lastData,
}),
);
}
}
} catch {}
}
if (this.currentQueryId !== myQueryId) return;
this.currentRunCancelFn = undefined;
this.setState(
new QueryState<R, E>({
data,
lastData: this.state.getSomeData(),
lastData,
}),
);
}

View File

@ -20,6 +20,7 @@ export type QueryStateState = 'init' | 'loading' | 'data' | 'error';
export interface QueryStateOptions<T, E extends Error = Error, I = never> {
loading?: boolean;
auxiliaryLoading?: boolean;
intermediate?: I;
intermediateError?: Error;
error?: E;
@ -37,20 +38,19 @@ export class QueryState<T, E extends Error = Error, I = never> {
public error?: E;
public data?: T;
public lastData?: T;
public auxiliaryLoading?: boolean;
constructor(opts: QueryStateOptions<T, E, I>) {
const hasData = typeof opts.data !== 'undefined';
if (typeof opts.error !== 'undefined') {
if (hasData) {
throw new Error('can not have both error and data');
} else {
this.state = 'error';
this.error = opts.error;
}
if (hasData) throw new Error('can not have both error and data');
this.state = 'error';
this.error = opts.error;
} else {
if (hasData) {
this.state = 'data';
this.data = opts.data;
this.auxiliaryLoading = opts.auxiliaryLoading;
} else if (opts.loading) {
this.state = 'loading';
this.intermediate = opts.intermediate;
@ -92,4 +92,8 @@ export class QueryState<T, E extends Error = Error, I = never> {
getSomeData(): T | undefined {
return this.data || this.lastData;
}
isAuxiliaryLoading(): boolean {
return Boolean(this.auxiliaryLoading);
}
}

View File

@ -0,0 +1,30 @@
/*
* 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 { CancelToken } from 'axios';
export type AuxiliaryQueryFn<R> = (result: R, cancelToken: CancelToken) => Promise<R>;
export class ResultWithAuxiliaryWork<R> {
public readonly result: R;
public readonly auxiliaryQueries: AuxiliaryQueryFn<R>[];
constructor(result: R, auxiliaryQueries: AuxiliaryQueryFn<R>[]) {
this.result = result;
this.auxiliaryQueries = auxiliaryQueries;
}
}

View File

@ -55,7 +55,7 @@ import { formatCompactionInfo, zeroCompactionStatus } from '../../druid-models';
import type { Capabilities, CapabilitiesMode } from '../../helpers';
import { STANDARD_TABLE_PAGE_SIZE, STANDARD_TABLE_PAGE_SIZE_OPTIONS } from '../../react-table';
import { Api, AppToaster } from '../../singletons';
import type { NumberLike } from '../../utils';
import type { AuxiliaryQueryFn, NumberLike } from '../../utils';
import {
assemble,
compact,
@ -77,6 +77,7 @@ import {
queryDruidSql,
QueryManager,
QueryState,
ResultWithAuxiliaryWork,
twoLines,
} from '../../utils';
import type { BasicAction } from '../../utils/basic-action';
@ -262,8 +263,7 @@ function countRunningTasks(runningTasks: Record<string, number> | undefined): nu
return sum(Object.values(runningTasks));
}
function formatRunningTasks(runningTasks: Record<string, number> | undefined): string {
if (!runningTasks) return 'n/a';
function formatRunningTasks(runningTasks: Record<string, number>): string {
const runningTaskEntries = Object.entries(runningTasks);
if (!runningTaskEntries.length) return 'No running tasks';
return moveToEnd(
@ -472,141 +472,180 @@ GROUP BY 1, 2`;
throw new Error(`must have SQL or coordinator access`);
}
let runningTasksByDatasource: Record<string, Record<string, number>> = {};
if (visibleColumns.shown('Running tasks')) {
try {
if (capabilities.hasSql()) {
const runningTasks = await queryDruidSql<RunningTaskRow>({
query: DatasourcesView.RUNNING_TASK_SQL,
});
const auxiliaryQueries: AuxiliaryQueryFn<DatasourcesAndDefaultRules>[] = [];
runningTasksByDatasource = groupByAsMap(
runningTasks,
x => x.datasource,
xs =>
groupByAsMap(
xs,
x => normalizeTaskType(x.type),
ys => sum(ys, y => y.num_running_tasks),
),
);
} else if (capabilities.hasOverlordAccess()) {
const taskList = (await Api.instance.get(`/druid/indexer/v1/tasks?state=running`))
.data;
runningTasksByDatasource = groupByAsMap(
taskList,
(t: any) => t.dataSource,
xs =>
groupByAsMap(
xs,
x => normalizeTaskType(x.type),
ys => ys.length,
),
);
} else {
throw new Error(`must have SQL or overlord access`);
}
} catch (e) {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get running task counts',
if (visibleColumns.shown('Running tasks')) {
if (capabilities.hasSql()) {
auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => {
try {
const runningTasks = await queryDruidSql<RunningTaskRow>(
{
query: DatasourcesView.RUNNING_TASK_SQL,
},
cancelToken,
);
const runningTasksByDatasource = groupByAsMap(
runningTasks,
x => x.datasource,
xs =>
groupByAsMap(
xs,
x => normalizeTaskType(x.type),
ys => sum(ys, y => y.num_running_tasks),
),
);
return {
...datasourcesAndDefaultRules,
datasources: datasourcesAndDefaultRules.datasources.map(ds => ({
...ds,
runningTasks: runningTasksByDatasource[ds.datasource] || {},
})),
};
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get running task counts',
});
return datasourcesAndDefaultRules;
}
});
}
if (capabilities.hasOverlordAccess()) {
auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => {
try {
const runningTasks = await queryDruidSql<RunningTaskRow>(
{
query: DatasourcesView.RUNNING_TASK_SQL,
},
cancelToken,
);
const runningTasksByDatasource = groupByAsMap(
runningTasks,
x => x.datasource,
xs =>
groupByAsMap(
xs,
x => normalizeTaskType(x.type),
ys => sum(ys, y => y.num_running_tasks),
),
);
return {
...datasourcesAndDefaultRules,
datasources: datasourcesAndDefaultRules.datasources.map(ds => ({
...ds,
runningTasks: runningTasksByDatasource[ds.datasource] || {},
})),
};
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get running task counts',
});
return datasourcesAndDefaultRules;
}
});
}
}
if (!capabilities.hasCoordinatorAccess()) {
return {
datasources: datasources.map(ds => ({ ...ds, rules: [] })),
defaultRules: [],
};
}
const seen = countBy(datasources, x => x.datasource);
let unused: string[] = [];
if (showUnused) {
try {
unused = (
await Api.instance.get<string[]>(
'/druid/coordinator/v1/metadata/datasources?includeUnused',
)
).data.filter(d => !seen[d]);
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get the list of unused datasources',
});
if (capabilities.hasCoordinatorAccess()) {
// Unused
const seen = countBy(datasources, x => x.datasource);
if (showUnused) {
try {
unused = (
await Api.instance.get<string[]>(
'/druid/coordinator/v1/metadata/datasources?includeUnused',
)
).data.filter(d => !seen[d]);
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get the list of unused datasources',
});
}
}
}
let rules: Record<string, Rule[]> = {};
try {
rules = (await Api.instance.get<Record<string, Rule[]>>('/druid/coordinator/v1/rules'))
.data;
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get load rules',
// Rules
auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => {
try {
const rules: Record<string, Rule[]> = (
await Api.instance.get<Record<string, Rule[]>>('/druid/coordinator/v1/rules', {
cancelToken,
})
).data;
return {
datasources: datasourcesAndDefaultRules.datasources.map(ds => ({
...ds,
rules: rules[ds.datasource] || [],
})),
defaultRules: rules[DEFAULT_RULES_KEY],
};
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get load rules',
});
return datasourcesAndDefaultRules;
}
});
// Compaction
auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => {
try {
const compactionConfigsResp = await Api.instance.get<{
compactionConfigs: CompactionConfig[];
}>('/druid/coordinator/v1/config/compaction', { cancelToken });
const compactionConfigs = lookupBy(
compactionConfigsResp.data.compactionConfigs || [],
c => c.dataSource,
);
const compactionStatusesResp = await Api.instance.get<{
latestStatus: CompactionStatus[];
}>('/druid/coordinator/v1/compaction/status', { cancelToken });
const compactionStatuses = lookupBy(
compactionStatusesResp.data.latestStatus || [],
c => c.dataSource,
);
return {
...datasourcesAndDefaultRules,
datasources: datasourcesAndDefaultRules.datasources.map(ds => ({
...ds,
compaction: {
config: compactionConfigs[ds.datasource],
status: compactionStatuses[ds.datasource],
},
})),
};
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get compaction information',
});
return datasourcesAndDefaultRules;
}
});
}
let compactionConfigs: Record<string, CompactionConfig> | undefined;
try {
const compactionConfigsResp = await Api.instance.get<{
compactionConfigs: CompactionConfig[];
}>('/druid/coordinator/v1/config/compaction');
compactionConfigs = lookupBy(
compactionConfigsResp.data.compactionConfigs || [],
c => c.dataSource,
);
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get compaction configs',
});
}
let compactionStatuses: Record<string, CompactionStatus> | undefined;
if (compactionConfigs) {
// Don't bother getting the statuses if we can not even get the configs
try {
const compactionStatusesResp = await Api.instance.get<{
latestStatus: CompactionStatus[];
}>('/druid/coordinator/v1/compaction/status');
compactionStatuses = lookupBy(
compactionStatusesResp.data.latestStatus || [],
c => c.dataSource,
);
} catch {
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message: 'Could not get compaction statuses',
});
}
}
return {
datasources: datasources.concat(unused.map(makeUnusedDatasource)).map(ds => {
return {
...ds,
runningTasks: runningTasksByDatasource[ds.datasource] || {},
rules: rules[ds.datasource],
compaction:
compactionConfigs && compactionStatuses
? {
config: compactionConfigs[ds.datasource],
status: compactionStatuses[ds.datasource],
}
: undefined,
};
}),
defaultRules: rules[DEFAULT_RULES_KEY],
};
return new ResultWithAuxiliaryWork(
{
datasources: datasources.concat(unused.map(makeUnusedDatasource)),
},
auxiliaryQueries,
);
},
onStateChange: datasourcesAndDefaultRulesState => {
this.setState({
@ -1271,15 +1310,19 @@ GROUP BY 1, 2`;
accessor: d => countRunningTasks(d.runningTasks),
filterable: false,
width: 200,
Cell: ({ original }) => (
<TableClickableCell
onClick={() => goToTasks(original.datasource)}
hoverIcon={IconNames.ARROW_TOP_RIGHT}
title="Go to tasks"
>
{formatRunningTasks(original.runningTasks)}
</TableClickableCell>
),
Cell: ({ original }) => {
const { runningTasks } = original;
if (!runningTasks) return;
return (
<TableClickableCell
onClick={() => goToTasks(original.datasource)}
hoverIcon={IconNames.ARROW_TOP_RIGHT}
title="Go to tasks"
>
{formatRunningTasks(runningTasks)}
</TableClickableCell>
);
},
},
{
Header: twoLines('Segment rows', 'minimum / average / maximum'),
@ -1451,6 +1494,7 @@ GROUP BY 1, 2`;
width: 180,
Cell: ({ original }) => {
const { datasource, compaction } = original as Datasource;
if (!compaction) return;
return (
<TableClickableCell
disabled={!compaction}
@ -1465,7 +1509,7 @@ GROUP BY 1, 2`;
}}
hoverIcon={IconNames.EDIT}
>
{compaction ? formatCompactionInfo(compaction) : 'Could not get compaction info'}
{formatCompactionInfo(compaction)}
</TableClickableCell>
);
},
@ -1485,9 +1529,7 @@ GROUP BY 1, 2`;
className: 'padded',
Cell: ({ original }) => {
const { compaction } = original as Datasource;
if (!compaction) {
return 'Could not get compaction info';
}
if (!compaction) return;
const { status } = compaction;
if (!status || zeroCompactionStatus(status)) {
@ -1543,9 +1585,7 @@ GROUP BY 1, 2`;
className: 'padded',
Cell: ({ original }) => {
const { compaction } = original as Datasource;
if (!compaction) {
return 'Could not get compaction info';
}
if (!compaction) return;
const { status } = compaction;
if (!status) {
@ -1569,6 +1609,8 @@ GROUP BY 1, 2`;
width: 200,
Cell: ({ original }) => {
const { datasource, rules } = original as Datasource;
if (!rules) return;
return (
<TableClickableCell
disabled={!defaultRules}
@ -1577,17 +1619,17 @@ GROUP BY 1, 2`;
this.setState({
retentionDialogOpenOn: {
datasource,
rules: rules || [],
rules,
},
});
}}
hoverIcon={IconNames.EDIT}
>
{rules?.length
{rules.length
? DatasourcesView.formatRules(rules)
: defaultRules
? `Cluster default: ${DatasourcesView.formatRules(defaultRules)}`
: 'Could not get default rules'}
: ''}
</TableClickableCell>
);
},

View File

@ -111,7 +111,7 @@
width: 100%;
}
.#{$bp-ns}-popover2-target {
.#{$bp-ns}-popover-target {
width: 100%;
}

View File

@ -23,7 +23,7 @@
flex: 1 1 auto;
}
.more-button.#{$ns}-popover2-target {
.more-button.#{$ns}-popover-target {
flex: 0;
}
}

View File

@ -152,7 +152,7 @@
width: 100%;
}
.#{$bp-ns}-popover2-target {
.#{$bp-ns}-popover-target {
width: 100%;
}

View File

@ -82,14 +82,14 @@ exports[`SupervisorsView matches snapshot 1`] = `
"label": "status API",
"text": "Aggregate lag",
},
{
"label": "status API",
"text": "Recent errors",
},
{
"label": "stats API",
"text": "Stats",
},
{
"label": "status API",
"text": "Recent errors",
},
]
}
onChange={[Function]}

View File

@ -61,9 +61,10 @@ import {
sqlQueryCustomTableFilter,
} from '../../react-table';
import { Api, AppToaster } from '../../singletons';
import type { TableState } from '../../utils';
import type { AuxiliaryQueryFn, TableState } from '../../utils';
import {
assemble,
changeByIndex,
checkedCircleIcon,
deepGet,
filterMap,
@ -73,6 +74,7 @@ import {
formatRate,
getDruidErrorMessage,
hasPopoverOpen,
isNumberLike,
LocalStorageBackedVisibility,
LocalStorageKeys,
nonEmptyArray,
@ -81,6 +83,7 @@ import {
queryDruidSql,
QueryManager,
QueryState,
ResultWithAuxiliaryWork,
sortedToOrderByClause,
twoLines,
} from '../../utils';
@ -96,8 +99,8 @@ const SUPERVISOR_TABLE_COLUMNS: TableColumnSelectorColumn[] = [
'Configured tasks',
{ text: 'Running tasks', label: 'status API' },
{ text: 'Aggregate lag', label: 'status API' },
{ text: 'Recent errors', label: 'status API' },
{ text: 'Stats', label: 'stats API' },
{ text: 'Recent errors', label: 'status API' },
];
const ROW_STATS_KEYS: RowStatsKey[] = ['1m', '5m', '15m'];
@ -118,14 +121,14 @@ interface SupervisorQuery extends TableState {
}
interface SupervisorQueryResultRow {
supervisor_id: string;
type: string;
source: string;
detailed_state: string;
spec?: IngestionSpec;
suspended: boolean;
status?: SupervisorStatus;
stats?: any;
readonly supervisor_id: string;
readonly type: string;
readonly source: string;
readonly detailed_state: string;
readonly spec?: IngestionSpec;
readonly suspended: boolean;
readonly status?: SupervisorStatus;
readonly stats?: any;
}
export interface SupervisorsViewProps {
@ -253,19 +256,18 @@ export class SupervisorsView extends React.PureComponent<
page ? `OFFSET ${page * pageSize}` : undefined,
).join('\n');
setIntermediateQuery(sqlQuery);
supervisors = await queryDruidSql<SupervisorQueryResultRow>(
{
query: sqlQuery,
},
cancelToken,
);
for (const supervisor of supervisors) {
supervisors = (
await queryDruidSql<SupervisorQueryResultRow>(
{
query: sqlQuery,
},
cancelToken,
)
).map(supervisor => {
const spec: any = supervisor.spec;
if (typeof spec === 'string') {
supervisor.spec = JSONBig.parse(spec);
}
}
if (typeof spec !== 'string') return supervisor;
return { ...supervisor, spec: JSONBig.parse(spec) };
});
} else if (capabilities.hasOverlordAccess()) {
const supervisorList = (
await Api.instance.get('/druid/indexer/v1/supervisor?full', { cancelToken })
@ -302,54 +304,48 @@ export class SupervisorsView extends React.PureComponent<
throw new Error(`must have SQL or overlord access`);
}
const auxiliaryQueries: AuxiliaryQueryFn<SupervisorQueryResultRow[]>[] = [];
if (capabilities.hasOverlordAccess()) {
let showIssue = (message: string) => {
showIssue = () => {}; // Only show once
AppToaster.show({
icon: IconNames.ERROR,
intent: Intent.DANGER,
message,
});
};
if (visibleColumns.shown('Running tasks', 'Aggregate lag', 'Recent errors')) {
try {
for (const supervisor of supervisors) {
cancelToken.throwIfRequested();
supervisor.status = (
await Api.instance.get(
`/druid/indexer/v1/supervisor/${Api.encodePath(
supervisor.supervisor_id,
)}/status`,
{ cancelToken, timeout: STATUS_API_TIMEOUT },
)
).data;
}
} catch (e) {
showIssue('Could not get status');
}
auxiliaryQueries.push(
...supervisors.map(
(supervisor, i): AuxiliaryQueryFn<SupervisorQueryResultRow[]> =>
async (rows, cancelToken) => {
const status = (
await Api.instance.get(
`/druid/indexer/v1/supervisor/${Api.encodePath(
supervisor.supervisor_id,
)}/status`,
{ cancelToken, timeout: STATUS_API_TIMEOUT },
)
).data;
return changeByIndex(rows, i, row => ({ ...row, status }));
},
),
);
}
if (visibleColumns.shown('Stats')) {
try {
for (const supervisor of supervisors) {
cancelToken.throwIfRequested();
supervisor.stats = (
await Api.instance.get(
`/druid/indexer/v1/supervisor/${Api.encodePath(
supervisor.supervisor_id,
)}/stats`,
{ cancelToken, timeout: STATS_API_TIMEOUT },
)
).data;
}
} catch (e) {
showIssue('Could not get stats');
}
auxiliaryQueries.push(
...supervisors.map(
(supervisor, i): AuxiliaryQueryFn<SupervisorQueryResultRow[]> =>
async (rows, cancelToken) => {
const stats = (
await Api.instance.get(
`/druid/indexer/v1/supervisor/${Api.encodePath(
supervisor.supervisor_id,
)}/stats`,
{ cancelToken, timeout: STATS_API_TIMEOUT },
)
).data;
return changeByIndex(rows, i, row => ({ ...row, stats }));
},
),
);
}
}
return supervisors;
return new ResultWithAuxiliaryWork(supervisors, auxiliaryQueries);
},
onStateChange: supervisorsState => {
this.setState({
@ -790,7 +786,7 @@ export class SupervisorsView extends React.PureComponent<
);
}
} else {
label = 'n/a';
label = '';
}
return (
<TableClickableCell
@ -812,7 +808,7 @@ export class SupervisorsView extends React.PureComponent<
sortable: false,
className: 'padded',
show: visibleColumns.shown('Aggregate lag'),
Cell: ({ value }) => formatInteger(value),
Cell: ({ value }) => (isNumberLike(value) ? formatInteger(value) : null),
},
{
Header: twoLines(
@ -899,13 +895,14 @@ export class SupervisorsView extends React.PureComponent<
sortable: false,
show: visibleColumns.shown('Recent errors'),
Cell: ({ value, original }) => {
if (!value) return null;
return (
<TableClickableCell
onClick={() => this.onSupervisorDetail(original)}
hoverIcon={IconNames.SEARCH_TEMPLATE}
title="See errors"
>
{pluralIfNeeded(value?.length, 'error')}
{pluralIfNeeded(value.length, 'error')}
</TableClickableCell>
);
},

View File

@ -59,7 +59,7 @@
animation: druid-glow 1s infinite alternate;
}
.#{$bp-ns}-popover2-target {
.#{$bp-ns}-popover-target {
width: 188px;
display: inline-block;
cursor: pointer;

View File

@ -111,7 +111,7 @@
width: 100%;
}
.#{$bp-ns}-popover2-target {
.#{$bp-ns}-popover-target {
width: 100%;
}