From 2831d79871407ffbc8e19b13bc9830559af7e35c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 26 Nov 2024 22:44:05 -0800 Subject: [PATCH 01/56] update kafka dependency version to 3.9.0 (#17513) * update kafka dependency version to 3.9.0 * update licenses.yaml --- licenses.yaml | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/licenses.yaml b/licenses.yaml index 32227ce03316..1f05bb5d1d7f 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -3165,7 +3165,7 @@ libraries: --- name: Apache Kafka -version: 3.6.1 +version: 3.9.0 license_category: binary module: extensions/druid-kafka-indexing-service license_name: Apache License version 2.0 @@ -3174,7 +3174,7 @@ libraries: notices: - kafka-clients: | Apache Kafka - Copyright 2023 The Apache Software Foundation. + Copyright 2024 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (https://www.apache.org/). diff --git a/pom.xml b/pom.xml index e5dbde3ac51e..ac9b91d23d0b 100644 --- a/pom.xml +++ b/pom.xml @@ -75,7 +75,7 @@ UTF-8 0.9.0.M2 5.5.0 - 3.6.1 + 3.9.0 2.4.0 From 5333c53d710f0238b5314b7508d6a7877233a33b Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 27 Nov 2024 13:26:10 +0530 Subject: [PATCH 02/56] Support non time order in MSQ compaction (#17318) This patch supports sorting segments by non-time columns (added in #16849) to MSQ compaction. Specifically, if `forceSegmentSortByTime` is set in the data schema, either via the user-supplied compaction config or in the inferred schema, the following steps are taken: - Skip adding `__time` explicitly as the first column to the dimension schema since it already comes as part of the schema - Ensure column mappings propagate `__time` in the order specified by the schema - Set `forceSegmentSortByTime` in the MSQ context. --- .../msq/indexing/MSQCompactionRunner.java | 62 ++++-- .../msq/indexing/MSQCompactionRunnerTest.java | 205 ++++++++++++------ 2 files changed, 176 insertions(+), 91 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index d05ab12ea3fe..2d8a510fd98b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -348,7 +348,10 @@ private static Integer getRowsPerSegment(CompactionTask compactionTask) private static RowSignature getRowSignature(DataSchema dataSchema) { RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG); + if (dataSchema.getDimensionsSpec().isForceSegmentSortByTime() == true) { + // If sort not forced by time, __time appears as part of dimensions in DimensionsSpec + rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG); + } if (!isQueryGranularityEmptyOrNone(dataSchema)) { // A virtual column for query granularity would have been added. Add corresponding column type. rowSignatureBuilder.add(TIME_VIRTUAL_COLUMN, ColumnType.LONG); @@ -398,25 +401,31 @@ private static List getAggregateDimensions( private static ColumnMappings getColumnMappings(DataSchema dataSchema) { - List columnMappings = dataSchema.getDimensionsSpec() - .getDimensions() - .stream() - .map(dim -> new ColumnMapping( - dim.getName(), dim.getName())) - .collect(Collectors.toList()); + List columnMappings = new ArrayList<>(); + // For scan queries, a virtual column is created from __time if a custom query granularity is provided. For + // group-by queries, as insert needs __time, it will always be one of the dimensions. Since dimensions in groupby + // aren't allowed to have time column as the output name, we map time dimension to TIME_VIRTUAL_COLUMN in + // dimensions, and map it back to the time column here. + String timeColumn = (isGroupBy(dataSchema) || !isQueryGranularityEmptyOrNone(dataSchema)) + ? TIME_VIRTUAL_COLUMN + : ColumnHolder.TIME_COLUMN_NAME; + ColumnMapping timeColumnMapping = new ColumnMapping(timeColumn, ColumnHolder.TIME_COLUMN_NAME); + if (dataSchema.getDimensionsSpec().isForceSegmentSortByTime()) { + // When not sorted by time, the __time column is missing from dimensionsSpec + columnMappings.add(timeColumnMapping); + } + columnMappings.addAll( + dataSchema.getDimensionsSpec() + .getDimensions() + .stream() + .map(dim -> dim.getName().equals(ColumnHolder.TIME_COLUMN_NAME) + ? timeColumnMapping + : new ColumnMapping(dim.getName(), dim.getName())) + .collect(Collectors.toList()) + ); columnMappings.addAll(Arrays.stream(dataSchema.getAggregators()) .map(agg -> new ColumnMapping(agg.getName(), agg.getName())) - .collect( - Collectors.toList())); - if (isGroupBy(dataSchema) || !isQueryGranularityEmptyOrNone(dataSchema)) { - // For scan queries, a virtual column is created from __time if a custom query granularity is provided. For - // group-by queries, as insert needs __time, it will always be one of the dimensions. Since dimensions in groupby - // aren't allowed to have time column as the output name, we map time dimension to TIME_VIRTUAL_COLUMN in - // dimensions, and map it back to the time column here. - columnMappings.add(new ColumnMapping(TIME_VIRTUAL_COLUMN, ColumnHolder.TIME_COLUMN_NAME)); - } else { - columnMappings.add(new ColumnMapping(ColumnHolder.TIME_COLUMN_NAME, ColumnHolder.TIME_COLUMN_NAME)); - } + .collect(Collectors.toList())); return new ColumnMappings(columnMappings); } @@ -431,6 +440,19 @@ private static List getOrderBySpec(PartitionsSpec partitionSp return Collections.emptyList(); } + private static Map buildQueryContext( + Map taskContext, + DataSchema dataSchema + ) + { + if (dataSchema.getDimensionsSpec().isForceSegmentSortByTime()) { + return taskContext; + } + Map queryContext = new HashMap<>(taskContext); + queryContext.put(MultiStageQueryContext.CTX_FORCE_TIME_SORT, false); + return queryContext; + } + private static Query buildScanQuery( CompactionTask compactionTask, Interval interval, @@ -447,7 +469,7 @@ private static Query buildScanQuery( .columnTypes(rowSignature.getColumnTypes()) .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) .filters(dataSchema.getTransformSpec().getFilter()) - .context(compactionTask.getContext()); + .context(buildQueryContext(compactionTask.getContext(), dataSchema)); if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { List orderByColumnSpecs = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()); @@ -599,7 +621,7 @@ private Query buildGroupByQuery( .setDimensions(getAggregateDimensions(dataSchema, inputColToVirtualCol)) .setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators())) .setPostAggregatorSpecs(postAggregators) - .setContext(compactionTask.getContext()) + .setContext(buildQueryContext(compactionTask.getContext(), dataSchema)) .setInterval(interval); if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 0a54f8550a93..a05ccd499d0c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -60,7 +60,9 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CompressionFactory; @@ -72,14 +74,13 @@ import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; -import org.hamcrest.MatcherAssert; -import org.hamcrest.Matchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -103,10 +104,14 @@ public class MSQCompactionRunnerTest private static final StringDimensionSchema STRING_DIMENSION = new StringDimensionSchema("string_dim", null, false); private static final StringDimensionSchema MV_STRING_DIMENSION = new StringDimensionSchema("mv_string_dim", null, null); private static final LongDimensionSchema LONG_DIMENSION = new LongDimensionSchema("long_dim"); + private static final NestedDataColumnSchema NESTED_DIMENSION = new NestedDataColumnSchema("nested_dim", 4); + private static final AutoTypeColumnSchema AUTO_DIMENSION = new AutoTypeColumnSchema("auto_dim", null); private static final List DIMENSIONS = ImmutableList.of( STRING_DIMENSION, LONG_DIMENSION, - MV_STRING_DIMENSION + MV_STRING_DIMENSION, + NESTED_DIMENSION, + AUTO_DIMENSION ); private static final Map INTERVAL_DATASCHEMAS = ImmutableMap.of( COMPACTION_INTERVAL, @@ -336,7 +341,7 @@ public void testRunCompactionTasksWithEmptyTaskListFails() throws Exception } @Test - public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingException + public void testCompactionConfigWithoutMetricsSpecProducesCorrectSpec() throws JsonProcessingException { DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); @@ -357,7 +362,7 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce .withGranularity( new UniformGranularitySpec( SEGMENT_GRANULARITY.getDefaultGranularity(), - null, + QUERY_GRANULARITY.getDefaultGranularity(), false, Collections.singletonList(COMPACTION_INTERVAL) ) @@ -375,37 +380,37 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); - Assert.assertEquals( - new MSQTuningConfig( - 1, - MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, - MAX_ROWS_PER_SEGMENT, - null, - createIndexSpec() - ), - actualMSQSpec.getTuningConfig() - ); - Assert.assertEquals( - new DataSourceMSQDestination( - DATA_SOURCE, - SEGMENT_GRANULARITY.getDefaultGranularity(), - null, - Collections.singletonList(COMPACTION_INTERVAL), - DIMENSIONS.stream().collect(Collectors.toMap(DimensionSchema::getName, Function.identity())), - null - ), - actualMSQSpec.getDestination() - ); + Assert.assertEquals(getExpectedTuningConfig(), actualMSQSpec.getTuningConfig()); + Assert.assertEquals(getExpectedDestination(), actualMSQSpec.getDestination()); Assert.assertTrue(actualMSQSpec.getQuery() instanceof ScanQuery); ScanQuery scanQuery = (ScanQuery) actualMSQSpec.getQuery(); + List expectedColumns = new ArrayList<>(); + List expectedColumnTypes = new ArrayList<>(); + // Add __time since this is a time-ordered query which doesn't have __time explicitly defined in dimensionsSpec + expectedColumns.add(ColumnHolder.TIME_COLUMN_NAME); + expectedColumnTypes.add(ColumnType.LONG); + + // Add TIME_VIRTUAL_COLUMN since a query granularity is specified + expectedColumns.add(MSQCompactionRunner.TIME_VIRTUAL_COLUMN); + expectedColumnTypes.add(ColumnType.LONG); + + expectedColumns.addAll(DIMENSIONS.stream().map(DimensionSchema::getName).collect(Collectors.toList())); + expectedColumnTypes.addAll(DIMENSIONS.stream().map(DimensionSchema::getColumnType).collect(Collectors.toList())); + + Assert.assertEquals(expectedColumns, scanQuery.getColumns()); + Assert.assertEquals(expectedColumnTypes, scanQuery.getColumnTypes()); + Assert.assertEquals(dimFilter, scanQuery.getFilter()); Assert.assertEquals( JSON_MAPPER.writeValueAsString(SEGMENT_GRANULARITY.toString()), msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY) ); - Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); + Assert.assertEquals( + JSON_MAPPER.writeValueAsString(QUERY_GRANULARITY.toString()), + msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY) + ); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); Assert.assertEquals( PARTITION_DIMENSIONS.stream().map(OrderBy::ascending).collect(Collectors.toList()), @@ -414,7 +419,60 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce } @Test - public void testMSQControllerTaskSpecWithAggregatorsIsValid() throws JsonProcessingException + public void testCompactionConfigWithSortOnNonTimeDimensionsProducesCorrectSpec() throws JsonProcessingException + { + List nonTimeSortedDimensions = ImmutableList.of( + STRING_DIMENSION, + new LongDimensionSchema(ColumnHolder.TIME_COLUMN_NAME), + LONG_DIMENSION + ); + CompactionTask taskCreatedWithTransformSpec = createCompactionTask( + new DynamicPartitionsSpec(TARGET_ROWS_PER_SEGMENT, null), + null, + Collections.emptyMap(), + null, + null + ); + + // Set forceSegmentSortByTime=false to enable non-time order + DimensionsSpec dimensionsSpec = DimensionsSpec.builder() + .setDimensions(nonTimeSortedDimensions) + .setForceSegmentSortByTime(false) + .build(); + DataSchema dataSchema = + DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) + .withDimensions(dimensionsSpec) + .withGranularity( + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + false, + Collections.singletonList(COMPACTION_INTERVAL) + ) + ) + .build(); + + List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( + taskCreatedWithTransformSpec, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); + + MSQSpec actualMSQSpec = Iterables.getOnlyElement(msqControllerTasks).getQuerySpec(); + + Assert.assertTrue(actualMSQSpec.getQuery() instanceof ScanQuery); + ScanQuery scanQuery = (ScanQuery) actualMSQSpec.getQuery(); + + // Dimensions should already list __time and the order should remain intact + Assert.assertEquals( + nonTimeSortedDimensions.stream().map(DimensionSchema::getName).collect(Collectors.toList()), + scanQuery.getColumns() + ); + } + + @Test + public void testCompactionConfigWithMetricsSpecProducesCorrectSpec() throws JsonProcessingException { DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); @@ -444,7 +502,6 @@ public void testMSQControllerTaskSpecWithAggregatorsIsValid() throws JsonProcess multiValuedDimensions ); - List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) @@ -454,27 +511,8 @@ public void testMSQControllerTaskSpecWithAggregatorsIsValid() throws JsonProcess MSQSpec actualMSQSpec = msqControllerTask.getQuerySpec(); - Assert.assertEquals( - new MSQTuningConfig( - 1, - MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, - MAX_ROWS_PER_SEGMENT, - null, - createIndexSpec() - ), - actualMSQSpec.getTuningConfig() - ); - Assert.assertEquals( - new DataSourceMSQDestination( - DATA_SOURCE, - SEGMENT_GRANULARITY.getDefaultGranularity(), - null, - Collections.singletonList(COMPACTION_INTERVAL), - DIMENSIONS.stream().collect(Collectors.toMap(DimensionSchema::getName, Function.identity())), - null - ), - actualMSQSpec.getDestination() - ); + Assert.assertEquals(getExpectedTuningConfig(), actualMSQSpec.getTuningConfig()); + Assert.assertEquals(getExpectedDestination(), actualMSQSpec.getDestination()); Assert.assertTrue(actualMSQSpec.getQuery() instanceof GroupByQuery); GroupByQuery groupByQuery = (GroupByQuery) actualMSQSpec.getQuery(); @@ -490,30 +528,32 @@ public void testMSQControllerTaskSpecWithAggregatorsIsValid() throws JsonProcess ); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); - - // Since only MV_STRING_DIMENSION is indicated to be MVD by the CombinedSchema, conversion to array should happen - // only for that column. - List expectedDimensionSpec = DIMENSIONS.stream() - .filter(dim -> !MV_STRING_DIMENSION.getName() - .equals(dim.getName())) - .map(dim -> new DefaultDimensionSpec( - dim.getName(), - dim.getName(), - dim.getColumnType() - )) - .collect( - Collectors.toList()); + List expectedDimensionSpec = new ArrayList<>(); expectedDimensionSpec.add( - new DefaultDimensionSpec(MSQCompactionRunner.TIME_VIRTUAL_COLUMN, - MSQCompactionRunner.TIME_VIRTUAL_COLUMN, - ColumnType.LONG) + new DefaultDimensionSpec( + MSQCompactionRunner.TIME_VIRTUAL_COLUMN, + MSQCompactionRunner.TIME_VIRTUAL_COLUMN, + ColumnType.LONG + ) ); String mvToArrayStringDim = MSQCompactionRunner.ARRAY_VIRTUAL_COLUMN_PREFIX + MV_STRING_DIMENSION.getName(); - expectedDimensionSpec.add(new DefaultDimensionSpec(mvToArrayStringDim, mvToArrayStringDim, ColumnType.STRING_ARRAY)); - MatcherAssert.assertThat( - expectedDimensionSpec, - Matchers.containsInAnyOrder(groupByQuery.getDimensions().toArray(new DimensionSpec[0])) - ); + // Since only MV_STRING_DIMENSION is indicated to be MVD by the CombinedSchema, conversion to array should happen + // only for that column. + expectedDimensionSpec.addAll(DIMENSIONS.stream() + .map(dim -> + MV_STRING_DIMENSION.getName().equals(dim.getName()) + ? new DefaultDimensionSpec( + mvToArrayStringDim, + mvToArrayStringDim, + ColumnType.STRING_ARRAY + ) + : new DefaultDimensionSpec( + dim.getName(), + dim.getName(), + dim.getColumnType() + )) + .collect(Collectors.toList())); + Assert.assertEquals(expectedDimensionSpec, groupByQuery.getDimensions()); } private CompactionTask createCompactionTask( @@ -580,4 +620,27 @@ private static IndexSpec createIndexSpec() .withLongEncoding(CompressionFactory.LongEncodingStrategy.LONGS) .build(); } + + private static DataSourceMSQDestination getExpectedDestination() + { + return new DataSourceMSQDestination( + DATA_SOURCE, + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + Collections.singletonList(COMPACTION_INTERVAL), + DIMENSIONS.stream().collect(Collectors.toMap(DimensionSchema::getName, Function.identity())), + null + ); + } + + private static MSQTuningConfig getExpectedTuningConfig() + { + return new MSQTuningConfig( + 1, + MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, + MAX_ROWS_PER_SEGMENT, + null, + createIndexSpec() + ); + } } From 80d6763e390303952db7b4f6a3246d1eccb3e4d5 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 27 Nov 2024 00:01:00 -0800 Subject: [PATCH 03/56] ServerSelector: Synchronize getAllServers(). (#17499) This method was missing some required synchronization. This patch also adds GuardedBy annotations to historicalServers and realtimeServers, which would have caught it. --- .../druid/client/selector/ServerSelector.java | 31 +++++++++++-------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java index 30b259b66194..99d883c3301a 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java @@ -19,6 +19,7 @@ package org.apache.druid.client.selector; +import com.google.errorprone.annotations.concurrent.GuardedBy; import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; import org.apache.druid.client.DataSegmentInterner; import org.apache.druid.query.Query; @@ -40,9 +41,10 @@ */ public class ServerSelector implements Overshadowable { - + @GuardedBy("this") private final Int2ObjectRBTreeMap> historicalServers; + @GuardedBy("this") private final Int2ObjectRBTreeMap> realtimeServers; private final TierSelectorStrategy strategy; @@ -145,18 +147,21 @@ public List getCandidates(final int numCandidates) public List getAllServers() { - List servers = new ArrayList<>(); - historicalServers.values() - .stream() - .flatMap(Collection::stream) - .map(server -> server.getServer().getMetadata()) - .forEach(servers::add); - - realtimeServers.values() - .stream() - .flatMap(Collection::stream) - .map(server -> server.getServer().getMetadata()) - .forEach(servers::add); + final List servers = new ArrayList<>(); + + synchronized (this) { + historicalServers.values() + .stream() + .flatMap(Collection::stream) + .map(server -> server.getServer().getMetadata()) + .forEach(servers::add); + + realtimeServers.values() + .stream() + .flatMap(Collection::stream) + .map(server -> server.getServer().getMetadata()) + .forEach(servers::add); + } return servers; } From 1b9a6dde9f45b77de1cc5a956e1038619a86dcc3 Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Wed, 27 Nov 2024 17:16:30 +0530 Subject: [PATCH 04/56] Fix compilation error for MSQCompactionRunnerTest (#17516) --- .../msq/indexing/MSQCompactionRunnerTest.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index a05ccd499d0c..ae59fe1383dc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -149,7 +149,6 @@ public void testMultipleDisjointCompactionIntervalsAreInvalid() null, Collections.emptyMap(), null, - null, null ); CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( @@ -171,7 +170,6 @@ public void testHashedPartitionsSpecIsInvalid() null, Collections.emptyMap(), null, - null, null ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -185,7 +183,6 @@ public void testStringDimensionInRangePartitionsSpecIsValid() null, Collections.emptyMap(), null, - null, null ); Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -200,7 +197,6 @@ public void testLongDimensionInRangePartitionsSpecIsInvalid() null, Collections.emptyMap(), null, - null, null ); @@ -223,7 +219,6 @@ public void testMultiValuedDimensionInRangePartitionsSpecIsInvalid() null, Collections.emptyMap(), null, - null, null ); @@ -245,7 +240,6 @@ public void testMaxTotalRowsIsInvalid() null, Collections.emptyMap(), null, - null, null ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -259,7 +253,6 @@ public void testDynamicPartitionsSpecIsValid() null, Collections.emptyMap(), null, - null, null ); Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -273,7 +266,6 @@ public void testQueryGranularityAllIsValid() null, Collections.emptyMap(), new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), - null, null ); Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -287,7 +279,6 @@ public void testRollupFalseWithMetricsSpecIsInValid() null, Collections.emptyMap(), new ClientCompactionTaskGranularitySpec(null, null, false), - null, AGGREGATORS.toArray(new AggregatorFactory[0]) ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -301,7 +292,6 @@ public void testRollupTrueWithoutMetricsSpecIsInvalid() null, Collections.emptyMap(), new ClientCompactionTaskGranularitySpec(null, null, true), - null, null ); Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, INTERVAL_DATASCHEMAS).isValid()); @@ -318,7 +308,6 @@ public void testMSQEngineWithUnsupportedMetricsSpecIsInvalid() null, Collections.emptyMap(), new ClientCompactionTaskGranularitySpec(null, null, true), - null, new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} ); CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( @@ -335,7 +324,7 @@ public void testMSQEngineWithUnsupportedMetricsSpecIsInvalid() @Test public void testRunCompactionTasksWithEmptyTaskListFails() throws Exception { - CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap(), null, null, null); + CompactionTask compactionTask = createCompactionTask(null, null, Collections.emptyMap(), null, null); TaskStatus taskStatus = MSQ_COMPACTION_RUNNER.runCompactionTasks(compactionTask, Collections.emptyMap(), null); Assert.assertTrue(taskStatus.isFailure()); } @@ -350,7 +339,6 @@ public void testCompactionConfigWithoutMetricsSpecProducesCorrectSpec() throws J dimFilter, Collections.emptyMap(), null, - null, null ); @@ -481,7 +469,6 @@ public void testCompactionConfigWithMetricsSpecProducesCorrectSpec() throws Json dimFilter, Collections.emptyMap(), null, - null, null ); @@ -561,7 +548,6 @@ private CompactionTask createCompactionTask( @Nullable DimFilter dimFilter, Map contextParams, @Nullable ClientCompactionTaskGranularitySpec granularitySpec, - @Nullable List dimensionSchemas, @Nullable AggregatorFactory[] metricsSpec ) { @@ -585,7 +571,7 @@ private CompactionTask createCompactionTask( )) .transformSpec(transformSpec) .granularitySpec(granularitySpec) - .dimensionsSpec(new DimensionsSpec(dimensionSchemas)) + .dimensionsSpec(new DimensionsSpec(null)) .metricsSpec(metricsSpec) .compactionRunner(MSQ_COMPACTION_RUNNER) .context(context); From 09432c099beb38fb0e79a497c589ab83da87ce81 Mon Sep 17 00:00:00 2001 From: Vadim Ogievetsky Date: Wed, 27 Nov 2024 09:37:01 -0800 Subject: [PATCH 05/56] Web console: refactor and improve the segment timeline (#17508) * refactor and improve the segment timeline * us consistent state * type cleanup * add shpitz * better bubble --- licenses.yaml | 94 +- web-console/e2e-tests/tutorial-batch.spec.ts | 2 +- web-console/jest.common.config.js | 2 + web-console/lib/keywords.ts | 8 + web-console/package-lock.json | 205 ++-- web-console/package.json | 5 +- web-console/script/create-sql-docs.mjs | 6 + web-console/script/licenses | 1 + web-console/src/bootstrap/json-parser.tsx | 2 +- .../__snapshots__/auto-form.spec.tsx.snap | 1 - .../src/components/auto-form/auto-form.tsx | 1 - .../cell-filter-menu/cell-filter-menu.tsx | 4 +- .../fancy-numeric-input.tsx | 2 +- web-console/src/components/index.ts | 1 + .../portal-bubble/portal-bubble.scss | 86 ++ .../portal-bubble/portal-bubble.tsx | 93 ++ .../record-table-pane/record-table-pane.tsx | 2 +- .../__snapshots__/rule-editor.spec.tsx.snap | 10 +- .../components/rule-editor/rule-editor.scss | 4 + .../components/rule-editor/rule-editor.tsx | 6 +- .../__snapshots__/bar-unit.spec.tsx.snap | 13 - .../segment-timeline.spec.tsx.snap | 422 ++++++--- .../components/segment-timeline/bar-group.tsx | 75 -- .../segment-timeline/chart-axis.tsx | 14 +- .../src/components/segment-timeline/common.ts | 87 ++ .../segment-bar-chart-render.scss | 169 ++++ .../segment-bar-chart-render.tsx | 793 ++++++++++++++++ ...r-unit.spec.tsx => segment-bar-chart.scss} | 18 +- .../segment-timeline/segment-bar-chart.tsx | 162 ++++ .../segment-timeline/segment-timeline.scss | 32 +- .../segment-timeline.spec.tsx | 52 +- .../segment-timeline/segment-timeline.tsx | 887 ++++++------------ .../segment-timeline/stacked-bar-chart.tsx | 174 ---- .../supervisor-history-panel.tsx | 13 +- .../table-clickable-cell.tsx | 5 +- .../table-filterable-cell.tsx | 2 +- web-console/src/console-application.tsx | 18 +- .../compaction-history-dialog.tsx | 7 +- .../coordinator-dynamic-config-dialog.tsx | 7 +- .../datasource-columns-table.tsx | 2 +- .../datasource-preview-pane.tsx | 4 +- .../lookup-values-table.tsx | 2 +- .../overlord-dynamic-config-dialog.tsx | 7 +- .../retention-dialog.spec.tsx.snap | 4 +- .../retention-dialog/retention-dialog.tsx | 17 +- .../segments-preview-pane.tsx | 4 +- .../dialogs/status-dialog/status-dialog.tsx | 3 +- .../supervisor-reset-offsets-dialog.tsx | 11 +- .../datasource/datasource.ts} | 55 +- .../src/druid-models/execution/execution.ts | 2 +- .../external-config/external-config.ts | 4 +- web-console/src/druid-models/index.ts | 3 + .../ingest-query-pattern.spec.ts | 2 +- .../ingest-query-pattern.ts | 2 +- .../load-rule}/load-rule.ts | 3 +- .../src/druid-models/segment/segment.ts | 50 + web-console/src/druid-models/task/task.ts | 2 +- .../workbench-query/workbench-query.spec.ts | 2 +- .../workbench-query/workbench-query.ts | 4 +- web-console/src/entry.scss | 1 + web-console/src/entry.tsx | 2 +- web-console/src/helpers/capabilities.ts | 4 + web-console/src/helpers/execution/general.ts | 2 +- .../helpers/execution/sql-task-execution.ts | 2 +- web-console/src/helpers/spec-conversion.ts | 2 +- web-console/src/hooks/use-clock.ts | 18 +- .../src/react-table/react-table-utils.spec.ts | 4 + .../src/react-table/react-table-utils.ts | 50 +- .../date-floor-shift-ceil-utc.spec.ts | 169 ++++ .../date-floor-shift-ceil.spec.ts | 181 ++++ .../date-floor-shift-ceil.ts | 296 ++++++ web-console/src/utils/date.spec.ts | 9 - web-console/src/utils/date.ts | 11 +- web-console/src/utils/download.ts | 2 +- web-console/src/utils/druid-query.spec.ts | 2 +- web-console/src/utils/druid-query.ts | 8 +- .../src/utils/duration/duration.spec.ts | 505 ++++++++++ web-console/src/utils/duration/duration.ts | 388 ++++++++ web-console/src/utils/general.spec.ts | 14 +- web-console/src/utils/general.tsx | 28 +- web-console/src/utils/index.tsx | 2 + web-console/src/utils/query-action.ts | 2 +- .../src/utils/query-manager/query-manager.ts | 2 +- web-console/src/utils/sampler.ts | 2 +- web-console/src/utils/sql.spec.ts | 2 +- web-console/src/utils/sql.ts | 4 +- .../explore-view/models => utils}/stage.ts | 14 + web-console/src/utils/table-helpers.ts | 26 +- web-console/src/utils/types.ts | 2 +- web-console/src/utils/values-query.spec.tsx | 2 +- web-console/src/utils/values-query.tsx | 4 +- .../datasources-view.spec.tsx.snap | 1 + .../datasources-view/datasources-view.scss | 9 +- .../datasources-view/datasources-view.tsx | 145 ++- .../column-picker-menu/column-picker-menu.tsx | 2 +- .../column-picker/column-picker.tsx | 2 +- .../components/control-pane/control-pane.tsx | 4 +- .../control-pane/expression-menu.tsx | 4 +- .../components/control-pane/measure-menu.tsx | 4 +- .../droppable-container.tsx | 2 +- .../contains-filter-control.tsx | 4 +- .../filter-pane/filter-menu/filter-menu.tsx | 4 +- .../number-range-filter-control.tsx | 2 +- .../regexp-filter-control.tsx | 4 +- .../time-interval-filter-control.tsx | 2 +- .../time-relative-filter-control.tsx | 2 +- .../values-filter-control.tsx | 4 +- .../components/filter-pane/filter-pane.tsx | 6 +- .../generic-output-table.tsx | 4 +- .../components/module-pane/module-pane.tsx | 5 +- .../components/preview-pane/preview-pane.tsx | 2 +- .../column-dialog/column-dialog.tsx | 4 +- .../measure-dialog/measure-dialog.tsx | 2 +- .../nested-column-dialog.tsx | 4 +- .../resource-pane/resource-pane.tsx | 2 +- .../components/source-pane/source-pane.tsx | 6 +- .../components/sql-input/sql-input.tsx | 4 +- .../src/views/explore-view/drag-helper.ts | 2 +- .../src/views/explore-view/explore-state.ts | 4 +- .../src/views/explore-view/explore-view.tsx | 4 +- .../explore-view/models/expression-meta.ts | 4 +- .../src/views/explore-view/models/index.ts | 1 - .../explore-view/models/measure-pattern.ts | 4 +- .../src/views/explore-view/models/measure.ts | 4 +- .../views/explore-view/models/query-source.ts | 4 +- .../module-repository/module-repository.ts | 5 +- .../explore-view/modules/bar-chart-module.tsx | 2 +- .../modules/grouping-table-module.tsx | 4 +- .../modules/multi-axis-chart-module.tsx | 16 +- .../explore-view/modules/pie-chart-module.tsx | 2 +- .../modules/record-table-module.tsx | 2 +- .../modules/time-chart-module.tsx | 19 +- .../query-macros/aggregate.spec.ts | 2 +- .../explore-view/query-macros/aggregate.ts | 2 +- .../query-macros/max-data-time.ts | 4 +- .../views/explore-view/utils/duration.spec.ts | 39 - .../src/views/explore-view/utils/duration.ts | 46 - .../utils/filter-pattern-helpers.ts | 10 +- .../src/views/explore-view/utils/general.ts | 4 +- .../utils/get-auto-granularity.ts | 87 +- .../src/views/explore-view/utils/index.ts | 2 - .../explore-view/utils/max-time-for-table.ts | 2 +- .../explore-view/utils/snap-to-granularity.ts | 57 -- .../explore-view/utils/table-query.spec.ts | 4 +- .../views/explore-view/utils/table-query.ts | 11 +- .../utils/time-manipulation.spec.ts | 2 +- .../explore-view/utils/time-manipulation.ts | 2 +- .../datasources-card/datasources-card.tsx | 10 +- .../home-view/segments-card/segments-card.tsx | 7 +- .../home-view/services-card/services-card.tsx | 13 +- .../supervisors-card/supervisors-card.tsx | 15 +- .../views/home-view/tasks-card/tasks-card.tsx | 17 +- .../views/load-data-view/load-data-view.tsx | 4 +- .../src/views/lookups-view/lookups-view.tsx | 14 +- .../__snapshots__/segments-view.spec.tsx.snap | 57 +- .../views/segments-view/segments-view.scss | 2 +- .../src/views/segments-view/segments-view.tsx | 521 +++++----- .../src/views/services-view/services-view.tsx | 48 +- .../column-actions/column-actions.tsx | 4 +- .../column-editor/column-editor.tsx | 4 +- .../expression-editor-dialog.tsx | 2 +- .../ingestion-progress-dialog.tsx | 2 +- .../schema-step/column-list/column-list.tsx | 2 +- .../expression-entry/expression-entry.tsx | 4 +- .../preview-table/preview-table.tsx | 6 +- .../rollup-analysis-pane.tsx | 4 +- .../schema-step/schema-step.tsx | 6 +- .../sql-data-loader-view.tsx | 2 +- .../supervisors-view.spec.tsx.snap | 12 +- .../supervisors-view/supervisors-view.tsx | 119 ++- .../src/views/tasks-view/tasks-view.tsx | 37 +- .../complex-menu-items.spec.tsx | 2 +- .../complex-menu-items/complex-menu-items.tsx | 4 +- .../number-menu-items.spec.tsx | 2 +- .../number-menu-items/number-menu-items.tsx | 4 +- .../string-menu-items.spec.tsx | 2 +- .../string-menu-items/string-menu-items.tsx | 4 +- .../time-menu-items/time-menu-items.spec.tsx | 2 +- .../time-menu-items/time-menu-items.tsx | 74 +- .../column-tree/column-tree.spec.tsx | 2 +- .../column-tree/column-tree.tsx | 4 +- .../connect-external-data-dialog.tsx | 2 +- .../current-dart-panel/current-dart-panel.tsx | 12 +- .../execution-details-pane.tsx | 2 +- .../execution-stages-pane.tsx | 2 +- .../flexible-query-input.tsx | 2 +- .../ingest-success-pane.tsx | 2 +- .../input-format-step/input-format-step.tsx | 4 +- .../input-source-step/input-source-step.tsx | 2 +- .../query-parameters-dialog.tsx | 4 +- .../workbench-view/query-tab/query-tab.tsx | 15 +- .../recent-query-task-panel.tsx | 20 +- .../result-table-pane/result-table-pane.tsx | 4 +- .../workbench-view/run-panel/run-panel.tsx | 71 +- .../time-floor-menu-item.tsx | 4 +- .../timezone-menu-items.spec.tsx.snap | 563 +++++++++++ .../timezone-menu-items.spec.tsx} | 48 +- .../timezone-menu-items.tsx | 151 +++ .../views/workbench-view/work-state-store.ts | 22 +- .../views/workbench-view/workbench-view.tsx | 4 +- 200 files changed, 5580 insertions(+), 2379 deletions(-) create mode 100644 web-console/src/components/portal-bubble/portal-bubble.scss create mode 100644 web-console/src/components/portal-bubble/portal-bubble.tsx delete mode 100644 web-console/src/components/segment-timeline/__snapshots__/bar-unit.spec.tsx.snap delete mode 100644 web-console/src/components/segment-timeline/bar-group.tsx create mode 100644 web-console/src/components/segment-timeline/common.ts create mode 100644 web-console/src/components/segment-timeline/segment-bar-chart-render.scss create mode 100644 web-console/src/components/segment-timeline/segment-bar-chart-render.tsx rename web-console/src/components/segment-timeline/{bar-unit.spec.tsx => segment-bar-chart.scss} (68%) create mode 100644 web-console/src/components/segment-timeline/segment-bar-chart.tsx delete mode 100644 web-console/src/components/segment-timeline/stacked-bar-chart.tsx rename web-console/src/{components/segment-timeline/stacked-bar-chart.scss => druid-models/datasource/datasource.ts} (64%) rename web-console/src/{utils => druid-models/load-rule}/load-rule.ts (97%) create mode 100644 web-console/src/druid-models/segment/segment.ts create mode 100755 web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts create mode 100755 web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts create mode 100755 web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts create mode 100755 web-console/src/utils/duration/duration.spec.ts create mode 100755 web-console/src/utils/duration/duration.ts rename web-console/src/{views/explore-view/models => utils}/stage.ts (80%) delete mode 100644 web-console/src/views/explore-view/utils/duration.spec.ts delete mode 100644 web-console/src/views/explore-view/utils/duration.ts delete mode 100644 web-console/src/views/explore-view/utils/snap-to-granularity.ts create mode 100644 web-console/src/views/workbench-view/timezone-menu-items/__snapshots__/timezone-menu-items.spec.tsx.snap rename web-console/src/{components/segment-timeline/bar-unit.tsx => views/workbench-view/timezone-menu-items/timezone-menu-items.spec.tsx} (58%) create mode 100644 web-console/src/views/workbench-view/timezone-menu-items/timezone-menu-items.tsx diff --git a/licenses.yaml b/licenses.yaml index 1f05bb5d1d7f..8044beb12bab 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -5125,15 +5125,6 @@ version: 5.2.5 --- -name: "@druid-toolkit/query" -license_category: binary -module: web-console -license_name: Apache License version 2.0 -copyright: Imply Data -version: 0.22.23 - ---- - name: "@emotion/cache" license_category: binary module: web-console @@ -5224,6 +5215,16 @@ license_file_path: licenses/bin/@emotion-weak-memoize.MIT --- +name: "@flatten-js/interval-tree" +license_category: binary +module: web-console +license_name: MIT License +copyright: Alex Bol +version: 1.1.3 +license_file_path: licenses/bin/@flatten-js-interval-tree.MIT + +--- + name: "@fontsource/open-sans" license_category: binary module: web-console @@ -5234,6 +5235,15 @@ license_file_path: licenses/bin/@fontsource-open-sans.OFL --- +name: "@internationalized/date" +license_category: binary +module: web-console +license_name: Apache License version 2.0 +copyright: Adobe +version: 3.5.6 + +--- + name: "@popperjs/core" license_category: binary module: web-console @@ -5244,6 +5254,15 @@ license_file_path: licenses/bin/@popperjs-core.MIT --- +name: "@swc/helpers" +license_category: binary +module: web-console +license_name: Apache License version 2.0 +copyright: 강동윤 +version: 0.5.13 + +--- + name: "@types/parse-json" license_category: binary module: web-console @@ -5404,15 +5423,6 @@ license_file_path: licenses/bin/change-case.MIT --- -name: "chronoshift" -license_category: binary -module: web-console -license_name: Apache License version 2.0 -copyright: Vadim Ogievetsky -version: 0.10.0 - ---- - name: "classnames" license_category: binary module: web-console @@ -5702,6 +5712,15 @@ license_file_path: licenses/bin/dot-case.MIT --- +name: "druid-query-toolkit" +license_category: binary +module: web-console +license_name: Apache License version 2.0 +copyright: Imply Data +version: 1.0.0 + +--- + name: "echarts" license_category: binary module: web-console @@ -5801,16 +5820,6 @@ license_file_path: licenses/bin/has-flag.MIT --- -name: "has-own-prop" -license_category: binary -module: web-console -license_name: MIT License -copyright: Sindre Sorhus -version: 2.0.0 -license_file_path: licenses/bin/has-own-prop.MIT - ---- - name: "hasown" license_category: binary module: web-console @@ -5871,15 +5880,6 @@ license_file_path: licenses/bin/iconv-lite.MIT --- -name: "immutable-class" -license_category: binary -module: web-console -license_name: Apache License version 2.0 -copyright: Vadim Ogievetsky -version: 0.11.2 - ---- - name: "import-fresh" license_category: binary module: web-console @@ -6060,26 +6060,6 @@ license_file_path: licenses/bin/mime-types.MIT --- -name: "moment-timezone" -license_category: binary -module: web-console -license_name: MIT License -copyright: Tim Wood -version: 0.5.43 -license_file_path: licenses/bin/moment-timezone.MIT - ---- - -name: "moment" -license_category: binary -module: web-console -license_name: MIT License -copyright: Iskren Ivov Chernev -version: 2.29.4 -license_file_path: licenses/bin/moment.MIT - ---- - name: "no-case" license_category: binary module: web-console diff --git a/web-console/e2e-tests/tutorial-batch.spec.ts b/web-console/e2e-tests/tutorial-batch.spec.ts index 3fa15af9d2a4..59ac0a407810 100644 --- a/web-console/e2e-tests/tutorial-batch.spec.ts +++ b/web-console/e2e-tests/tutorial-batch.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { T } from '@druid-toolkit/query'; +import { T } from 'druid-query-toolkit'; import type * as playwright from 'playwright-chromium'; import { DatasourcesOverview } from './component/datasources/overview'; diff --git a/web-console/jest.common.config.js b/web-console/jest.common.config.js index 89e3dab5852b..1ea8f55ad18b 100644 --- a/web-console/jest.common.config.js +++ b/web-console/jest.common.config.js @@ -18,6 +18,8 @@ const { createJsWithTsPreset } = require('ts-jest'); +process.env.TZ = 'UTC'; + module.exports = { testEnvironment: 'jsdom', transformIgnorePatterns: ['/node_modules/(?!(d3-.+)/)'], diff --git a/web-console/lib/keywords.ts b/web-console/lib/keywords.ts index 06d7ccdcc949..5985ae0a6aa6 100644 --- a/web-console/lib/keywords.ts +++ b/web-console/lib/keywords.ts @@ -100,15 +100,23 @@ export const SQL_EXPRESSION_PARTS = [ 'TRAILING', 'EPOCH', 'SECOND', + 'SECONDS', 'MINUTE', + 'MINUTES', 'HOUR', + 'HOURS', 'DAY', + 'DAYS', 'DOW', 'DOY', 'WEEK', + 'WEEKS', 'MONTH', + 'MONTHS', 'QUARTER', + 'QUARTERS', 'YEAR', + 'YEARS', 'TIMESTAMP', 'INTERVAL', 'CSV', diff --git a/web-console/package-lock.json b/web-console/package-lock.json index 95768117648c..27a3d463aad2 100644 --- a/web-console/package-lock.json +++ b/web-console/package-lock.json @@ -14,11 +14,11 @@ "@blueprintjs/datetime2": "^2.3.11", "@blueprintjs/icons": "^5.13.0", "@blueprintjs/select": "^5.2.5", - "@druid-toolkit/query": "^0.22.23", + "@flatten-js/interval-tree": "^1.1.3", "@fontsource/open-sans": "^5.0.30", + "@internationalized/date": "^3.5.6", "ace-builds": "~1.5.3", "axios": "^1.7.7", - "chronoshift": "^0.10.0", "classnames": "^2.2.6", "copy-to-clipboard": "^3.3.3", "d3-array": "^3.2.4", @@ -28,6 +28,7 @@ "d3-scale-chromatic": "^3.1.0", "d3-selection": "^3.0.0", "date-fns": "^2.28.0", + "druid-query-toolkit": "^1.0.0", "echarts": "^5.5.1", "file-saver": "^2.0.5", "hjson": "^3.2.2", @@ -2085,15 +2086,6 @@ "node": ">=10.0.0" } }, - "node_modules/@druid-toolkit/query": { - "version": "0.22.23", - "resolved": "https://registry.npmjs.org/@druid-toolkit/query/-/query-0.22.23.tgz", - "integrity": "sha512-yQOUAQJP63rzsTCdLcqNB8aRtsYPw8rYBfPSXc4zfAA4y/GJc9OJeHcLFRMdUtpwBtm0ueARMUlTSQcTsyV8gQ==", - "license": "Apache-2.0", - "dependencies": { - "tslib": "^2.5.2" - } - }, "node_modules/@dual-bundle/import-meta-resolve": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/@dual-bundle/import-meta-resolve/-/import-meta-resolve-4.1.0.tgz", @@ -2373,9 +2365,9 @@ } }, "node_modules/@eslint/plugin-kit": { - "version": "0.2.0", - "resolved": "https://registry.npmjs.org/@eslint/plugin-kit/-/plugin-kit-0.2.0.tgz", - "integrity": "sha512-vH9PiIMMwvhCx31Af3HiGzsVNULDbyVkHXwlemn/B0TFj/00ho3y55efXrUZTfQipxoHC5u4xq6zblww1zm1Ig==", + "version": "0.2.3", + "resolved": "https://registry.npmjs.org/@eslint/plugin-kit/-/plugin-kit-0.2.3.tgz", + "integrity": "sha512-2b/g5hRmpbb1o4GnTZax9N9m0FXzz9OV42ZzI4rDDMDuHUqigAiQCEWChBWCY4ztAGVRjoWT19v0yMmc5/L5kA==", "dev": true, "license": "Apache-2.0", "dependencies": { @@ -2385,6 +2377,12 @@ "node": "^18.18.0 || ^20.9.0 || >=21.1.0" } }, + "node_modules/@flatten-js/interval-tree": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/@flatten-js/interval-tree/-/interval-tree-1.1.3.tgz", + "integrity": "sha512-xhFWUBoHJFF77cJO1D6REjdgJEMRf2Y2Z+eKEPav8evGKcLSnj1ud5pLXQSbGuxF3VSvT1rWhMfVpXEKJLTL+A==", + "license": "MIT" + }, "node_modules/@fontsource/open-sans": { "version": "5.1.0", "resolved": "https://registry.npmjs.org/@fontsource/open-sans/-/open-sans-5.1.0.tgz", @@ -2442,6 +2440,15 @@ "url": "https://github.com/sponsors/nzakas" } }, + "node_modules/@internationalized/date": { + "version": "3.5.6", + "resolved": "https://registry.npmjs.org/@internationalized/date/-/date-3.5.6.tgz", + "integrity": "sha512-jLxQjefH9VI5P9UQuqB6qNKnvFt1Ky1TPIzHGsIlCi7sZZoMR8SdYbBGRvM0y+Jtb+ez4ieBzmiAUcpmPYpyOw==", + "license": "Apache-2.0", + "dependencies": { + "@swc/helpers": "^0.5.0" + } + }, "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", @@ -3473,6 +3480,15 @@ "url": "https://opencollective.com/eslint" } }, + "node_modules/@swc/helpers": { + "version": "0.5.13", + "resolved": "https://registry.npmjs.org/@swc/helpers/-/helpers-0.5.13.tgz", + "integrity": "sha512-UoKGxQ3r5kYI9dALKJapMmuK+1zWM/H17Z1+iwnNmzcJRnfFuevZs375TA5rW31pu4BS4NoSy1fRsexDXfWn5w==", + "license": "Apache-2.0", + "dependencies": { + "tslib": "^2.4.0" + } + }, "node_modules/@testing-library/dom": { "version": "10.4.0", "resolved": "https://registry.npmjs.org/@testing-library/dom/-/dom-10.4.0.tgz", @@ -5802,16 +5818,6 @@ "node": ">=6.0" } }, - "node_modules/chronoshift": { - "version": "0.10.0", - "resolved": "https://registry.npmjs.org/chronoshift/-/chronoshift-0.10.0.tgz", - "integrity": "sha512-dNvumPg7R6ACUOKbGo1zH6DtmTo5ut9/LNbzqaKGnpC9VdArIos8+kApHOVIZH4FCpm9M9XYh++jwlRHhc1PyA==", - "dependencies": { - "immutable-class": "^0.11.0", - "moment-timezone": "^0.5.26", - "tslib": "^2.3.1" - } - }, "node_modules/ci-info": { "version": "3.3.0", "resolved": "https://registry.npmjs.org/ci-info/-/ci-info-3.3.0.tgz", @@ -6205,10 +6211,11 @@ "dev": true }, "node_modules/cross-spawn": { - "version": "7.0.3", - "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", - "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "version": "7.0.6", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.6.tgz", + "integrity": "sha512-uV2QOWP2nWzsy2aMp8aRibhi9dlzF5Hgh5SHaB9OiTGEyDTiJJyx0uy51QXdyWbtAHNua4XJzUKca3OzKUd3vA==", "dev": true, + "license": "MIT", "dependencies": { "path-key": "^3.1.0", "shebang-command": "^2.0.0", @@ -7013,6 +7020,15 @@ "tslib": "^2.0.3" } }, + "node_modules/druid-query-toolkit": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/druid-query-toolkit/-/druid-query-toolkit-1.0.0.tgz", + "integrity": "sha512-yBQR4uDcks0lcsRSWoLQy16YQ4dx264m6i7TNQDFrACUKHlMtnw5l+4+UDZKbXbpUFLMLWCr/kLhmXzLJk50+Q==", + "license": "Apache-2.0", + "dependencies": { + "tslib": "^2.5.2" + } + }, "node_modules/duplexer": { "version": "0.1.2", "resolved": "https://registry.npmjs.org/duplexer/-/duplexer-0.1.2.tgz", @@ -8909,14 +8925,6 @@ "node": ">=4" } }, - "node_modules/has-own-prop": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/has-own-prop/-/has-own-prop-2.0.0.tgz", - "integrity": "sha512-Pq0h+hvsVm6dDEa8x82GnLSYHOzNDt7f0ddFa3FqcQlgzEiptPqL+XrOJNavjOzSYiYWIrgeVYYgGlLmnxwilQ==", - "engines": { - "node": ">=8" - } - }, "node_modules/has-property-descriptors": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/has-property-descriptors/-/has-property-descriptors-1.0.2.tgz", @@ -9213,10 +9221,11 @@ "dev": true }, "node_modules/http-proxy-middleware": { - "version": "2.0.6", - "resolved": "https://registry.npmjs.org/http-proxy-middleware/-/http-proxy-middleware-2.0.6.tgz", - "integrity": "sha512-ya/UeJ6HVBYxrgYotAZo1KvPWlgB48kUJLDePFeneHsVujFaW5WNj2NgWCAE//B1Dl02BIfYlpNgBy8Kf8Rjmw==", + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/http-proxy-middleware/-/http-proxy-middleware-2.0.7.tgz", + "integrity": "sha512-fgVY8AV7qU7z/MmXJ/rxwbrtQH4jBQ9m7kp3llF0liB7glmFeVZFBepQb32T3y8n8k2+AEYuMPCpinYW+/CuRA==", "dev": true, + "license": "MIT", "dependencies": { "@types/http-proxy": "^1.17.8", "http-proxy": "^1.18.1", @@ -9354,15 +9363,6 @@ "integrity": "sha512-0AOCmOip+xgJwEVTQj1EfiDDOkPmuyllDuTuEX+DDXUgapLAsBIfkg3sxCYyCEA8mQqZrrxPUGjcOQ2JS3WLkg==", "dev": true }, - "node_modules/immutable-class": { - "version": "0.11.2", - "resolved": "https://registry.npmjs.org/immutable-class/-/immutable-class-0.11.2.tgz", - "integrity": "sha512-CzkVPkJXzkspt6RX+ipNgtvt16+rzEBUlA3yNPLkK5/S042c9wvuyfE4F5TfMfPJ6XF86Fp+OCwu6eeAnMICuw==", - "dependencies": { - "has-own-prop": "^2.0.0", - "tslib": "^2.3.1" - } - }, "node_modules/import-fresh": { "version": "3.3.0", "resolved": "https://registry.npmjs.org/import-fresh/-/import-fresh-3.3.0.tgz", @@ -12889,25 +12889,6 @@ "mkdirp": "bin/cmd.js" } }, - "node_modules/moment": { - "version": "2.29.4", - "resolved": "https://registry.npmjs.org/moment/-/moment-2.29.4.tgz", - "integrity": "sha512-5LC9SOxjSc2HF6vO2CyuTDNivEdoz2IvyJJGj6X8DJ0eFyfszE0QiEd+iXmBvUP3WHxSjFH/vIsA0EN00cgr8w==", - "engines": { - "node": "*" - } - }, - "node_modules/moment-timezone": { - "version": "0.5.43", - "resolved": "https://registry.npmjs.org/moment-timezone/-/moment-timezone-0.5.43.tgz", - "integrity": "sha512-72j3aNyuIsDxdF1i7CEgV2FfxM1r6aaqJyLB2vwb33mXYyoyLly+F1zbWqhA3/bVIoJ4szlUoMbUnVdid32NUQ==", - "dependencies": { - "moment": "^2.29.4" - }, - "engines": { - "node": "*" - } - }, "node_modules/mrmime": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/mrmime/-/mrmime-1.0.1.tgz", @@ -19560,14 +19541,6 @@ "integrity": "sha512-dBVuXR082gk3jsFp7Rd/JI4kytwGHecnCoTtXFb7DB6CNHp4rg5k1bhg0nWdLGLnOV71lmDzGQaLMy8iPLY0pw==", "dev": true }, - "@druid-toolkit/query": { - "version": "0.22.23", - "resolved": "https://registry.npmjs.org/@druid-toolkit/query/-/query-0.22.23.tgz", - "integrity": "sha512-yQOUAQJP63rzsTCdLcqNB8aRtsYPw8rYBfPSXc4zfAA4y/GJc9OJeHcLFRMdUtpwBtm0ueARMUlTSQcTsyV8gQ==", - "requires": { - "tslib": "^2.5.2" - } - }, "@dual-bundle/import-meta-resolve": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/@dual-bundle/import-meta-resolve/-/import-meta-resolve-4.1.0.tgz", @@ -19774,14 +19747,19 @@ "dev": true }, "@eslint/plugin-kit": { - "version": "0.2.0", - "resolved": "https://registry.npmjs.org/@eslint/plugin-kit/-/plugin-kit-0.2.0.tgz", - "integrity": "sha512-vH9PiIMMwvhCx31Af3HiGzsVNULDbyVkHXwlemn/B0TFj/00ho3y55efXrUZTfQipxoHC5u4xq6zblww1zm1Ig==", + "version": "0.2.3", + "resolved": "https://registry.npmjs.org/@eslint/plugin-kit/-/plugin-kit-0.2.3.tgz", + "integrity": "sha512-2b/g5hRmpbb1o4GnTZax9N9m0FXzz9OV42ZzI4rDDMDuHUqigAiQCEWChBWCY4ztAGVRjoWT19v0yMmc5/L5kA==", "dev": true, "requires": { "levn": "^0.4.1" } }, + "@flatten-js/interval-tree": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/@flatten-js/interval-tree/-/interval-tree-1.1.3.tgz", + "integrity": "sha512-xhFWUBoHJFF77cJO1D6REjdgJEMRf2Y2Z+eKEPav8evGKcLSnj1ud5pLXQSbGuxF3VSvT1rWhMfVpXEKJLTL+A==" + }, "@fontsource/open-sans": { "version": "5.1.0", "resolved": "https://registry.npmjs.org/@fontsource/open-sans/-/open-sans-5.1.0.tgz", @@ -19815,6 +19793,14 @@ "integrity": "sha512-JBxkERygn7Bv/GbN5Rv8Ul6LVknS+5Bp6RgDC/O8gEBU/yeH5Ui5C/OlWrTb6qct7LjjfT6Re2NxB0ln0yYybA==", "dev": true }, + "@internationalized/date": { + "version": "3.5.6", + "resolved": "https://registry.npmjs.org/@internationalized/date/-/date-3.5.6.tgz", + "integrity": "sha512-jLxQjefH9VI5P9UQuqB6qNKnvFt1Ky1TPIzHGsIlCi7sZZoMR8SdYbBGRvM0y+Jtb+ez4ieBzmiAUcpmPYpyOw==", + "requires": { + "@swc/helpers": "^0.5.0" + } + }, "@istanbuljs/load-nyc-config": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", @@ -20614,6 +20600,14 @@ } } }, + "@swc/helpers": { + "version": "0.5.13", + "resolved": "https://registry.npmjs.org/@swc/helpers/-/helpers-0.5.13.tgz", + "integrity": "sha512-UoKGxQ3r5kYI9dALKJapMmuK+1zWM/H17Z1+iwnNmzcJRnfFuevZs375TA5rW31pu4BS4NoSy1fRsexDXfWn5w==", + "requires": { + "tslib": "^2.4.0" + } + }, "@testing-library/dom": { "version": "10.4.0", "resolved": "https://registry.npmjs.org/@testing-library/dom/-/dom-10.4.0.tgz", @@ -22411,16 +22405,6 @@ "integrity": "sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg==", "dev": true }, - "chronoshift": { - "version": "0.10.0", - "resolved": "https://registry.npmjs.org/chronoshift/-/chronoshift-0.10.0.tgz", - "integrity": "sha512-dNvumPg7R6ACUOKbGo1zH6DtmTo5ut9/LNbzqaKGnpC9VdArIos8+kApHOVIZH4FCpm9M9XYh++jwlRHhc1PyA==", - "requires": { - "immutable-class": "^0.11.0", - "moment-timezone": "^0.5.26", - "tslib": "^2.3.1" - } - }, "ci-info": { "version": "3.3.0", "resolved": "https://registry.npmjs.org/ci-info/-/ci-info-3.3.0.tgz", @@ -22735,9 +22719,9 @@ "dev": true }, "cross-spawn": { - "version": "7.0.3", - "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", - "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "version": "7.0.6", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.6.tgz", + "integrity": "sha512-uV2QOWP2nWzsy2aMp8aRibhi9dlzF5Hgh5SHaB9OiTGEyDTiJJyx0uy51QXdyWbtAHNua4XJzUKca3OzKUd3vA==", "dev": true, "requires": { "path-key": "^3.1.0", @@ -23276,6 +23260,14 @@ "tslib": "^2.0.3" } }, + "druid-query-toolkit": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/druid-query-toolkit/-/druid-query-toolkit-1.0.0.tgz", + "integrity": "sha512-yBQR4uDcks0lcsRSWoLQy16YQ4dx264m6i7TNQDFrACUKHlMtnw5l+4+UDZKbXbpUFLMLWCr/kLhmXzLJk50+Q==", + "requires": { + "tslib": "^2.5.2" + } + }, "duplexer": { "version": "0.1.2", "resolved": "https://registry.npmjs.org/duplexer/-/duplexer-0.1.2.tgz", @@ -24650,11 +24642,6 @@ "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=" }, - "has-own-prop": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/has-own-prop/-/has-own-prop-2.0.0.tgz", - "integrity": "sha512-Pq0h+hvsVm6dDEa8x82GnLSYHOzNDt7f0ddFa3FqcQlgzEiptPqL+XrOJNavjOzSYiYWIrgeVYYgGlLmnxwilQ==" - }, "has-property-descriptors": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/has-property-descriptors/-/has-property-descriptors-1.0.2.tgz", @@ -24884,9 +24871,9 @@ } }, "http-proxy-middleware": { - "version": "2.0.6", - "resolved": "https://registry.npmjs.org/http-proxy-middleware/-/http-proxy-middleware-2.0.6.tgz", - "integrity": "sha512-ya/UeJ6HVBYxrgYotAZo1KvPWlgB48kUJLDePFeneHsVujFaW5WNj2NgWCAE//B1Dl02BIfYlpNgBy8Kf8Rjmw==", + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/http-proxy-middleware/-/http-proxy-middleware-2.0.7.tgz", + "integrity": "sha512-fgVY8AV7qU7z/MmXJ/rxwbrtQH4jBQ9m7kp3llF0liB7glmFeVZFBepQb32T3y8n8k2+AEYuMPCpinYW+/CuRA==", "dev": true, "requires": { "@types/http-proxy": "^1.17.8", @@ -24982,15 +24969,6 @@ "integrity": "sha512-0AOCmOip+xgJwEVTQj1EfiDDOkPmuyllDuTuEX+DDXUgapLAsBIfkg3sxCYyCEA8mQqZrrxPUGjcOQ2JS3WLkg==", "dev": true }, - "immutable-class": { - "version": "0.11.2", - "resolved": "https://registry.npmjs.org/immutable-class/-/immutable-class-0.11.2.tgz", - "integrity": "sha512-CzkVPkJXzkspt6RX+ipNgtvt16+rzEBUlA3yNPLkK5/S042c9wvuyfE4F5TfMfPJ6XF86Fp+OCwu6eeAnMICuw==", - "requires": { - "has-own-prop": "^2.0.0", - "tslib": "^2.3.1" - } - }, "import-fresh": { "version": "3.3.0", "resolved": "https://registry.npmjs.org/import-fresh/-/import-fresh-3.3.0.tgz", @@ -27557,19 +27535,6 @@ "minimist": "^1.2.5" } }, - "moment": { - "version": "2.29.4", - "resolved": "https://registry.npmjs.org/moment/-/moment-2.29.4.tgz", - "integrity": "sha512-5LC9SOxjSc2HF6vO2CyuTDNivEdoz2IvyJJGj6X8DJ0eFyfszE0QiEd+iXmBvUP3WHxSjFH/vIsA0EN00cgr8w==" - }, - "moment-timezone": { - "version": "0.5.43", - "resolved": "https://registry.npmjs.org/moment-timezone/-/moment-timezone-0.5.43.tgz", - "integrity": "sha512-72j3aNyuIsDxdF1i7CEgV2FfxM1r6aaqJyLB2vwb33mXYyoyLly+F1zbWqhA3/bVIoJ4szlUoMbUnVdid32NUQ==", - "requires": { - "moment": "^2.29.4" - } - }, "mrmime": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/mrmime/-/mrmime-1.0.1.tgz", diff --git a/web-console/package.json b/web-console/package.json index 1e76234841ab..39eae99e101b 100644 --- a/web-console/package.json +++ b/web-console/package.json @@ -55,11 +55,11 @@ "@blueprintjs/datetime2": "^2.3.11", "@blueprintjs/icons": "^5.13.0", "@blueprintjs/select": "^5.2.5", - "@druid-toolkit/query": "^0.22.23", + "@flatten-js/interval-tree": "^1.1.3", "@fontsource/open-sans": "^5.0.30", + "@internationalized/date": "^3.5.6", "ace-builds": "~1.5.3", "axios": "^1.7.7", - "chronoshift": "^0.10.0", "classnames": "^2.2.6", "copy-to-clipboard": "^3.3.3", "d3-array": "^3.2.4", @@ -69,6 +69,7 @@ "d3-scale-chromatic": "^3.1.0", "d3-selection": "^3.0.0", "date-fns": "^2.28.0", + "druid-query-toolkit": "^1.0.0", "echarts": "^5.5.1", "file-saver": "^2.0.5", "hjson": "^3.2.2", diff --git a/web-console/script/create-sql-docs.mjs b/web-console/script/create-sql-docs.mjs index 62e1f44d0177..b14055341a0d 100755 --- a/web-console/script/create-sql-docs.mjs +++ b/web-console/script/create-sql-docs.mjs @@ -39,6 +39,12 @@ const initialFunctionDocs = { ), ], ], + UNNEST: [ + [ + 'arrayExpression', + convertMarkdownToHtml("Unnests ARRAY typed values. The source for UNNEST can be an array type column, or an input that's been transformed into an array, such as with helper functions like `MV_TO_ARRAY` or `ARRAY`.") + ] + ] }; function hasHtmlTags(str) { diff --git a/web-console/script/licenses b/web-console/script/licenses index 90f1420282ae..f4e67862d011 100755 --- a/web-console/script/licenses +++ b/web-console/script/licenses @@ -193,6 +193,7 @@ checker.init( if (name === 'diff-match-patch') publisher = 'Google'; if (name === 'esutils') publisher = 'Yusuke Suzuki'; // https://github.com/estools/esutils#license if (name === 'echarts') publisher = 'Apache Software Foundation'; + if (name === '@internationalized/date') publisher = 'Adobe'; } if (!publisher) { diff --git a/web-console/src/bootstrap/json-parser.tsx b/web-console/src/bootstrap/json-parser.tsx index 7e8fc0994610..d8fd232795e7 100644 --- a/web-console/src/bootstrap/json-parser.tsx +++ b/web-console/src/bootstrap/json-parser.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { QueryResult } from '@druid-toolkit/query'; +import { QueryResult } from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; export function bootstrapJsonParse() { diff --git a/web-console/src/components/auto-form/__snapshots__/auto-form.spec.tsx.snap b/web-console/src/components/auto-form/__snapshots__/auto-form.spec.tsx.snap index dc566363abee..0016d1c90323 100644 --- a/web-console/src/components/auto-form/__snapshots__/auto-form.spec.tsx.snap +++ b/web-console/src/components/auto-form/__snapshots__/auto-form.spec.tsx.snap @@ -170,7 +170,6 @@ exports[`AutoForm matches snapshot 1`] = ` > extends React.PureComponent text={showMore ? 'Show less' : 'Show more'} rightIcon={showMore ? IconNames.CHEVRON_UP : IconNames.CHEVRON_DOWN} minimal - fill onClick={() => { this.setState(({ showMore }) => ({ showMore: !showMore })); }} diff --git a/web-console/src/components/cell-filter-menu/cell-filter-menu.tsx b/web-console/src/components/cell-filter-menu/cell-filter-menu.tsx index dac3e2fc1979..5d2098c7db4d 100644 --- a/web-console/src/components/cell-filter-menu/cell-filter-menu.tsx +++ b/web-console/src/components/cell-filter-menu/cell-filter-menu.tsx @@ -18,8 +18,8 @@ import { Menu, MenuItem } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { Column, SqlExpression, SqlQuery } from '@druid-toolkit/query'; -import { C, L, SqlComparison, SqlLiteral, SqlRecord, trimString } from '@druid-toolkit/query'; +import type { Column, SqlExpression, SqlQuery } from 'druid-query-toolkit'; +import { C, L, SqlComparison, SqlLiteral, SqlRecord, trimString } from 'druid-query-toolkit'; import type { QueryAction } from '../../utils'; import { copyAndAlert, prettyPrintSql, stringifyValue } from '../../utils'; diff --git a/web-console/src/components/fancy-numeric-input/fancy-numeric-input.tsx b/web-console/src/components/fancy-numeric-input/fancy-numeric-input.tsx index cb389ff532dc..dad7c9a0617a 100644 --- a/web-console/src/components/fancy-numeric-input/fancy-numeric-input.tsx +++ b/web-console/src/components/fancy-numeric-input/fancy-numeric-input.tsx @@ -19,8 +19,8 @@ import type { InputGroupProps2, Intent } from '@blueprintjs/core'; import { Button, ButtonGroup, Classes, ControlGroup, InputGroup, Keys } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import { SqlExpression, SqlFunction, SqlLiteral, SqlMulti } from '@druid-toolkit/query'; import classNames from 'classnames'; +import { SqlExpression, SqlFunction, SqlLiteral, SqlMulti } from 'druid-query-toolkit'; import React, { useEffect, useState } from 'react'; import { clamp } from '../../utils'; diff --git a/web-console/src/components/index.ts b/web-console/src/components/index.ts index d885bf07434a..18a4c21e72c2 100644 --- a/web-console/src/components/index.ts +++ b/web-console/src/components/index.ts @@ -43,6 +43,7 @@ export * from './menu-checkbox/menu-checkbox'; export * from './more-button/more-button'; export * from './plural-pair-if-needed/plural-pair-if-needed'; export * from './popover-text/popover-text'; +export * from './portal-bubble/portal-bubble'; export * from './query-error-pane/query-error-pane'; export * from './record-table-pane/record-table-pane'; export * from './refresh-button/refresh-button'; diff --git a/web-console/src/components/portal-bubble/portal-bubble.scss b/web-console/src/components/portal-bubble/portal-bubble.scss new file mode 100644 index 000000000000..f29091150afc --- /dev/null +++ b/web-console/src/components/portal-bubble/portal-bubble.scss @@ -0,0 +1,86 @@ +/* + * 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'; + +$shpitz-size: 9px; + +.portal-bubble { + position: absolute; + @include card-like; + + .#{$bp-ns}-dark & { + background: $dark-gray1; + } + + &.up { + transform: translate(-50%, -100%); + } + + &.down { + transform: translate(-50%, 0); + } + + &.mute { + pointer-events: none; + } + + & > .shpitz { + content: ''; + position: absolute; + transform: translate(-50%, 0); + border-right: $shpitz-size solid transparent; + border-left: $shpitz-size solid transparent; + } + + &.up > .shpitz { + bottom: -$shpitz-size; + border-top: $shpitz-size solid $dark-gray1; + } + + &.down > .shpitz { + top: -$shpitz-size; + border-bottom: $shpitz-size solid $dark-gray1; + } + + & > .bubble-title-bar { + position: relative; + padding: 5px 5px 0 5px; + white-space: nowrap; + font-weight: bold; + + &.with-close { + padding-right: 26px; + + .close-button { + position: absolute; + top: 0; + right: 0; + } + } + } + + & > .bubble-content { + padding: 5px; + white-space: nowrap; + } + + .bubble-title-bar + .bubble-content { + padding-top: 0; + } +} diff --git a/web-console/src/components/portal-bubble/portal-bubble.tsx b/web-console/src/components/portal-bubble/portal-bubble.tsx new file mode 100644 index 000000000000..b8831c84bb09 --- /dev/null +++ b/web-console/src/components/portal-bubble/portal-bubble.tsx @@ -0,0 +1,93 @@ +/* + * 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 } from '@blueprintjs/core'; +import { IconNames } from '@blueprintjs/icons'; +import classNames from 'classnames'; +import type { ReactNode } from 'react'; +import { useState } from 'react'; +import { createPortal } from 'react-dom'; + +import { clamp } from '../../utils'; + +import './portal-bubble.scss'; + +const SHPITZ_SIZE = 10; + +export interface PortalBubbleOpenOn { + x: number; + y: number; + title?: string; + text: ReactNode; +} + +export interface PortalBubbleProps { + className?: string; + openOn: PortalBubbleOpenOn | undefined; + direction?: 'up' | 'down'; + onClose?(): void; + mute?: boolean; + minimal?: boolean; +} + +export const PortalBubble = function PortalBubble(props: PortalBubbleProps) { + const { className, openOn, direction = 'up', onClose, mute, minimal } = props; + const [myWidth, setMyWidth] = useState(200); + if (!openOn) return null; + + const halfMyWidth = myWidth / 2; + + const x = clamp(openOn.x, halfMyWidth, window.innerWidth - halfMyWidth); + const offset = clamp(x - openOn.x, -halfMyWidth, halfMyWidth); + + return createPortal( +
{ + if (!element) return; + setMyWidth(element.offsetWidth); + }} + style={{ + left: x, + top: openOn.y + (minimal ? 0 : direction === 'up' ? -SHPITZ_SIZE : SHPITZ_SIZE), + }} + > + {(openOn.title || onClose) && ( +
+ {openOn.title} + {onClose && ( +
+ )} +
{openOn.text}
+ {!minimal && ( +
+ )} +
, + document.body, + ); +}; diff --git a/web-console/src/components/record-table-pane/record-table-pane.tsx b/web-console/src/components/record-table-pane/record-table-pane.tsx index e44eab355602..803ee783c2cd 100644 --- a/web-console/src/components/record-table-pane/record-table-pane.tsx +++ b/web-console/src/components/record-table-pane/record-table-pane.tsx @@ -18,8 +18,8 @@ import { Button, Icon, Popover } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { Column, QueryResult } from '@druid-toolkit/query'; import classNames from 'classnames'; +import type { Column, QueryResult } from 'druid-query-toolkit'; import React, { useEffect, useState } from 'react'; import type { RowRenderProps } from 'react-table'; import ReactTable from 'react-table'; diff --git a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap index 2769c8af739e..7246fe89490f 100644 --- a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap +++ b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap @@ -70,7 +70,7 @@ exports[`RuleEditor matches snapshot no tier in rule 1`] = ` style="transform: translateY(0); transition: none;" >
- + - - -`; diff --git a/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap b/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap index ae30df246c2b..206c2be703bd 100644 --- a/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap +++ b/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap @@ -4,191 +4,309 @@ exports[`SegmentTimeline matches snapshot 1`] = `
-
-
- -
-
-
+ + + +
- - -
-
+ + +
- -
+ 1D + + + + + + + + + - -
+ + + + + 2024-11-01 → 2024-11-18 + + + +
+
`; diff --git a/web-console/src/components/segment-timeline/bar-group.tsx b/web-console/src/components/segment-timeline/bar-group.tsx deleted file mode 100644 index d0cf867e2b2d..000000000000 --- a/web-console/src/components/segment-timeline/bar-group.tsx +++ /dev/null @@ -1,75 +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 { AxisScale } from 'd3-axis'; -import React from 'react'; - -import { BarUnit } from './bar-unit'; -import type { BarUnitData, HoveredBarInfo } from './stacked-bar-chart'; - -interface BarGroupProps { - dataToRender: BarUnitData[]; - changeActiveDatasource: (dataSource: string) => void; - formatTick: (e: number) => string; - xScale: AxisScale; - yScale: AxisScale; - barWidth: number; - onHoverBar?: (e: any) => void; - offHoverBar?: () => void; - hoverOn?: HoveredBarInfo | null; -} - -export class BarGroup extends React.Component { - shouldComponentUpdate(nextProps: BarGroupProps): boolean { - return nextProps.hoverOn === this.props.hoverOn; - } - - render() { - const { dataToRender, changeActiveDatasource, xScale, yScale, onHoverBar, barWidth } = - this.props; - if (dataToRender === undefined) return null; - - return dataToRender.map((entry: BarUnitData, i: number) => { - const y0 = yScale(entry.y0 || 0) || 0; - const x = xScale(new Date(entry.x + 'T00:00:00Z')); - const y = yScale((entry.y0 || 0) + entry.y) || 0; - const height = Math.max(y0 - y, 0); - const barInfo: HoveredBarInfo = { - xCoordinate: x, - yCoordinate: y, - height, - datasource: entry.datasource, - xValue: entry.x, - yValue: entry.y, - dailySize: entry.dailySize, - }; - return ( - changeActiveDatasource(entry.datasource)} - onHover={() => onHoverBar && onHoverBar(barInfo)} - /> - ); - }); - } -} diff --git a/web-console/src/components/segment-timeline/chart-axis.tsx b/web-console/src/components/segment-timeline/chart-axis.tsx index bc333d33b773..b8ee4e9cbb11 100644 --- a/web-console/src/components/segment-timeline/chart-axis.tsx +++ b/web-console/src/components/segment-timeline/chart-axis.tsx @@ -16,22 +16,22 @@ * limitations under the License. */ +import type { Axis } from 'd3-axis'; import { select } from 'd3-selection'; -import React from 'react'; interface ChartAxisProps { - transform?: string; - scale: any; className?: string; + transform?: string; + axis: Axis; } -export const ChartAxis = React.memo(function ChartAxis(props: ChartAxisProps) { - const { transform, scale, className } = props; +export const ChartAxis = function ChartAxis(props: ChartAxisProps) { + const { transform, axis, className } = props; return ( select(node).call(scale)} + ref={node => select(node).call(axis as any)} /> ); -}); +}; diff --git a/web-console/src/components/segment-timeline/common.ts b/web-console/src/components/segment-timeline/common.ts new file mode 100644 index 000000000000..48aa5ce48d46 --- /dev/null +++ b/web-console/src/components/segment-timeline/common.ts @@ -0,0 +1,87 @@ +/* + * 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 { sum } from 'd3-array'; + +import type { Duration } from '../../utils'; +import { formatBytes, formatInteger } from '../../utils'; + +export type IntervalStat = 'segments' | 'size' | 'rows'; + +export const INTERVAL_STATS: IntervalStat[] = ['segments', 'size', 'rows']; + +export function getIntervalStatTitle(intervalStat: IntervalStat): string { + switch (intervalStat) { + case 'segments': + return 'Num. segments'; + + case 'size': + return 'Size'; + + case 'rows': + return 'Rows'; + + default: + return intervalStat; + } +} + +export function aggregateSegmentStats( + xs: readonly Record[], +): Record { + return { + segments: sum(xs, s => s.segments), + size: sum(xs, s => s.size), + rows: sum(xs, s => s.rows), + }; +} + +export function formatIntervalStat(stat: IntervalStat, n: number) { + switch (stat) { + case 'segments': + case 'rows': + return formatInteger(n); + + case 'size': + return formatBytes(n); + + default: + return ''; + } +} + +export interface IntervalRow extends Record { + start: Date; + end: Date; + datasource: string; + realtime: boolean; + originalTimeSpan: Duration; +} + +export interface TrimmedIntervalRow extends IntervalRow { + shownDays: number; + normalized: Record; +} + +export interface IntervalBar extends TrimmedIntervalRow { + offset: Record; +} + +export function formatIsoDateOnly(date: Date): string { + return date.toISOString().slice(0, 10); +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart-render.scss b/web-console/src/components/segment-timeline/segment-bar-chart-render.scss new file mode 100644 index 000000000000..762772a0f5c5 --- /dev/null +++ b/web-console/src/components/segment-timeline/segment-bar-chart-render.scss @@ -0,0 +1,169 @@ +/* + * 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'; + +.segment-bar-chart-render { + position: relative; + overflow: hidden; + + @keyframes pulseOpacity { + 0% { + opacity: 0.8; + } + 100% { + opacity: 0.95; + } + } + + svg { + position: absolute; + + .chart-axis text { + user-select: none; + } + + .hover-highlight { + fill: white; + fill-opacity: 0.1; + } + + .hovered-bar { + fill: none; + stroke: #ffffff; + stroke-width: 1.5px; + } + + .selection { + fill: transparent; + stroke: #ffffff; + stroke-width: 1px; + opacity: 0.8; + + &.done { + opacity: 1; + } + } + + .shifter { + fill: white; + fill-opacity: 0.2; + filter: blur(1px); + } + + .time-shift-indicator { + fill: white; + fill-opacity: 0.001; + cursor: grab; + + &:hover { + fill-opacity: 0.1; + } + + &.shifting { + fill-opacity: 0.2; + cursor: grabbing; + } + } + + .gridline-x { + line { + stroke-dasharray: 5, 5; + opacity: 0.5; + } + } + + .now-line { + stroke: $orange4; + stroke-dasharray: 2, 2; + opacity: 0.7; + } + + .bar-unit { + &.realtime { + animation: pulseOpacity 3s alternate infinite; + } + } + } + + .rule-tape { + position: absolute; + top: 5px; + height: 15px; + font-size: 10px; + + .rule-error { + @include pin-full(); + background-color: $red3; + color: $white; + } + + .load-rule { + position: absolute; + overflow: hidden; + padding-left: 2px; + border-left: 1px solid $dark-gray2; + border-right: 1px solid $dark-gray2; + top: 0; + height: 100%; + text-overflow: ellipsis; + + &.load { + background-color: $green1; + + &:nth-child(even) { + background-color: $green3; + } + } + + &.drop { + background-color: $dark-gray5; + + &:nth-child(even) { + background-color: $gray1; + } + } + + &.broadcast { + background-color: $indigo1; + + &:nth-child(even) { + background-color: $indigo3; + } + } + } + } + + .empty-placeholder { + @include pin-full; + display: flex; + align-items: center; + justify-content: center; + font-size: 20px; + user-select: none; + pointer-events: none; + } +} + +.segment-bar-chart-bubble { + .button-bar { + padding-top: 5px; + display: flex; + gap: 5px; + } +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx b/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx new file mode 100644 index 000000000000..6f2817a202eb --- /dev/null +++ b/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx @@ -0,0 +1,793 @@ +/* + * 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, Intent } from '@blueprintjs/core'; +import type { NonNullDateRange } from '@blueprintjs/datetime'; +import { IconNames } from '@blueprintjs/icons'; +import IntervalTree from '@flatten-js/interval-tree'; +import classNames from 'classnames'; +import { max, sort, sum } from 'd3-array'; +import { axisBottom, axisLeft } from 'd3-axis'; +import { scaleLinear, scaleUtc } from 'd3-scale'; +import type { MouseEvent as ReactMouseEvent, ReactNode } from 'react'; +import { useMemo, useRef, useState } from 'react'; + +import type { Rule } from '../../druid-models'; +import { getDatasourceColor, RuleUtil } from '../../druid-models'; +import { useClock, useGlobalEventListener } from '../../hooks'; +import { + allSameValue, + arraysEqualByElement, + clamp, + day, + Duration, + formatBytes, + formatNumber, + groupBy, + groupByAsMap, + minute, + month, + pluralIfNeeded, + TZ_UTC, + uniq, +} from '../../utils'; +import type { Margin, Stage } from '../../utils/stage'; +import type { PortalBubbleOpenOn } from '../portal-bubble/portal-bubble'; +import { PortalBubble } from '../portal-bubble/portal-bubble'; + +import { ChartAxis } from './chart-axis'; +import type { IntervalBar, IntervalRow, IntervalStat, TrimmedIntervalRow } from './common'; +import { aggregateSegmentStats, formatIntervalStat, formatIsoDateOnly } from './common'; + +import './segment-bar-chart-render.scss'; + +const CHART_MARGIN: Margin = { top: 20, right: 0, bottom: 25, left: 70 }; +const MIN_BAR_WIDTH = 4; +const POSSIBLE_GRANULARITIES = [ + new Duration('PT15M'), + new Duration('PT1H'), + new Duration('PT6H'), + new Duration('P1D'), + new Duration('P1M'), + new Duration('P1Y'), +]; + +const EXTEND_X_SCALE_DOMAIN_BY = 1; + +function formatStartDuration(start: Date, duration: Duration): string { + let sliceLength; + const { singleSpan } = duration; + switch (singleSpan) { + case 'year': + sliceLength = 4; + break; + + case 'month': + sliceLength = 7; + break; + + case 'day': + sliceLength = 10; + break; + + case 'hour': + sliceLength = 13; + break; + + case 'minute': + sliceLength = 16; + break; + + default: + sliceLength = 19; + break; + } + + return `${start.toISOString().slice(0, sliceLength)}/${duration}`; +} + +// --------------------------------------- +// Load rule stuff + +function loadRuleToBaseType(loadRule: Rule): string { + const m = /^(load|drop|broadcast)/.exec(loadRule.type); + return m ? m[1] : 'load'; +} + +const NEGATIVE_INFINITY_DATE = new Date(Date.UTC(1000, 0, 1)); +const POSITIVE_INFINITY_DATE = new Date(Date.UTC(3000, 0, 1)); + +function loadRuleToDateRange(loadRule: Rule): NonNullDateRange { + switch (loadRule.type) { + case 'loadByInterval': + case 'dropByInterval': + case 'broadcastByInterval': + return String(loadRule.interval) + .split('/') + .map(d => new Date(d)) as NonNullDateRange; + + case 'loadByPeriod': + case 'dropByPeriod': + case 'broadcastByPeriod': + return [ + new Duration(loadRule.period || 'P1D').shift(new Date(), TZ_UTC, -1), + loadRule.includeFuture ? POSITIVE_INFINITY_DATE : new Date(), + ]; + + case 'dropBeforeByPeriod': + return [ + NEGATIVE_INFINITY_DATE, + new Duration(loadRule.period || 'P1D').shift(new Date(), TZ_UTC, -1), + ]; + + default: + return [NEGATIVE_INFINITY_DATE, POSITIVE_INFINITY_DATE]; + } +} + +// --------------------------------------- + +function offsetDateRange(dateRange: NonNullDateRange, offset: number): NonNullDateRange { + return [new Date(dateRange[0].valueOf() + offset), new Date(dateRange[1].valueOf() + offset)]; +} + +function stackIntervalRows(trimmedIntervalRows: TrimmedIntervalRow[]): { + intervalBars: IntervalBar[]; + intervalTree: IntervalTree; +} { + // Total size of the datasource will be user as an ordering tiebreaker + const datasourceToTotalSize = groupByAsMap( + trimmedIntervalRows, + intervalRow => intervalRow.datasource, + intervalRows => sum(intervalRows, intervalRow => intervalRow.size), + ); + + const sortedIntervalRows = sort(trimmedIntervalRows, (a, b) => { + const shownDaysDiff = b.shownDays - a.shownDays; + if (shownDaysDiff) return shownDaysDiff; + + const timeSpanDiff = + b.originalTimeSpan.getCanonicalLength() - a.originalTimeSpan.getCanonicalLength(); + if (timeSpanDiff) return timeSpanDiff; + + const totalSizeDiff = datasourceToTotalSize[b.datasource] - datasourceToTotalSize[a.datasource]; + if (totalSizeDiff) return totalSizeDiff; + + return Number(a.realtime) - Number(b.realtime); + }); + + const intervalTree = new IntervalTree(); + const intervalBars = sortedIntervalRows.map(intervalRow => { + const startMs = intervalRow.start.valueOf(); + const endMs = intervalRow.end.valueOf(); + const intervalRowsBelow = intervalTree.search([startMs + 1, startMs + 2]) as IntervalBar[]; + const intervalBar: IntervalBar = { + ...intervalRow, + offset: aggregateSegmentStats(intervalRowsBelow.map(i => i.normalized)), + }; + intervalTree.insert([startMs, endMs], intervalBar); + return intervalBar; + }); + + return { + intervalBars, + intervalTree, + }; +} + +interface BubbleInfo { + start: Date; + end: Date; + timeLabel: string; + intervalBars: IntervalBar[]; +} + +interface SelectionRange { + start: Date; + end: Date; + done?: boolean; +} + +export interface DatasourceRules { + loadRules: Rule[]; + defaultLoadRules: Rule[]; +} + +export interface SegmentBarChartRenderProps { + intervalRows: IntervalRow[]; + datasourceRules: DatasourceRules | undefined; + datasourceRulesError: string | undefined; + + stage: Stage; + dateRange: NonNullDateRange; + changeDateRange(dateRange: NonNullDateRange): void; + shownIntervalStat: IntervalStat; + shownDatasource: string | undefined; + changeShownDatasource(datasource: string | undefined): void; + getIntervalActionButton?( + start: Date, + end: Date, + datasource?: string, + realtime?: boolean, + ): ReactNode; +} + +export const SegmentBarChartRender = function SegmentBarChartRender( + props: SegmentBarChartRenderProps, +) { + const { + intervalRows, + datasourceRules, + datasourceRulesError, + + stage, + shownIntervalStat, + dateRange, + changeDateRange, + shownDatasource, + changeShownDatasource, + getIntervalActionButton, + } = props; + const [mouseDownAt, setMouseDownAt] = useState< + { time: Date; action: 'select' | 'shift' } | undefined + >(); + const [selection, setSelection] = useState(); + + function setSelectionIfNeeded(newSelection: SelectionRange) { + if ( + selection && + selection.start.valueOf() === newSelection.start.valueOf() && + selection.end.valueOf() === newSelection.end.valueOf() && + selection.done === newSelection.done + ) { + return; + } + setSelection(newSelection); + } + + const [bubbleInfo, setBubbleInfo] = useState(); + + function setBubbleInfoIfNeeded(newBubbleInfo: BubbleInfo) { + if ( + bubbleInfo && + bubbleInfo.start.valueOf() === newBubbleInfo.start.valueOf() && + bubbleInfo.end.valueOf() === newBubbleInfo.end.valueOf() && + bubbleInfo.timeLabel === newBubbleInfo.timeLabel && + arraysEqualByElement(bubbleInfo.intervalBars, newBubbleInfo.intervalBars) + ) { + return; + } + setBubbleInfo(newBubbleInfo); + } + + const [shiftOffset, setShiftOffset] = useState(); + + const now = useClock(minute.canonicalLength); + const svgRef = useRef(null); + + const trimGranularity = useMemo(() => { + return Duration.pickSmallestGranularityThatFits( + POSSIBLE_GRANULARITIES, + dateRange[1].valueOf() - dateRange[0].valueOf(), + Math.floor(stage.width / MIN_BAR_WIDTH), + ).toString(); + }, [dateRange, stage.width]); + + const { intervalBars, intervalTree } = useMemo(() => { + const shownIntervalRows = intervalRows.filter( + ({ start, end, datasource }) => + start <= dateRange[1] && + dateRange[0] < end && + (!shownDatasource || datasource === shownDatasource), + ); + const averageRowSizeByDatasource = groupByAsMap( + shownIntervalRows.filter(intervalRow => intervalRow.size > 0 && intervalRow.rows > 0), + intervalRow => intervalRow.datasource, + intervalRows => sum(intervalRows, d => d.size) / sum(intervalRows, d => d.rows), + ); + + const trimDuration = new Duration(trimGranularity); + const trimmedIntervalRows = shownIntervalRows.map(intervalRow => { + const { start, end, segments, size, rows } = intervalRow; + const startTrimmed = trimDuration.floor(start, TZ_UTC); + let endTrimmed = trimDuration.ceil(end, TZ_UTC); + + // Special handling to catch WEEK intervals when trimming to month. + if (trimGranularity === 'P1M' && intervalRow.originalTimeSpan.toString() === 'P7D') { + endTrimmed = trimDuration.shift(startTrimmed, TZ_UTC); + } + + const shownDays = (endTrimmed.valueOf() - startTrimmed.valueOf()) / day.canonicalLength; + const shownSize = + size === 0 ? rows * averageRowSizeByDatasource[intervalRow.datasource] : size; + return { + ...intervalRow, + start: startTrimmed, + end: endTrimmed, + shownDays, + size: shownSize, + normalized: { + size: shownSize / shownDays, + rows: rows / shownDays, + segments: segments / shownDays, + }, + }; + }); + + const fullyGroupedSegmentRows = groupBy( + trimmedIntervalRows, + trimmedIntervalRow => + [ + trimmedIntervalRow.start.toISOString(), + trimmedIntervalRow.end.toISOString(), + trimmedIntervalRow.originalTimeSpan, + trimmedIntervalRow.datasource, + trimmedIntervalRow.realtime, + ].join('/'), + (trimmedIntervalRows): TrimmedIntervalRow => { + const firstIntervalRow = trimmedIntervalRows[0]; + return { + ...firstIntervalRow, + ...aggregateSegmentStats(trimmedIntervalRows), + normalized: aggregateSegmentStats(trimmedIntervalRows.map(t => t.normalized)), + }; + }, + ); + + return stackIntervalRows(fullyGroupedSegmentRows); + }, [intervalRows, trimGranularity, dateRange, shownDatasource]); + + const innerStage = stage.applyMargin(CHART_MARGIN); + + const baseTimeScale = scaleUtc() + .domain(dateRange) + .range([EXTEND_X_SCALE_DOMAIN_BY, innerStage.width - EXTEND_X_SCALE_DOMAIN_BY]); + const timeScale = shiftOffset + ? baseTimeScale.copy().domain(offsetDateRange(dateRange, shiftOffset)) + : baseTimeScale; + + const maxNormalizedStat = max( + intervalBars, + d => d.normalized[shownIntervalStat] + d.offset[shownIntervalStat], + ); + const statScale = scaleLinear() + .rangeRound([innerStage.height, 0]) + .domain([0, (maxNormalizedStat ?? 1) * 1.05]); + + const formatTickRate = (n: number) => { + switch (shownIntervalStat) { + case 'segments': + return formatNumber(n); // + ' seg/day'; + + case 'rows': + return formatNumber(n); // + ' row/day'; + + case 'size': + return formatBytes(n); + } + }; + + function handleMouseDown(e: ReactMouseEvent) { + const svg = svgRef.current; + if (!svg) return; + e.preventDefault(); + + if (selection) { + setSelection(undefined); + } else { + const rect = svg.getBoundingClientRect(); + const x = e.clientX - rect.x - CHART_MARGIN.left; + const y = e.clientY - rect.y - CHART_MARGIN.top; + const time = baseTimeScale.invert(x); + const action = y > innerStage.height || e.shiftKey ? 'shift' : 'select'; + setBubbleInfo(undefined); + setMouseDownAt({ + time, + action, + }); + } + } + + useGlobalEventListener('mousemove', (e: MouseEvent) => { + const svg = svgRef.current; + if (!svg) return; + const rect = svg.getBoundingClientRect(); + const x = e.clientX - rect.x - CHART_MARGIN.left; + const y = e.clientY - rect.y - CHART_MARGIN.top; + + if (mouseDownAt) { + e.preventDefault(); + + const b = baseTimeScale.invert(x); + if (mouseDownAt.action === 'shift' || e.shiftKey) { + setShiftOffset(mouseDownAt.time.valueOf() - b.valueOf()); + } else { + if (mouseDownAt.time < b) { + setSelectionIfNeeded({ + start: day.floor(mouseDownAt.time, TZ_UTC), + end: day.ceil(b, TZ_UTC), + }); + } else { + setSelectionIfNeeded({ + start: day.floor(b, TZ_UTC), + end: day.ceil(mouseDownAt.time, TZ_UTC), + }); + } + } + } else if (!selection) { + if ( + 0 <= x && + x <= innerStage.width && + 0 <= y && + y <= innerStage.height + CHART_MARGIN.bottom + ) { + const time = baseTimeScale.invert(x); + const shifter = + new Duration(trimGranularity).getCanonicalLength() > day.canonicalLength * 25 + ? month + : day; + const start = shifter.floor(time, TZ_UTC); + const end = shifter.ceil(time, TZ_UTC); + + let intervalBars: IntervalBar[] = []; + if (y <= innerStage.height) { + const bars = intervalTree.search([ + time.valueOf() + 1, + time.valueOf() + 2, + ]) as IntervalBar[]; + + if (bars.length) { + const stat = statScale.invert(y); + const hoverBar = bars.find( + bar => + bar.offset[shownIntervalStat] <= stat && + stat < bar.offset[shownIntervalStat] + bar.normalized[shownIntervalStat], + ); + intervalBars = hoverBar ? [hoverBar] : bars; + } + } + setBubbleInfoIfNeeded({ + start, + end, + timeLabel: start.toISOString().slice(0, shifter === day ? 10 : 7), + intervalBars, + }); + } else { + setBubbleInfo(undefined); + } + } + }); + + useGlobalEventListener('mouseup', (e: MouseEvent) => { + if (!mouseDownAt) return; + e.preventDefault(); + setMouseDownAt(undefined); + + const svg = svgRef.current; + if (!svg) return; + const rect = svg.getBoundingClientRect(); + const x = e.clientX - rect.x - CHART_MARGIN.left; + const y = e.clientY - rect.y - CHART_MARGIN.top; + + if (shiftOffset || selection) { + setShiftOffset(undefined); + if (mouseDownAt.action === 'shift' || e.shiftKey) { + if (shiftOffset) { + changeDateRange(offsetDateRange(dateRange, shiftOffset)); + } + } else { + if (selection) { + setSelection({ ...selection, done: true }); + } + } + } else if (0 <= x && x <= innerStage.width && 0 <= y && y <= innerStage.height) { + const time = baseTimeScale.invert(x); + + const bars = intervalTree.search([time.valueOf() + 1, time.valueOf() + 2]) as IntervalBar[]; + + if (bars.length) { + const stat = statScale.invert(y); + const hoverBar = bars.find( + bar => + bar.offset[shownIntervalStat] <= stat && + stat < bar.offset[shownIntervalStat] + bar.normalized[shownIntervalStat], + ); + if (hoverBar) { + changeShownDatasource(shownDatasource ? undefined : hoverBar.datasource); + } + } + } + }); + + useGlobalEventListener('keydown', (e: KeyboardEvent) => { + if (e.key === 'Escape' && mouseDownAt) { + setMouseDownAt(undefined); + setSelection(undefined); + } + }); + + function startEndToXWidth({ start, end }: { start: Date; end: Date }) { + const xStart = clamp(timeScale(start), 0, innerStage.width); + const xEnd = clamp(timeScale(end), 0, innerStage.width); + + return { + x: xStart, + width: Math.max(xEnd - xStart - 1, 1), + }; + } + + function segmentBarToRect(intervalBar: IntervalBar) { + const y0 = statScale(intervalBar.offset[shownIntervalStat]); + const y = statScale( + intervalBar.normalized[shownIntervalStat] + intervalBar.offset[shownIntervalStat], + ); + + return { + ...startEndToXWidth(intervalBar), + y: y, + height: y0 - y, + }; + } + + let hoveredOpenOn: PortalBubbleOpenOn | undefined; + if (svgRef.current) { + const rect = svgRef.current.getBoundingClientRect(); + + if (bubbleInfo) { + const hoveredIntervalBars = bubbleInfo.intervalBars; + + let title: string | undefined; + let text: ReactNode; + if (hoveredIntervalBars.length === 0) { + title = bubbleInfo.timeLabel; + text = ''; + } else if (hoveredIntervalBars.length === 1) { + const hoveredIntervalBar = hoveredIntervalBars[0]; + title = `${formatStartDuration( + hoveredIntervalBar.start, + hoveredIntervalBar.originalTimeSpan, + )}${hoveredIntervalBar.realtime ? ' (realtime)' : ''}`; + text = ( + <> + {!shownDatasource &&
{`Datasource: ${hoveredIntervalBar.datasource}`}
} +
{`Size: ${ + hoveredIntervalBar.realtime + ? 'estimated for realtime' + : formatIntervalStat('size', hoveredIntervalBar.size) + }`}
+
{`Rows: ${formatIntervalStat('rows', hoveredIntervalBar.rows)}`}
+
{`Segments: ${formatIntervalStat('segments', hoveredIntervalBar.segments)}`}
+ + ); + } else { + const datasources = uniq(hoveredIntervalBars.map(b => b.datasource)); + const agg = aggregateSegmentStats(hoveredIntervalBars); + title = bubbleInfo.timeLabel; + text = ( + <> + {!shownDatasource && ( +
{`Totals for ${pluralIfNeeded(datasources.length, 'datasource')}`}
+ )} +
{`Size: ${formatIntervalStat('size', agg.size)}`}
+
{`Rows: ${formatIntervalStat('rows', agg.rows)}`}
+
{`Segments: ${formatIntervalStat('segments', agg.segments)}`}
+ + ); + } + + hoveredOpenOn = { + x: + rect.x + + CHART_MARGIN.left + + timeScale(new Date((bubbleInfo.start.valueOf() + bubbleInfo.end.valueOf()) / 2)), + y: rect.y + CHART_MARGIN.top, + title, + text, + }; + } else if (selection) { + const selectedBars = intervalTree.search([ + selection.start.valueOf() + 1, + selection.end.valueOf() - 1, + ]) as IntervalBar[]; + const datasources = uniq(selectedBars.map(b => b.datasource)); + const realtime = allSameValue(selectedBars.map(b => b.realtime)); + const agg = aggregateSegmentStats(selectedBars); + hoveredOpenOn = { + x: + rect.x + + CHART_MARGIN.left + + timeScale(new Date((selection.start.valueOf() + selection.end.valueOf()) / 2)), + y: rect.y + CHART_MARGIN.top, + title: `${formatIsoDateOnly(selection.start)} → ${formatIsoDateOnly(selection.end)}`, + text: ( + <> + {selectedBars.length ? ( + <> + {!shownDatasource && ( +
{`Totals for ${pluralIfNeeded(datasources.length, 'datasource')}`}
+ )} +
{`Size: ${formatIntervalStat('size', agg.size)}`}
+
{`Rows: ${formatIntervalStat('rows', agg.rows)}`}
+
{`Segments: ${formatIntervalStat('segments', agg.segments)}`}
+ + ) : ( +
No segments in this interval
+ )} + {selection.done && ( +
+
+ )} + + ), + }; + } + } + + function renderLoadRule(loadRule: Rule, i: number, isDefault: boolean) { + const [start, end] = loadRuleToDateRange(loadRule); + const { x, width } = startEndToXWidth({ start, end }); + const title = RuleUtil.ruleToString(loadRule) + (isDefault ? ' (cluster default)' : ''); + return ( +
+ {title} +
+ ); + } + + const nowX = timeScale(now); + return ( +
+ + + v !== 0)) + .tickSize(-innerStage.width) + .tickFormat(() => '') + .tickSizeOuter(0)} + /> + + + formatTickRate(e.valueOf()))} + /> + + {bubbleInfo && ( + + )} + {0 < nowX && nowX < innerStage.width && ( + + )} + {intervalBars.map((intervalBar, i) => { + return ( + + ); + })} + {bubbleInfo?.intervalBars.length === 1 && + bubbleInfo.intervalBars.map((intervalBar, i) => ( + + ))} + {selection && ( + + )} + {!!shiftOffset && ( + 0 ? timeScale(dateRange[1]) : 0} + y={0} + height={innerStage.height} + width={ + shiftOffset > 0 + ? innerStage.width - timeScale(dateRange[1]) + : timeScale(dateRange[0]) + } + /> + )} + + + + {(datasourceRules || datasourceRulesError) && ( +
+ {datasourceRules?.defaultLoadRules.map((rule, index) => + renderLoadRule(rule, index, true), + )} + {datasourceRules?.loadRules.map((rule, index) => renderLoadRule(rule, index, false))} + {datasourceRulesError && ( +
Rule loading error: {datasourceRulesError}
+ )} +
+ )} + {!intervalRows.length && ( +
+
There are no segments in the selected range
+
+ )} + setSelection(undefined) : undefined} + mute + direction="up" + /> +
+ ); +}; diff --git a/web-console/src/components/segment-timeline/bar-unit.spec.tsx b/web-console/src/components/segment-timeline/segment-bar-chart.scss similarity index 68% rename from web-console/src/components/segment-timeline/bar-unit.spec.tsx rename to web-console/src/components/segment-timeline/segment-bar-chart.scss index d5926dcf69f0..130997119e3e 100644 --- a/web-console/src/components/segment-timeline/bar-unit.spec.tsx +++ b/web-console/src/components/segment-timeline/segment-bar-chart.scss @@ -16,18 +16,6 @@ * limitations under the License. */ -import { render } from '@testing-library/react'; - -import { BarUnit } from './bar-unit'; - -describe('BarUnit', () => { - it('matches snapshot', () => { - const barGroup = ( - - - - ); - const { container } = render(barGroup); - expect(container.firstChild).toMatchSnapshot(); - }); -}); +.segment-bar-chart { + position: relative; +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart.tsx b/web-console/src/components/segment-timeline/segment-bar-chart.tsx new file mode 100644 index 000000000000..297ea4afe617 --- /dev/null +++ b/web-console/src/components/segment-timeline/segment-bar-chart.tsx @@ -0,0 +1,162 @@ +/* + * 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 { C, F, L, N, sql, SqlExpression, SqlQuery } from 'druid-query-toolkit'; +import { useMemo } from 'react'; + +import { END_OF_TIME_DATE, type Rule, RuleUtil, START_OF_TIME_DATE } from '../../druid-models'; +import type { Capabilities } from '../../helpers'; +import { useQueryManager } from '../../hooks'; +import { Api } from '../../singletons'; +import { Duration, filterMap, getApiArray, queryDruidSql, TZ_UTC } from '../../utils'; +import { Loader } from '../loader/loader'; + +import type { IntervalRow } from './common'; +import type { SegmentBarChartRenderProps } from './segment-bar-chart-render'; +import { SegmentBarChartRender } from './segment-bar-chart-render'; + +import './segment-bar-chart.scss'; + +export interface SegmentBarChartProps + extends Omit< + SegmentBarChartRenderProps, + 'intervalRows' | 'datasourceRules' | 'datasourceRulesError' + > { + capabilities: Capabilities; +} + +export const SegmentBarChart = function SegmentBarChart(props: SegmentBarChartProps) { + const { capabilities, dateRange, shownDatasource, ...otherProps } = props; + + const intervalsQuery = useMemo( + () => ({ capabilities, dateRange, shownDatasource: shownDatasource }), + [capabilities, dateRange, shownDatasource], + ); + + const [intervalRowsState] = useQueryManager({ + query: intervalsQuery, + processQuery: async ({ capabilities, dateRange, shownDatasource }, cancelToken) => { + if (capabilities.hasSql()) { + const query = SqlQuery.from(N('sys').table('segments')) + .changeWhereExpression( + SqlExpression.and( + sql`"start" <= '${dateRange[1].toISOString()}' AND '${dateRange[0].toISOString()}' < "end"`, + C('start').unequal(START_OF_TIME_DATE), + C('end').unequal(END_OF_TIME_DATE), + C('is_overshadowed').equal(0), + shownDatasource ? C('datasource').equal(L(shownDatasource)) : undefined, + ), + ) + .addSelect(C('start'), { addToGroupBy: 'end' }) + .addSelect(C('end'), { addToGroupBy: 'end' }) + .addSelect(C('datasource'), { addToGroupBy: 'end' }) + .addSelect(C('is_realtime').as('realtime'), { addToGroupBy: 'end' }) + .addSelect(F.count().as('segments')) + .addSelect(F.sum(C('size')).as('size')) + .addSelect(F.sum(C('num_rows')).as('rows')) + .toString(); + + return (await queryDruidSql({ query }, cancelToken)).map(sr => { + const start = new Date(sr.start); + const end = new Date(sr.end); + + return { + ...sr, + start, + end, + realtime: Boolean(sr.realtime), + originalTimeSpan: Duration.fromRange(start, end, TZ_UTC), + } as IntervalRow; + }); + } else { + return filterMap( + await getApiArray( + `/druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&includeRealtimeSegments&${ + shownDatasource ? `datasources=${Api.encodePath(shownDatasource)}` : '' + }`, + cancelToken, + ), + (segment: any) => { + if (segment.overshadowed) return; // We have to include overshadowed segments to get the realtime segments in this API + const [startStr, endStr] = segment.interval.split('/'); + if (startStr === START_OF_TIME_DATE && endStr === END_OF_TIME_DATE) return; + const start = new Date(startStr); + const end = new Date(endStr); + if (!(start <= dateRange[1] && dateRange[0] < end)) return; + + return { + start, + end, + datasource: segment.dataSource, + realtime: Boolean(segment.realtime), + originalTimeSpan: Duration.fromRange(start, end, TZ_UTC), + segments: 1, + size: segment.size, + rows: segment.num_rows || 0, // segment.num_rows is really null on this API :-( + } as IntervalRow; + }, + ); + } + }, + }); + + const [allLoadRulesState] = useQueryManager({ + query: shownDatasource ? '' : undefined, + processQuery: async (_, cancelToken) => { + return ( + await Api.instance.get>('/druid/coordinator/v1/rules', { + cancelToken, + }) + ).data; + }, + }); + + const datasourceRules = useMemo(() => { + const allLoadRules = allLoadRulesState.data; + if (!allLoadRules || !shownDatasource) return; + return { + loadRules: (allLoadRules[shownDatasource] || []).toReversed(), + defaultLoadRules: (allLoadRules[RuleUtil.DEFAULT_RULES_KEY] || []).toReversed(), + }; + }, [allLoadRulesState.data, shownDatasource]); + + if (intervalRowsState.error) { + return ( +
+ {`Error when loading data: ${intervalRowsState.getErrorMessage()}`} +
+ ); + } + + const intervalRows = intervalRowsState.getSomeData(); + return ( + <> + {intervalRows && ( + + )} + {intervalRowsState.loading && } + + ); +}; diff --git a/web-console/src/components/segment-timeline/segment-timeline.scss b/web-console/src/components/segment-timeline/segment-timeline.scss index aa437052d617..4224c1cab89d 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.scss +++ b/web-console/src/components/segment-timeline/segment-timeline.scss @@ -16,12 +16,20 @@ * limitations under the License. */ +@import '../../variables'; + .segment-timeline { - display: grid; - grid-template-columns: 1fr 220px; + .control-bar { + @include card-like; + height: 34px; + display: flex; + align-items: start; + padding: 5px; + gap: 10px; - .loader { - width: 85%; + & > .expander { + flex: 1; + } } .loading-error { @@ -31,14 +39,16 @@ transform: translate(-50%, -50%); } - .no-data-text { + .chart-container { position: absolute; - left: 30vw; - top: 15vh; - font-size: 20px; - } + top: 34px; + width: 100%; + bottom: 0; + overflow: hidden; - .side-control { - padding-top: 20px; + .segment-bar-chart, + .segment-bar-chart-render { + @include pin-full; + } } } diff --git a/web-console/src/components/segment-timeline/segment-timeline.spec.tsx b/web-console/src/components/segment-timeline/segment-timeline.spec.tsx index 4f95842801a4..ebc83ae143c7 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.spec.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.spec.tsx @@ -16,40 +16,42 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; import { render } from '@testing-library/react'; import { Capabilities } from '../../helpers'; +import { QueryState } from '../../utils'; import { SegmentTimeline } from './segment-timeline'; -jest.useFakeTimers('modern').setSystemTime(Date.parse('2021-06-08T12:34:56Z')); +jest.useFakeTimers('modern').setSystemTime(Date.parse('2024-11-19T12:34:56Z')); -describe('SegmentTimeline', () => { - it('.getSqlQuery', () => { - expect( - SegmentTimeline.getSqlQuery([ - new Date('2020-01-01T00:00:00Z'), - new Date('2021-02-01T00:00:00Z'), - ]), - ).toEqual(sane` - SELECT - "start", "end", "datasource", - COUNT(*) AS "count", - SUM("size") AS "size" - FROM sys.segments - WHERE - '2020-01-01T00:00:00.000Z' <= "start" AND - "end" <= '2021-02-01T00:00:00.000Z' AND - is_published = 1 AND - is_overshadowed = 0 - GROUP BY 1, 2, 3 - ORDER BY "start" DESC - `); - }); +jest.mock('../../hooks', () => { + return { + useQueryManager: (options: any) => { + if (options.initQuery instanceof Capabilities) { + // This is a query for data sources + return [new QueryState({ data: ['ds1', 'ds2'] })]; + } + + if (options.query === null) { + // This is a query for the data source time range + return [ + new QueryState({ + data: [new Date('2024-11-01 00:00:00Z'), new Date('2024-11-18 00:00:00Z')], + }), + ]; + } + return new QueryState({ error: new Error('not covered') }); + }, + }; +}); + +describe('SegmentTimeline', () => { it('matches snapshot', () => { - const segmentTimeline = ; + const segmentTimeline = ( + + ); const { container } = render(segmentTimeline); expect(container.firstChild).toMatchSnapshot(); }); diff --git a/web-console/src/components/segment-timeline/segment-timeline.tsx b/web-console/src/components/segment-timeline/segment-timeline.tsx index 8aee0c66d477..2774c73276eb 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.tsx @@ -16,628 +16,361 @@ * limitations under the License. */ -import { Button, FormGroup, MenuItem, ResizeSensor, SegmentedControl } from '@blueprintjs/core'; -import type { DateRange, NonNullDateRange } from '@blueprintjs/datetime'; -import { DateRangeInput3 } from '@blueprintjs/datetime2'; +import { + Button, + ButtonGroup, + Intent, + Menu, + MenuItem, + Popover, + Position, + ResizeSensor, +} from '@blueprintjs/core'; +import type { NonNullDateRange } from '@blueprintjs/datetime'; +import { DateRangePicker3 } from '@blueprintjs/datetime2'; import { IconNames } from '@blueprintjs/icons'; -import type { ItemPredicate, ItemRenderer } from '@blueprintjs/select'; import { Select } from '@blueprintjs/select'; -import type { AxisScale } from 'd3-axis'; -import { scaleLinear, scaleUtc } from 'd3-scale'; -import enUS from 'date-fns/locale/en-US'; -import React from 'react'; +import { C, L, N, SqlExpression, SqlQuery } from 'druid-query-toolkit'; +import { useEffect, useMemo, useState } from 'react'; +import { END_OF_TIME_DATE, START_OF_TIME_DATE } from '../../druid-models'; import type { Capabilities } from '../../helpers'; -import { Api } from '../../singletons'; +import { useQueryManager } from '../../hooks'; import { - ceilToUtcDay, - formatBytes, - formatInteger, + checkedCircleIcon, + day, + Duration, + getApiArray, isNonNullRange, localToUtcDateRange, + maxDate, queryDruidSql, - QueryManager, - uniq, + TZ_UTC, utcToLocalDateRange, } from '../../utils'; +import { Stage } from '../../utils/stage'; import { Loader } from '../loader/loader'; -import type { BarUnitData } from './stacked-bar-chart'; -import { StackedBarChart } from './stacked-bar-chart'; +import type { IntervalStat } from './common'; +import { formatIsoDateOnly, getIntervalStatTitle, INTERVAL_STATS } from './common'; +import type { SegmentBarChartProps } from './segment-bar-chart'; +import { SegmentBarChart } from './segment-bar-chart'; import './segment-timeline.scss'; -interface SegmentTimelineProps { - capabilities: Capabilities; -} - -type ActiveDataType = 'sizeData' | 'countData'; - -interface SegmentTimelineState { - chartHeight: number; - chartWidth: number; - data?: Record; - datasources: string[]; - stackedData?: Record; - singleDatasourceData?: Record>; - activeDatasource: string | null; - activeDataType: ActiveDataType; - dataToRender: BarUnitData[]; - loading: boolean; - error?: Error; - xScale: AxisScale | null; - yScale: AxisScale | null; - dateRange: NonNullDateRange; - selectedDateRange?: DateRange; +const DEFAULT_SHOWN_DURATION = new Duration('P1Y'); +const SHOWN_DURATION_OPTIONS: Duration[] = [ + new Duration('P1D'), + new Duration('P1W'), + new Duration('P1M'), + new Duration('P3M'), + new Duration('P1Y'), + new Duration('P5Y'), + new Duration('P10Y'), +]; + +function getDateRange(shownDuration: Duration): NonNullDateRange { + const end = day.ceil(new Date(), TZ_UTC); + return [shownDuration.shift(end, TZ_UTC, -1), end]; } -interface BarChartScales { - xScale: AxisScale; - yScale: AxisScale; +function formatDateRange(dateRange: NonNullDateRange): string { + return `${formatIsoDateOnly(dateRange[0])} → ${formatIsoDateOnly(dateRange[1])}`; } -interface IntervalRow { - start: string; - end: string; - datasource: string; - count: number; - size: number; +function dateRangesEqual(dr1: NonNullDateRange, dr2: NonNullDateRange): boolean { + return dr1[0].valueOf() === dr2[0].valueOf() && dr2[1].valueOf() === dr2[1].valueOf(); } -const DEFAULT_TIME_SPAN_MONTHS = 3; - -function getDefaultDateRange(): NonNullDateRange { - const start = ceilToUtcDay(new Date()); - const end = new Date(start.valueOf()); - start.setUTCMonth(start.getUTCMonth() - DEFAULT_TIME_SPAN_MONTHS); - return [start, end]; +interface SegmentTimelineProps extends Pick { + capabilities: Capabilities; + datasource: string | undefined; } -export class SegmentTimeline extends React.PureComponent< - SegmentTimelineProps, - SegmentTimelineState -> { - static COLORS = [ - '#b33040', - '#d25c4d', - '#f2b447', - '#d9d574', - '#4FAA7E', - '#57ceff', - '#789113', - '#098777', - '#b33040', - '#d2757b', - '#f29063', - '#d9a241', - '#80aa61', - '#c4ff9e', - '#915412', - '#87606c', - ]; - - static getColor(index: number): string { - return SegmentTimeline.COLORS[index % SegmentTimeline.COLORS.length]; - } - - static getSqlQuery(dateRange: NonNullDateRange): string { - return `SELECT - "start", "end", "datasource", - COUNT(*) AS "count", - SUM("size") AS "size" -FROM sys.segments -WHERE - '${dateRange[0].toISOString()}' <= "start" AND - "end" <= '${dateRange[1].toISOString()}' AND - is_published = 1 AND - is_overshadowed = 0 -GROUP BY 1, 2, 3 -ORDER BY "start" DESC`; - } - - static processRawData(data: IntervalRow[]) { - if (data === null) return []; +export const SegmentTimeline = function SegmentTimeline(props: SegmentTimelineProps) { + const { capabilities, datasource, ...otherProps } = props; + const [stage, setStage] = useState(); + const [activeSegmentStat, setActiveSegmentStat] = useState('size'); + const [shownDatasource, setShownDatasource] = useState(datasource); + const [dateRange, setDateRange] = useState(); + const [showCustomDatePicker, setShowCustomDatePicker] = useState(false); + + useEffect(() => { + setShownDatasource(datasource); + }, [datasource]); + + const defaultDateRange = useMemo(() => { + return getDateRange(DEFAULT_SHOWN_DURATION); + }, []); + + const [datasourcesState] = useQueryManager({ + initQuery: capabilities, + processQuery: async (capabilities, cancelToken) => { + if (capabilities.hasSql()) { + const tables = await queryDruidSql<{ TABLE_NAME: string }>( + { + query: `SELECT TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE = 'TABLE'`, + }, + cancelToken, + ); - const countData: Record = {}; - const sizeData: Record = {}; - data.forEach(entry => { - const start = entry.start; - const day = start.split('T')[0]; - const datasource = entry.datasource; - const count = entry.count; - const segmentSize = entry.size; - if (countData[day] === undefined) { - countData[day] = { - day, - [datasource]: count, - total: count, - }; - sizeData[day] = { - day, - [datasource]: segmentSize, - total: segmentSize, - }; + return tables.map(d => d.TABLE_NAME); } else { - const countDataEntry: number | undefined = countData[day][datasource]; - countData[day][datasource] = count + (countDataEntry === undefined ? 0 : countDataEntry); - const sizeDataEntry: number | undefined = sizeData[day][datasource]; - sizeData[day][datasource] = segmentSize + (sizeDataEntry === undefined ? 0 : sizeDataEntry); - countData[day].total += count; - sizeData[day].total += segmentSize; + return await getApiArray(`/druid/coordinator/v1/datasources`, cancelToken); } - }); - - const countDataArray = Object.keys(countData) - .reverse() - .map((time: any) => { - return countData[time]; - }); - - const sizeDataArray = Object.keys(sizeData) - .reverse() - .map((time: any) => { - return sizeData[time]; - }); - - return { countData: countDataArray, sizeData: sizeDataArray }; - } - - static calculateStackedData( - data: Record, - datasources: string[], - ): Record { - const newStackedData: Record = {}; - Object.keys(data).forEach((type: any) => { - const stackedData: any = data[type].map((d: any) => { - let y0 = 0; - return datasources.map((datasource: string, i) => { - const barUnitData = { - x: d.day, - y: d[datasource] === undefined ? 0 : d[datasource], - y0, - datasource, - color: SegmentTimeline.getColor(i), - dailySize: d.total, - }; - y0 += d[datasource] === undefined ? 0 : d[datasource]; - return barUnitData; - }); - }); - newStackedData[type] = stackedData.flat(); - }); + }, + }); + + const [initDatasourceDateRangeState] = useQueryManager({ + query: dateRange ? undefined : shownDatasource ?? null, + processQuery: async (datasource, cancelToken) => { + let queriedStart: Date; + let queriedEnd: Date; + if (capabilities.hasSql()) { + const baseQuery = SqlQuery.from(N('sys').table('segments')) + .changeWhereExpression( + SqlExpression.and( + C('start').unequal(START_OF_TIME_DATE), + C('end').unequal(END_OF_TIME_DATE), + C('is_overshadowed').equal(0), + datasource ? C('datasource').equal(L(datasource)) : undefined, + ), + ) + .changeLimitValue(1); - return newStackedData; - } + const endQuery = baseQuery + .addSelect(C('end'), { addToOrderBy: 'end', direction: 'DESC' }) + .toString(); - static calculateSingleDatasourceData( - data: Record, - datasources: string[], - ): Record> { - const singleDatasourceData: Record> = {}; - Object.keys(data).forEach(dataType => { - singleDatasourceData[dataType] = {}; - datasources.forEach((datasource, i) => { - const currentData = data[dataType]; - if (currentData.length === 0) return; - const dataResult = currentData.map((d: any) => { - let y = 0; - if (d[datasource] !== undefined) { - y = d[datasource]; - } - return { - x: d.day, - y, - datasource, - color: SegmentTimeline.getColor(i), - dailySize: d.total, - }; - }); - if (!dataResult.every((d: any) => d.y === 0)) { - singleDatasourceData[dataType][datasource] = dataResult; + const endRes = await queryDruidSql<{ end: string }>({ query: endQuery }, cancelToken).catch( + () => [], + ); + if (endRes.length !== 1) { + return getDateRange(DEFAULT_SHOWN_DURATION); } - }); - }); - return singleDatasourceData; - } - - private readonly dataQueryManager: QueryManager< - { capabilities: Capabilities; dateRange: NonNullDateRange }, - any - >; - - private readonly chartMargin = { top: 40, right: 15, bottom: 20, left: 60 }; - - constructor(props: SegmentTimelineProps) { - super(props); - const dateRange = getDefaultDateRange(); - - this.state = { - chartWidth: 1, // Dummy init values to be replaced - chartHeight: 1, // after first render - data: {}, - datasources: [], - stackedData: {}, - singleDatasourceData: {}, - dataToRender: [], - activeDatasource: null, - activeDataType: 'sizeData', - loading: true, - xScale: null, - yScale: null, - dateRange, - }; - - this.dataQueryManager = new QueryManager({ - processQuery: async ({ capabilities, dateRange }, cancelToken) => { - let intervals: IntervalRow[]; - let datasources: string[]; - if (capabilities.hasSql()) { - intervals = await queryDruidSql( - { - query: SegmentTimeline.getSqlQuery(dateRange), - }, - cancelToken, - ); - datasources = uniq(intervals.map(r => r.datasource).sort()); - } else if (capabilities.hasCoordinatorAccess()) { - const startIso = dateRange[0].toISOString(); + queriedEnd = day.ceil(new Date(endRes[0].end), TZ_UTC); - datasources = ( - await Api.instance.get(`/druid/coordinator/v1/datasources`, { cancelToken }) - ).data; - intervals = ( - await Promise.all( - datasources.map(async datasource => { - const intervalMap = ( - await Api.instance.get( - `/druid/coordinator/v1/datasources/${Api.encodePath( - datasource, - )}/intervals?simple`, - { cancelToken }, - ) - ).data; + const startQuery = baseQuery + .addSelect(C('start'), { addToOrderBy: 'end', direction: 'ASC' }) + .toString(); - return Object.keys(intervalMap) - .map(interval => { - const [start, end] = interval.split('/'); - const { count, size } = intervalMap[interval]; - return { - start, - end, - datasource, - count, - size, - }; - }) - .filter(a => startIso < a.start); - }), - ) - ) - .flat() - .sort((a, b) => b.start.localeCompare(a.start)); - } else { - throw new Error(`must have SQL or coordinator access`); + const startRes = await queryDruidSql<{ start: string }>( + { query: startQuery }, + cancelToken, + ).catch(() => []); + if (startRes.length !== 1) { + return [DEFAULT_SHOWN_DURATION.shift(queriedEnd, TZ_UTC, -1), queriedEnd]; // Should not really get here } - const data = SegmentTimeline.processRawData(intervals); - const stackedData = SegmentTimeline.calculateStackedData(data, datasources); - const singleDatasourceData = SegmentTimeline.calculateSingleDatasourceData( - data, - datasources, - ); - return { data, datasources, stackedData, singleDatasourceData }; - }, - onStateChange: ({ data, loading, error }) => { - this.setState({ - data: data ? data.data : undefined, - datasources: data ? data.datasources : [], - stackedData: data ? data.stackedData : undefined, - singleDatasourceData: data ? data.singleDatasourceData : undefined, - loading, - error, - }); - }, - }); - } - - componentDidMount(): void { - const { capabilities } = this.props; - const { dateRange } = this.state; - - if (isNonNullRange(dateRange)) { - this.dataQueryManager.runQuery({ capabilities, dateRange }); - } - } - - componentWillUnmount(): void { - this.dataQueryManager.terminate(); - } - - componentDidUpdate(_prevProps: SegmentTimelineProps, prevState: SegmentTimelineState): void { - const { activeDatasource, activeDataType, singleDatasourceData, stackedData } = this.state; - if ( - prevState.data !== this.state.data || - prevState.activeDataType !== this.state.activeDataType || - prevState.activeDatasource !== this.state.activeDatasource || - prevState.chartWidth !== this.state.chartWidth || - prevState.chartHeight !== this.state.chartHeight - ) { - const scales: BarChartScales | undefined = this.calculateScales(); - const dataToRender: BarUnitData[] | undefined = activeDatasource - ? singleDatasourceData - ? singleDatasourceData[activeDataType][activeDatasource] - : undefined - : stackedData - ? stackedData[activeDataType] - : undefined; - - if (scales && dataToRender) { - this.setState({ - dataToRender, - xScale: scales.xScale, - yScale: scales.yScale, - }); + queriedStart = day.floor(new Date(startRes[0].start), TZ_UTC); + } else { + // Don't bother querying if there is no SQL + return getDateRange(DEFAULT_SHOWN_DURATION); } - } - } - - private calculateScales(): BarChartScales | undefined { - const { - chartWidth, - chartHeight, - data, - activeDataType, - activeDatasource, - singleDatasourceData, - dateRange, - } = this.state; - if (!data || !Object.keys(data).length || !isNonNullRange(dateRange)) return; - const activeData = data[activeDataType]; - let yDomain: number[] = [ - 0, - activeData.length === 0 - ? 0 - : activeData.reduce((max: any, d: any) => (max.total > d.total ? max : d)).total, - ]; - - if ( - activeDatasource !== null && - singleDatasourceData![activeDataType][activeDatasource] !== undefined - ) { - yDomain = [ - 0, - singleDatasourceData![activeDataType][activeDatasource].reduce((max: any, d: any) => - max.y > d.y ? max : d, - ).y, + return [ + maxDate(queriedStart, DEFAULT_SHOWN_DURATION.shift(queriedEnd, TZ_UTC, -1)), + queriedEnd, ]; - } - - const xScale: AxisScale = scaleUtc() - .domain(dateRange) - .range([0, chartWidth - this.chartMargin.left - this.chartMargin.right]); - - const yScale: AxisScale = scaleLinear() - .rangeRound([chartHeight - this.chartMargin.top - this.chartMargin.bottom, 0]) - .domain(yDomain); - - return { - xScale, - yScale, - }; + }, + }); + + const effectiveDateRange = + dateRange || + initDatasourceDateRangeState.data || + (initDatasourceDateRangeState.isLoading() ? undefined : defaultDateRange); + + let previousDateRange: NonNullDateRange | undefined; + let zoomedOutDateRange: NonNullDateRange | undefined; + let nextDateRange: NonNullDateRange | undefined; + if (effectiveDateRange) { + const d = Duration.fromRange(effectiveDateRange[0], effectiveDateRange[1], TZ_UTC); + const shiftStartBack = d.shift(effectiveDateRange[0], TZ_UTC, -1); + const shiftEndForward = d.shift(effectiveDateRange[1], TZ_UTC); + const now = day.ceil(new Date(), TZ_UTC); + previousDateRange = [shiftStartBack, effectiveDateRange[0]]; + zoomedOutDateRange = [shiftStartBack, shiftEndForward < now ? shiftEndForward : now]; + nextDateRange = [effectiveDateRange[1], shiftEndForward]; } - private readonly formatTick = (n: number) => { - if (isNaN(n)) return ''; - const { activeDataType } = this.state; - if (activeDataType === 'countData') { - return formatInteger(n); - } else { - return formatBytes(n); - } - }; - - private readonly handleResize = (entries: ResizeObserverEntry[]) => { - const chartRect = entries[0].contentRect; - this.setState({ - chartWidth: chartRect.width, - chartHeight: chartRect.height, - }); - }; - - renderStackedBarChart() { - const { - chartWidth, - chartHeight, - loading, - dataToRender, - activeDataType, - error, - xScale, - yScale, - data, - activeDatasource, - dateRange, - } = this.state; - - if (loading) { - return ( -
- -
- ); - } - - if (error) { - return ( -
- Error when loading data: {error.message} -
- ); - } - - if (xScale === null || yScale === null) { - return ( -
- Error when calculating scales -
- ); - } - - if (data![activeDataType].length === 0) { - return ( -
- There are no segments for the selected interval -
- ); - } - - if ( - activeDatasource !== null && - data![activeDataType].every((d: any) => d[activeDatasource] === undefined) - ) { - return ( -
- - No data available for {activeDatasource} - -
- ); - } - - const millisecondsPerDay = 24 * 60 * 60 * 1000; - const barCounts = (dateRange[1].getTime() - dateRange[0].getTime()) / millisecondsPerDay; - const barWidth = Math.max( - 0, - (chartWidth - this.chartMargin.left - this.chartMargin.right) / barCounts, - ); - return ( - - - this.setState(prevState => ({ - activeDatasource: prevState.activeDatasource ? null : datasource, - })) + return ( +
+
+ + + items={datasourcesState.data || []} + disabled={datasourcesState.isError()} + onItemSelect={setShownDatasource} + itemRenderer={(val, { handleClick, handleFocus, modifiers }) => { + if (!modifiers.matchesPredicate) return null; + return ( + + ); + }} + noResults={} + itemPredicate={(query, val, _index, exactMatch) => { + const normalizedTitle = val.toLowerCase(); + const normalizedQuery = query.toLowerCase(); + + if (exactMatch) { + return normalizedTitle === normalizedQuery; + } else { + return normalizedTitle.includes(normalizedQuery); + } + }} + > +
+ { + const rect = entries[0].contentRect; + setStage(new Stage(rect.width, rect.height)); + }} + > +
+ {stage && effectiveDateRange && ( + - - - - + )} + {initDatasourceDateRangeState.isLoading() && }
-
- ); - } -} +
+
+ ); +}; diff --git a/web-console/src/components/segment-timeline/stacked-bar-chart.tsx b/web-console/src/components/segment-timeline/stacked-bar-chart.tsx deleted file mode 100644 index 8018aaee5f6f..000000000000 --- a/web-console/src/components/segment-timeline/stacked-bar-chart.tsx +++ /dev/null @@ -1,174 +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 { AxisScale } from 'd3-axis'; -import { axisBottom, axisLeft } from 'd3-axis'; -import React, { useState } from 'react'; - -import { BarGroup } from './bar-group'; -import { ChartAxis } from './chart-axis'; - -import './stacked-bar-chart.scss'; - -export interface BarUnitData { - x: number; - y: number; - y0?: number; - width: number; - datasource: string; - color: string; - dailySize?: number; -} - -export interface BarChartMargin { - top: number; - right: number; - bottom: number; - left: number; -} - -export interface HoveredBarInfo { - xCoordinate?: number; - yCoordinate?: number; - height?: number; - width?: number; - datasource?: string; - xValue?: number; - yValue?: number; - dailySize?: number; -} - -interface StackedBarChartProps { - svgWidth: number; - svgHeight: number; - margin: BarChartMargin; - activeDataType?: string; - dataToRender: BarUnitData[]; - changeActiveDatasource: (e: string | null) => void; - formatTick: (e: number) => string; - xScale: AxisScale; - yScale: AxisScale; - barWidth: number; -} - -export const StackedBarChart = React.forwardRef(function StackedBarChart( - props: StackedBarChartProps, - ref, -) { - const { - activeDataType, - svgWidth, - svgHeight, - margin, - formatTick, - xScale, - yScale, - dataToRender, - changeActiveDatasource, - barWidth, - } = props; - const [hoverOn, setHoverOn] = useState(); - - const width = svgWidth - margin.left - margin.right; - const height = svgHeight - margin.top - margin.bottom; - - function renderBarChart() { - return ( - - setHoverOn(undefined)} - > - '') - .tickSizeOuter(0)} - /> - setHoverOn(e)} - hoverOn={hoverOn} - barWidth={barWidth} - /> - - formatTick(e))} - /> - {hoverOn && ( - { - setHoverOn(undefined); - changeActiveDatasource(hoverOn.datasource ?? null); - }} - > - - - )} - - - ); - } - - return ( -
- {hoverOn && ( -
-
Datasource: {hoverOn.datasource}
-
Time: {hoverOn.xValue}
-
- {`${ - activeDataType === 'countData' ? 'Daily total count:' : 'Daily total size:' - } ${formatTick(hoverOn.dailySize!)}`} -
-
- {`${activeDataType === 'countData' ? 'Count:' : 'Size:'} ${formatTick( - hoverOn.yValue!, - )}`} -
-
- )} - {renderBarChart()} -
- ); -}); diff --git a/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx b/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx index 74cb55682f30..a7fa38fd445c 100644 --- a/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx +++ b/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx @@ -25,7 +25,7 @@ import type { IngestionSpec } from '../../druid-models'; import { cleanSpec } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { Api } from '../../singletons'; -import { deepSet } from '../../utils'; +import { deepSet, getApiArray } from '../../utils'; import { Loader } from '../loader/loader'; import { ShowValue } from '../show-value/show-value'; @@ -49,11 +49,12 @@ export const SupervisorHistoryPanel = React.memo(function SupervisorHistoryPanel const [historyState] = useQueryManager({ initQuery: supervisorId, processQuery: async (supervisorId, cancelToken) => { - const resp = await Api.instance.get( - `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/history`, - { cancelToken }, - ); - return resp.data.map((vs: SupervisorHistoryEntry) => deepSet(vs, 'spec', cleanSpec(vs.spec))); + return ( + await getApiArray( + `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/history`, + cancelToken, + ) + ).map(vs => deepSet(vs, 'spec', cleanSpec(vs.spec))); }, }); diff --git a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx index aec706a50fc0..38c69e297e0c 100644 --- a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx +++ b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx @@ -28,7 +28,7 @@ export interface TableClickableCellProps { className?: string; onClick: MouseEventHandler; hoverIcon?: IconName; - title?: string; + tooltip?: string; disabled?: boolean; children?: ReactNode; } @@ -36,12 +36,13 @@ export interface TableClickableCellProps { export const TableClickableCell = React.memo(function TableClickableCell( props: TableClickableCellProps, ) { - const { className, onClick, hoverIcon, disabled, children, ...rest } = props; + const { className, onClick, hoverIcon, disabled, children, tooltip, ...rest } = props; return (
{children} diff --git a/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx b/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx index 631fa224aaf4..c03f0038cc7f 100644 --- a/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx +++ b/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx @@ -27,7 +27,7 @@ import { Deferred } from '../deferred/deferred'; import './table-filterable-cell.scss'; -const FILTER_MODES: FilterMode[] = ['=', '!=', '<=', '>=']; +const FILTER_MODES: FilterMode[] = ['=', '!=', '<', '>=']; const FILTER_MODES_NO_COMPARISONS: FilterMode[] = ['=', '!=']; export interface TableFilterableCellProps { diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 95336713dbb6..5dfbba77101a 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -182,12 +182,24 @@ export class ConsoleApplication extends React.PureComponent< changeTabWithFilter('datasources', [{ id: 'datasource', value: `=${datasource}` }]); }; - private readonly goToSegments = (datasource: string, onlyUnavailable = false) => { + private readonly goToSegments = ({ + start, + end, + datasource, + realtime, + }: { + start?: Date; + end?: Date; + datasource?: string; + realtime?: boolean; + }) => { changeTabWithFilter( 'segments', compact([ - { id: 'datasource', value: `=${datasource}` }, - onlyUnavailable ? { id: 'is_available', value: '=false' } : undefined, + start && { id: 'start', value: `>=${start.toISOString()}` }, + end && { id: 'end', value: `<${end.toISOString()}` }, + datasource && { id: 'datasource', value: `=${datasource}` }, + typeof realtime === 'boolean' ? { id: 'is_realtime', value: `=${realtime}` } : undefined, ]), ); }; diff --git a/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx b/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx index 9e19e043c713..8ee0ffdbfbc3 100644 --- a/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx +++ b/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx @@ -24,7 +24,7 @@ import { Loader, ShowValue } from '../../components'; import type { CompactionConfig } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { Api } from '../../singletons'; -import { formatInteger, formatPercent } from '../../utils'; +import { formatInteger, formatPercent, getApiArray } from '../../utils'; import { DiffDialog } from '../diff-dialog/diff-dialog'; import './compaction-history-dialog.scss'; @@ -65,11 +65,10 @@ export const CompactionHistoryDialog = React.memo(function CompactionHistoryDial initQuery: datasource, processQuery: async (datasource, cancelToken) => { try { - const resp = await Api.instance.get( + return await getApiArray( `/druid/coordinator/v1/config/compaction/${Api.encodePath(datasource)}/history?count=20`, - { cancelToken }, + cancelToken, ); - return resp.data; } catch (e) { if (e.response?.status === 404) return []; throw e; diff --git a/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx b/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx index ec964f5507ee..ab4fed8ae1ad 100644 --- a/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx @@ -27,7 +27,7 @@ import { COORDINATOR_DYNAMIC_CONFIG_FIELDS } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { getLink } from '../../links'; import { Api, AppToaster } from '../../singletons'; -import { getDruidErrorMessage } from '../../utils'; +import { getApiArray, getDruidErrorMessage } from '../../utils'; import { SnitchDialog } from '..'; import './coordinator-dynamic-config-dialog.scss'; @@ -47,10 +47,7 @@ export const CoordinatorDynamicConfigDialog = React.memo(function CoordinatorDyn const [historyRecordsState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - const historyResp = await Api.instance.get(`/druid/coordinator/v1/config/history?count=100`, { - cancelToken, - }); - return historyResp.data; + return await getApiArray(`/druid/coordinator/v1/config/history?count=100`, cancelToken); }, }); diff --git a/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx b/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx index 82187c866f44..3dfb2f7879d7 100644 --- a/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx +++ b/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { L } from '@druid-toolkit/query'; +import { L } from 'druid-query-toolkit'; import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx b/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx index 1ca92787c1cb..c48cbfa51989 100644 --- a/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx +++ b/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; -import { QueryRunner, T } from '@druid-toolkit/query'; +import type { QueryResult } from 'druid-query-toolkit'; +import { QueryRunner, T } from 'druid-query-toolkit'; import React from 'react'; import { Loader, RecordTablePane } from '../../../components'; diff --git a/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx b/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx index c36445d17731..122cc4566b85 100644 --- a/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx +++ b/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { N } from '@druid-toolkit/query'; +import { N } from 'druid-query-toolkit'; import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx b/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx index ba30118b0a32..5b1233c6384e 100644 --- a/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx @@ -27,7 +27,7 @@ import { OVERLORD_DYNAMIC_CONFIG_FIELDS } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { getLink } from '../../links'; import { Api, AppToaster } from '../../singletons'; -import { getDruidErrorMessage } from '../../utils'; +import { getApiArray, getDruidErrorMessage } from '../../utils'; import { SnitchDialog } from '..'; import './overlord-dynamic-config-dialog.scss'; @@ -47,10 +47,7 @@ export const OverlordDynamicConfigDialog = React.memo(function OverlordDynamicCo const [historyRecordsState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - const historyResp = await Api.instance.get(`/druid/indexer/v1/worker/history?count=100`, { - cancelToken, - }); - return historyResp.data; + return await getApiArray(`/druid/indexer/v1/worker/history?count=100`, cancelToken); }, }); diff --git a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap index 19c90c7de335..8bba057fac2d 100644 --- a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap +++ b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap @@ -188,7 +188,7 @@ exports[`RetentionDialog matches snapshot 1`] = ` style="transform: translateY(0); transition: none;" >
d.tier); } else if (capabilities.hasCoordinatorAccess()) { - const allServiceResp = await Api.instance.get('/druid/coordinator/v1/servers?simple', { - cancelToken, - }); - return filterMap(allServiceResp.data, (s: any) => - s.type === 'historical' ? s.tier : undefined, + return filterMap( + await getApiArray('/druid/coordinator/v1/servers?simple', cancelToken), + (s: any) => (s.type === 'historical' ? s.tier : undefined), ); } else { throw new Error(`must have sql or coordinator access`); @@ -84,11 +82,10 @@ ORDER BY 1`, const [historyQueryState] = useQueryManager({ initQuery: props.datasource, processQuery: async (datasource, cancelToken) => { - const historyResp = await Api.instance.get( + return await getApiArray( `/druid/coordinator/v1/rules/${Api.encodePath(datasource)}/history?count=200`, - { cancelToken }, + cancelToken, ); - return historyResp.data; }, }); diff --git a/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx b/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx index 4c2487ce0cc6..2925482d9901 100644 --- a/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx +++ b/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; -import { QueryRunner } from '@druid-toolkit/query'; +import type { QueryResult } from 'druid-query-toolkit'; +import { QueryRunner } from 'druid-query-toolkit'; import React from 'react'; import { Loader, RecordTablePane } from '../../../components'; diff --git a/web-console/src/dialogs/status-dialog/status-dialog.tsx b/web-console/src/dialogs/status-dialog/status-dialog.tsx index 311f3e056646..672fc40910f3 100644 --- a/web-console/src/dialogs/status-dialog/status-dialog.tsx +++ b/web-console/src/dialogs/status-dialog/status-dialog.tsx @@ -50,8 +50,7 @@ export const StatusDialog = React.memo(function StatusDialog(props: StatusDialog const [responseState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - const resp = await Api.instance.get(`/status`, { cancelToken }); - return resp.data; + return (await Api.instance.get(`/status`, { cancelToken })).data; }, }); diff --git a/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx b/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx index 009d83260604..d10fc00eb5ce 100644 --- a/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx +++ b/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx @@ -106,11 +106,12 @@ export const SupervisorResetOffsetsDialog = React.memo(function SupervisorResetO const [statusResp] = useQueryManager({ initQuery: supervisorId, processQuery: async (supervisorId, cancelToken) => { - const statusResp = await Api.instance.get( - `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/status`, - { cancelToken }, - ); - return statusResp.data; + return ( + await Api.instance.get( + `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/status`, + { cancelToken }, + ) + ).data; }, }); diff --git a/web-console/src/components/segment-timeline/stacked-bar-chart.scss b/web-console/src/druid-models/datasource/datasource.ts similarity index 64% rename from web-console/src/components/segment-timeline/stacked-bar-chart.scss rename to web-console/src/druid-models/datasource/datasource.ts index 26e5f5186b5f..313d711b35c2 100644 --- a/web-console/src/components/segment-timeline/stacked-bar-chart.scss +++ b/web-console/src/druid-models/datasource/datasource.ts @@ -16,35 +16,32 @@ * limitations under the License. */ -.stacked-bar-chart { - position: relative; - overflow: hidden; +import { hashJoaat } from '../../utils'; - .bar-chart-tooltip { - position: absolute; - left: 100px; - right: 0; +const COLORS = [ + '#1f77b4', + '#aec7e8', + '#ff7f0e', + '#ffbb78', + '#2ca02c', + '#98df8a', + '#d62728', + '#ff9896', + '#9467bd', + '#c5b0d5', + '#8c564b', + '#c49c94', + '#e377c2', + '#f7b6d2', + '#7f7f7f', + '#c7c7c7', + '#bcbd22', + '#dbdb8d', + '#17becf', + '#9edae5', +]; - div { - display: inline-block; - width: 230px; - } - } - - svg { - position: absolute; - - .hovered-bar { - fill: transparent; - stroke: #ffffff; - stroke-width: 1.5px; - } - - .gridline-x { - line { - stroke-dasharray: 5, 5; - opacity: 0.5; - } - } - } +export function getDatasourceColor(datasource: string) { + const hash = hashJoaat(datasource); + return COLORS[hash % COLORS.length]; } diff --git a/web-console/src/druid-models/execution/execution.ts b/web-console/src/druid-models/execution/execution.ts index f4a73eb97207..1237b49e62da 100644 --- a/web-console/src/druid-models/execution/execution.ts +++ b/web-console/src/druid-models/execution/execution.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { Column, QueryResult, SqlExpression, SqlQuery, SqlWithQuery } from '@druid-toolkit/query'; +import { Column, QueryResult, SqlExpression, SqlQuery, SqlWithQuery } from 'druid-query-toolkit'; import { maybeGetClusterCapacity } from '../../helpers'; import { diff --git a/web-console/src/druid-models/external-config/external-config.ts b/web-console/src/druid-models/external-config/external-config.ts index d6541747ce5a..6fa74eb038bd 100644 --- a/web-console/src/druid-models/external-config/external-config.ts +++ b/web-console/src/druid-models/external-config/external-config.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { SqlQuery } from '@druid-toolkit/query'; +import type { SqlQuery } from 'druid-query-toolkit'; import { C, F, @@ -28,7 +28,7 @@ import { SqlLiteral, SqlStar, SqlType, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import { nonEmptyArray } from '../../utils'; diff --git a/web-console/src/druid-models/index.ts b/web-console/src/druid-models/index.ts index dfeeeeaac837..e31eedeea738 100644 --- a/web-console/src/druid-models/index.ts +++ b/web-console/src/druid-models/index.ts @@ -21,6 +21,7 @@ export * from './compaction-config/compaction-config'; export * from './compaction-status/compaction-status'; export * from './coordinator-dynamic-config/coordinator-dynamic-config'; export * from './dart/dart-query-entry'; +export * from './datasource/datasource'; export * from './dimension-spec/dimension-spec'; export * from './druid-engine/druid-engine'; export * from './execution/execution'; @@ -32,10 +33,12 @@ export * from './ingest-query-pattern/ingest-query-pattern'; export * from './ingestion-spec/ingestion-spec'; export * from './input-format/input-format'; export * from './input-source/input-source'; +export * from './load-rule/load-rule'; export * from './lookup-spec/lookup-spec'; export * from './metric-spec/metric-spec'; export * from './overlord-dynamic-config/overlord-dynamic-config'; export * from './query-context/query-context'; +export * from './segment/segment'; export * from './stages/stages'; export * from './supervisor-status/supervisor-status'; export * from './task/task'; diff --git a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts index 996385fc08ff..cf108c2b3890 100644 --- a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts +++ b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane, SqlQuery } from '@druid-toolkit/query'; +import { sane, SqlQuery } from 'druid-query-toolkit'; import { fitIngestQueryPattern, ingestQueryPatternToQuery } from './ingest-query-pattern'; diff --git a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts index f0ce8a0eb288..d2009ad2e5ee 100644 --- a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts +++ b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts @@ -26,7 +26,7 @@ import { SqlTable, SqlWithPart, T, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import { filterMap, oneOf } from '../../utils'; import type { ExternalConfig } from '../external-config/external-config'; diff --git a/web-console/src/utils/load-rule.ts b/web-console/src/druid-models/load-rule/load-rule.ts similarity index 97% rename from web-console/src/utils/load-rule.ts rename to web-console/src/druid-models/load-rule/load-rule.ts index a32422bbb6a4..31b4d600585a 100644 --- a/web-console/src/utils/load-rule.ts +++ b/web-console/src/druid-models/load-rule/load-rule.ts @@ -18,7 +18,7 @@ import { sum } from 'd3-array'; -import { deepMove, deepSet } from './object-change'; +import { deepMove, deepSet } from '../../utils'; export type RuleType = | 'loadForever' @@ -41,6 +41,7 @@ export interface Rule { } export class RuleUtil { + static DEFAULT_RULES_KEY = '_default'; static TYPES: RuleType[] = [ 'loadForever', 'loadByInterval', diff --git a/web-console/src/druid-models/segment/segment.ts b/web-console/src/druid-models/segment/segment.ts new file mode 100644 index 000000000000..1bff0711fcdf --- /dev/null +++ b/web-console/src/druid-models/segment/segment.ts @@ -0,0 +1,50 @@ +/* + * 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 { Duration } from '../../utils'; + +export const START_OF_TIME_DATE = '-146136543-09-08T08:23:32.096Z'; +export const END_OF_TIME_DATE = '146140482-04-24T15:36:27.903Z'; + +export function computeSegmentTimeSpan(start: string, end: string): string { + if (start === START_OF_TIME_DATE && end === END_OF_TIME_DATE) { + return 'All'; + } + + const startDate = new Date(start); + if (isNaN(startDate.valueOf())) { + return 'Invalid start'; + } + + const endDate = new Date(end); + if (isNaN(endDate.valueOf())) { + return 'Invalid end'; + } + + return Duration.fromRange(startDate, endDate, 'Etc/UTC').getDescription(true); +} + +export interface ShardSpec { + type: string; + partitionNum?: number; + partitions?: number; + dimensions?: string[]; + partitionDimensions?: string[]; + start?: string[]; + end?: string[]; +} diff --git a/web-console/src/druid-models/task/task.ts b/web-console/src/druid-models/task/task.ts index 3445cc5fb45f..e1743c116897 100644 --- a/web-console/src/druid-models/task/task.ts +++ b/web-console/src/druid-models/task/task.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { C } from '@druid-toolkit/query'; +import { C } from 'druid-query-toolkit'; import type { Counters, StageDefinition } from '../stages/stages'; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts index 02bb3e399b02..8456b0d065d5 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; +import { sane } from 'druid-query-toolkit'; import { WorkbenchQuery } from './workbench-query'; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.ts b/web-console/src/druid-models/workbench-query/workbench-query.ts index da37127f1d2d..9b1ed42b516b 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.ts @@ -22,7 +22,7 @@ import type { SqlClusteredByClause, SqlExpression, SqlPartitionedByClause, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import { C, F, @@ -30,7 +30,7 @@ import { SqlOrderByClause, SqlOrderByExpression, SqlQuery, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import Hjson from 'hjson'; import * as JSONBig from 'json-bigint-native'; import { v4 as uuidv4 } from 'uuid'; diff --git a/web-console/src/entry.scss b/web-console/src/entry.scss index 8d368a4146b7..46ea9d298d0f 100644 --- a/web-console/src/entry.scss +++ b/web-console/src/entry.scss @@ -56,6 +56,7 @@ body { position: absolute; height: 100%; width: 100%; + z-index: 0; .console-application { position: absolute; diff --git a/web-console/src/entry.tsx b/web-console/src/entry.tsx index 42cfe08ad13c..55f7abfc59a2 100644 --- a/web-console/src/entry.tsx +++ b/web-console/src/entry.tsx @@ -20,7 +20,7 @@ import 'regenerator-runtime/runtime'; import './bootstrap/ace'; import { OverlaysProvider } from '@blueprintjs/core'; -import { QueryRunner } from '@druid-toolkit/query'; +import { QueryRunner } from 'druid-query-toolkit'; import { createRoot } from 'react-dom/client'; import { bootstrapJsonParse } from './bootstrap/json-parser'; diff --git a/web-console/src/helpers/capabilities.ts b/web-console/src/helpers/capabilities.ts index 013f9368c58c..3255ec93be73 100644 --- a/web-console/src/helpers/capabilities.ts +++ b/web-console/src/helpers/capabilities.ts @@ -211,6 +211,10 @@ export class Capabilities { }; } + public clone(): Capabilities { + return new Capabilities(this.valueOf()); + } + public getMode(): CapabilitiesMode { if (!this.hasSql()) return 'no-sql'; if (!this.hasCoordinatorAccess()) return 'no-proxy'; diff --git a/web-console/src/helpers/execution/general.ts b/web-console/src/helpers/execution/general.ts index a90c02a63061..0e78834172b1 100644 --- a/web-console/src/helpers/execution/general.ts +++ b/web-console/src/helpers/execution/general.ts @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; import type { CancelToken } from 'axios'; +import type { QueryResult } from 'druid-query-toolkit'; import type { Execution } from '../../druid-models'; import { IntermediateQueryState } from '../../utils'; diff --git a/web-console/src/helpers/execution/sql-task-execution.ts b/web-console/src/helpers/execution/sql-task-execution.ts index f4dd45a2cb91..68cf0763b425 100644 --- a/web-console/src/helpers/execution/sql-task-execution.ts +++ b/web-console/src/helpers/execution/sql-task-execution.ts @@ -16,8 +16,8 @@ * limitations under the License. */ -import { QueryResult } from '@druid-toolkit/query'; import type { AxiosResponse, CancelToken } from 'axios'; +import { QueryResult } from 'druid-query-toolkit'; import type { AsyncStatusResponse, MsqTaskPayloadResponse, QueryContext } from '../../druid-models'; import { Execution } from '../../druid-models'; diff --git a/web-console/src/helpers/spec-conversion.ts b/web-console/src/helpers/spec-conversion.ts index 00bc3d5f832c..9fc95d3b60f5 100644 --- a/web-console/src/helpers/spec-conversion.ts +++ b/web-console/src/helpers/spec-conversion.ts @@ -25,7 +25,7 @@ import { SqlExpression, SqlType, T, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import type { diff --git a/web-console/src/hooks/use-clock.ts b/web-console/src/hooks/use-clock.ts index 5d1264421957..a909a2c1f740 100644 --- a/web-console/src/hooks/use-clock.ts +++ b/web-console/src/hooks/use-clock.ts @@ -16,7 +16,9 @@ * limitations under the License. */ -import { useEffect, useState } from 'react'; +import { useState } from 'react'; + +import { useInterval } from './use-interval'; function getNowToSecond(): Date { const now = new Date(); @@ -24,18 +26,12 @@ function getNowToSecond(): Date { return now; } -export function useClock() { +export function useClock(updateInterval = 1000) { const [now, setNow] = useState(getNowToSecond); - useEffect(() => { - const checkInterval = setInterval(() => { - setNow(getNowToSecond()); - }, 1000); - - return () => { - clearInterval(checkInterval); - }; - }, []); + useInterval(() => { + setNow(getNowToSecond()); + }, updateInterval); return now; } diff --git a/web-console/src/react-table/react-table-utils.spec.ts b/web-console/src/react-table/react-table-utils.spec.ts index 0a1bbf3f9ad5..1dd245b49f70 100644 --- a/web-console/src/react-table/react-table-utils.spec.ts +++ b/web-console/src/react-table/react-table-utils.spec.ts @@ -75,5 +75,9 @@ describe('react-table-utils', () => { { id: 'x', value: '~y' }, { id: 'z', value: '=w&' }, ]); + expect(stringToTableFilters('x<3&y<=3')).toEqual([ + { id: 'x', value: '<3' }, + { id: 'y', value: '<=3' }, + ]); }); }); diff --git a/web-console/src/react-table/react-table-utils.ts b/web-console/src/react-table/react-table-utils.ts index ce9497a1209a..7732c90b231c 100644 --- a/web-console/src/react-table/react-table-utils.ts +++ b/web-console/src/react-table/react-table-utils.ts @@ -18,8 +18,7 @@ import type { IconName } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { SqlExpression } from '@druid-toolkit/query'; -import { C, F } from '@druid-toolkit/query'; +import { C, F, SqlExpression } from 'druid-query-toolkit'; import type { Filter } from 'react-table'; import { addOrUpdate, caseInsensitiveContains, filterMap } from '../utils'; @@ -32,9 +31,9 @@ export const STANDARD_TABLE_PAGE_SIZE_OPTIONS = [50, 100, 200]; export const SMALL_TABLE_PAGE_SIZE = 25; export const SMALL_TABLE_PAGE_SIZE_OPTIONS = [25, 50, 100]; -export type FilterMode = '~' | '=' | '!=' | '<=' | '>='; +export type FilterMode = '~' | '=' | '!=' | '<' | '<=' | '>' | '>='; -export const FILTER_MODES: FilterMode[] = ['~', '=', '!=', '<=', '>=']; +export const FILTER_MODES: FilterMode[] = ['~', '=', '!=', '<', '<=', '>', '>=']; export const FILTER_MODES_NO_COMPARISON: FilterMode[] = ['~', '=', '!=']; export function filterModeToIcon(mode: FilterMode): IconName { @@ -45,8 +44,12 @@ export function filterModeToIcon(mode: FilterMode): IconName { return IconNames.EQUALS; case '!=': return IconNames.NOT_EQUAL_TO; + case '<': + return IconNames.LESS_THAN; case '<=': return IconNames.LESS_THAN_OR_EQUAL_TO; + case '>': + return IconNames.GREATER_THAN; case '>=': return IconNames.GREATER_THAN_OR_EQUAL_TO; default: @@ -62,8 +65,12 @@ export function filterModeToTitle(mode: FilterMode): string { return 'Equals'; case '!=': return 'Not equals'; + case '<': + return 'Less than'; case '<=': return 'Less than or equal'; + case '>': + return 'Greater than'; case '>=': return 'Greater than or equal'; default: @@ -89,7 +96,7 @@ export function parseFilterModeAndNeedle( filter: Filter, loose = false, ): FilterModeAndNeedle | undefined { - const m = /^(~|=|!=|<=|>=)?(.*)$/.exec(String(filter.value)); + const m = /^(~|=|!=|<(?!=)|<=|>(?!=)|>=)?(.*)$/.exec(String(filter.value)); if (!m) return; if (!loose && !m[2]) return; const mode = (m[1] as FilterMode) || '~'; @@ -112,21 +119,28 @@ export function booleanCustomTableFilter(filter: Filter, value: unknown): boolea const modeAndNeedle = parseFilterModeAndNeedle(filter); if (!modeAndNeedle) return true; const { mode, needle } = modeAndNeedle; + const strValue = String(value); switch (mode) { case '=': - return String(value) === needle; + return strValue === needle; case '!=': - return String(value) !== needle; + return strValue !== needle; + + case '<': + return strValue < needle; case '<=': - return String(value) <= needle; + return strValue <= needle; + + case '>': + return strValue > needle; case '>=': - return String(value) >= needle; + return strValue >= needle; default: - return caseInsensitiveContains(String(value), needle); + return caseInsensitiveContains(strValue, needle); } } @@ -142,9 +156,15 @@ export function sqlQueryCustomTableFilter(filter: Filter): SqlExpression | undef case '!=': return column.unequal(needle); + case '<': + return column.lessThan(needle); + case '<=': return column.lessThanOrEqual(needle); + case '>': + return column.greaterThan(needle); + case '>=': return column.greaterThanOrEqual(needle); @@ -153,6 +173,10 @@ export function sqlQueryCustomTableFilter(filter: Filter): SqlExpression | undef } } +export function sqlQueryCustomTableFilters(filters: Filter[]): SqlExpression { + return SqlExpression.and(...filterMap(filters, sqlQueryCustomTableFilter)); +} + export function tableFiltersToString(tableFilters: Filter[]): string { return tableFilters .map(({ id, value }) => `${id}${value.replace(/[&%]/g, encodeURIComponent)}`) @@ -161,9 +185,11 @@ export function tableFiltersToString(tableFilters: Filter[]): string { export function stringToTableFilters(str: string | undefined): Filter[] { if (!str) return []; - // '~' | '=' | '!=' | '<=' | '>='; + // '~' | '=' | '!=' | '<' | '<=' | '>' | '>='; return filterMap(str.split('&'), clause => { - const m = /^(\w+)((?:~|=|!=|<=|>=).*)$/.exec(clause.replace(/%2[56]/g, decodeURIComponent)); + const m = /^(\w+)((?:~|=|!=|<(?!=)|<=|>(?!=)|>=).*)$/.exec( + clause.replace(/%2[56]/g, decodeURIComponent), + ); if (!m) return; return { id: m[1], value: m[2] }; }); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts new file mode 100755 index 000000000000..5ba63b04468f --- /dev/null +++ b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts @@ -0,0 +1,169 @@ +/* + * 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 { shifters } from './date-floor-shift-ceil'; + +function pairwise(array: T[], callback: (t1: T, t2: T) => void) { + for (let i = 0; i < array.length - 1; i++) { + callback(array[i], array[i + 1]); + } +} + +describe('floor, shift, ceil (UTC)', () => { + const tz = 'Etc/UTC'; + + it('moves seconds', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-04T00:00:03Z'), + new Date('2012-11-04T00:00:06Z'), + new Date('2012-11-04T00:00:09Z'), + new Date('2012-11-04T00:00:12Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.second.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('rounds minutes', () => { + expect(shifters.minute.round(new Date('2012-11-04T00:29:00Z'), 15, tz)).toEqual( + new Date('2012-11-04T00:15:00Z'), + ); + + expect(shifters.minute.round(new Date('2012-11-04T00:29:00Z'), 4, tz)).toEqual( + new Date('2012-11-04T00:28:00Z'), + ); + }); + + it('moves minutes', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-04T00:03:00Z'), + new Date('2012-11-04T00:06:00Z'), + new Date('2012-11-04T00:09:00Z'), + new Date('2012-11-04T00:12:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.minute.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('floors hour correctly', () => { + expect(shifters.hour.floor(new Date('2012-11-04T00:30:00Z'), tz)).toEqual( + new Date('2012-11-04T00:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00Z'), tz)).toEqual( + new Date('2012-11-04T01:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00Z'), tz)).toEqual( + new Date('2012-11-04T01:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T02:30:00Z'), tz)).toEqual( + new Date('2012-11-04T02:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T03:30:00Z'), tz)).toEqual( + new Date('2012-11-04T03:00:00Z'), + ); + }); + + it('moves hour', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-04T01:00:00Z'), + new Date('2012-11-04T02:00:00Z'), + new Date('2012-11-04T03:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('moves day', () => { + const dates: Date[] = [ + new Date('2012-11-03T00:00:00Z'), + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-05T00:00:00Z'), + new Date('2012-11-06T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.day.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('ceils day', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-12T00:00:00.000Z'); + expect(shifters.day.ceil(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-08T00:00:00.000Z'); + d2 = new Date('2014-12-08T00:00:00.000Z'); + expect(shifters.day.ceil(d1, tz)).toEqual(d2); + }); + + it('moves week', () => { + const dates: Date[] = [ + new Date('2012-10-29T00:00:00Z'), + new Date('2012-11-05T00:00:00Z'), + new Date('2012-11-12T00:00:00Z'), + new Date('2012-11-19T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.week.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('floors week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-08T00:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-01T00:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + }); + + it('ceils week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-15T00:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-08T00:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + }); + + it('moves month', () => { + const dates: Date[] = [ + new Date('2012-11-01T00:00:00Z'), + new Date('2012-12-01T00:00:00Z'), + new Date('2013-01-01T00:00:00Z'), + new Date('2013-02-01T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.month.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts month on the 31st', () => { + const d1 = new Date('2016-03-31T00:00:00.000Z'); + const d2 = new Date('2016-05-01T00:00:00.000Z'); + expect(shifters.month.shift(d1, tz, 1)).toEqual(d2); + }); + + it('moves year', () => { + const dates: Date[] = [ + new Date('2010-01-01T00:00:00Z'), + new Date('2011-01-01T00:00:00Z'), + new Date('2012-01-01T00:00:00Z'), + new Date('2013-01-01T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.year.shift(d1, tz, 1)).toEqual(d2)); + }); +}); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts new file mode 100755 index 000000000000..1612c9ce5792 --- /dev/null +++ b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts @@ -0,0 +1,181 @@ +/* + * 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 { shifters } from './date-floor-shift-ceil'; + +function pairwise(array: T[], callback: (t1: T, t2: T) => void) { + for (let i = 0; i < array.length - 1; i++) { + callback(array[i], array[i + 1]); + } +} + +describe('floor/shift/ceil', () => { + const tz = 'America/Los_Angeles'; + + it('shifts seconds', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-04T00:00:03-07:00'), + new Date('2012-11-04T00:00:06-07:00'), + new Date('2012-11-04T00:00:09-07:00'), + new Date('2012-11-04T00:00:12-07:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.second.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('shifts minutes', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-04T00:03:00-07:00'), + new Date('2012-11-04T00:06:00-07:00'), + new Date('2012-11-04T00:09:00-07:00'), + new Date('2012-11-04T00:12:00-07:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.minute.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('floors hour correctly', () => { + expect(shifters.hour.floor(new Date('2012-11-04T00:30:00-07:00'), tz)).toEqual( + new Date('2012-11-04T00:00:00-07:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00-07:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00-08:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T02:30:00-08:00'), tz)).toEqual( + new Date('2012-11-04T02:00:00-08:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T03:30:00-08:00'), tz)).toEqual( + new Date('2012-11-04T03:00:00-08:00'), + ); + }); + + it('shifting 24 hours over DST is not the same as shifting a day', () => { + const start = new Date('2012-11-04T07:00:00Z'); + + const shift1Day = shifters.day.shift(start, tz, 1); + const shift24Hours = shifters.hour.shift(start, tz, 24); + + expect(shift1Day).toEqual(new Date('2012-11-05T08:00:00Z')); + expect(shift24Hours).toEqual(new Date('2012-11-05T07:00:00Z')); + }); + + it('shifts hour over DST 1', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-04T08:00:00Z'), + new Date('2012-11-04T09:00:00Z'), + new Date('2012-11-04T10:00:00Z'), + new Date('2012-11-04T11:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('floors hour over DST 1', () => { + expect(shifters.hour.floor(new Date('2012-11-04T00:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T00:00:00-07:00'), + ); + expect(shifters.hour.floor(new Date('2012-11-04T01:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + expect(shifters.hour.floor(new Date('2012-11-04T02:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + expect(shifters.hour.floor(new Date('2012-11-04T03:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T03:00:00-07:00'), + ); + }); + + it('shifts hour over DST 2', () => { + // "2018-03-11T09:00:00Z" + const dates: Date[] = [ + new Date('2018-03-11T01:00:00-07:00'), + new Date('2018-03-11T09:00:00Z'), + new Date('2018-03-11T10:00:00Z'), + new Date('2018-03-11T11:00:00Z'), + new Date('2018-03-11T12:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts day over DST', () => { + const dates: Date[] = [ + new Date('2012-11-03T00:00:00-07:00'), + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-05T00:00:00-08:00'), + new Date('2012-11-06T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.day.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts week over DST', () => { + const dates: Date[] = [ + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-11-05T00:00:00-08:00'), + new Date('2012-11-12T00:00:00-08:00'), + new Date('2012-11-19T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.week.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('floors week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-08T08:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-01T08:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + }); + + it('ceils week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-15T08:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-08T08:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + }); + + it('shifts month over DST', () => { + const dates: Date[] = [ + new Date('2012-11-01T00:00:00-07:00'), + new Date('2012-12-01T00:00:00-08:00'), + new Date('2013-01-01T00:00:00-08:00'), + new Date('2013-02-01T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.month.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts year', () => { + const dates: Date[] = [ + new Date('2010-01-01T00:00:00-08:00'), + new Date('2011-01-01T00:00:00-08:00'), + new Date('2012-01-01T00:00:00-08:00'), + new Date('2013-01-01T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.year.shift(d1, tz, 1)).toEqual(d2)); + }); +}); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts new file mode 100755 index 000000000000..3306b05267d9 --- /dev/null +++ b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts @@ -0,0 +1,296 @@ +/* + * 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 { fromDate, startOfWeek } from '@internationalized/date'; + +export type AlignFn = (dt: Date, tz: string) => Date; + +export type ShiftFn = (dt: Date, tz: string, step: number) => Date; + +export type RoundFn = (dt: Date, roundTo: number, tz: string) => Date; + +export interface TimeShifterNoCeil { + canonicalLength: number; + siblings?: number; + floor: AlignFn; + round: RoundFn; + shift: ShiftFn; +} + +export interface TimeShifter extends TimeShifterNoCeil { + ceil: AlignFn; +} + +function isUTC(tz: string): boolean { + return tz === 'Etc/UTC'; +} + +function adjustDay(day: number): number { + return (day + 6) % 7; +} + +function floorTo(n: number, roundTo: number): number { + return Math.floor(n / roundTo) * roundTo; +} + +function timeShifterFiller(tm: TimeShifterNoCeil): TimeShifter { + const { floor, shift } = tm; + return { + ...tm, + ceil: (dt: Date, tz: string) => { + const floored = floor(dt, tz); + if (floored.valueOf() === dt.valueOf()) return dt; // Just like ceil(3) is 3 and not 4 + return shift(floored, tz, 1); + }, + }; +} + +export const second = timeShifterFiller({ + canonicalLength: 1000, + siblings: 60, + floor: (dt, _tz) => { + // Seconds do not actually need a timezone because all timezones align on seconds... for now... + dt = new Date(dt.valueOf()); + dt.setUTCMilliseconds(0); + return dt; + }, + round: (dt, roundTo, _tz) => { + const cur = dt.getUTCSeconds(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCSeconds(adj); + return dt; + }, + shift: (dt, _tz, step) => { + dt = new Date(dt.valueOf()); + dt.setUTCSeconds(dt.getUTCSeconds() + step); + return dt; + }, +}); + +export const minute = timeShifterFiller({ + canonicalLength: 60000, + siblings: 60, + floor: (dt, _tz) => { + // Minutes do not actually need a timezone because all timezones align on minutes... for now... + dt = new Date(dt.valueOf()); + dt.setUTCSeconds(0, 0); + return dt; + }, + round: (dt, roundTo, _tz) => { + const cur = dt.getUTCMinutes(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCMinutes(adj); + return dt; + }, + shift: (dt, _tz, step) => { + dt = new Date(dt.valueOf()); + dt.setUTCMinutes(dt.getUTCMinutes() + step); + return dt; + }, +}); + +// Movement by hour is tz independent because in every timezone an hour is 60 min +function hourMove(dt: Date, _tz: string, step: number) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(dt.getUTCHours() + step); + return dt; +} + +export const hour = timeShifterFiller({ + canonicalLength: 3600000, + siblings: 24, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCMinutes(0, 0, 0); + return dt; + } else { + return fromDate(dt, tz).set({ second: 0, minute: 0, millisecond: 0 }).toDate(); + } + }, + round: (dt, roundTo, tz) => { + if (isUTC(tz)) { + const cur = dt.getUTCHours(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCHours(adj); + } else { + const cur = fromDate(dt, tz).hour; + const adj = floorTo(cur, roundTo); + if (cur !== adj) return hourMove(dt, tz, adj - cur); + } + return dt; + }, + shift: hourMove, +}); + +export const day = timeShifterFiller({ + canonicalLength: 24 * 3600000, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + return dt; + } else { + return fromDate(dt, tz).set({ hour: 0, second: 0, minute: 0, millisecond: 0 }).toDate(); + } + }, + shift: (dt, tz, step) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCDate(dt.getUTCDate() + step); + return dt; + } else { + return fromDate(dt, tz).add({ days: step }).toDate(); + } + }, + round: () => { + throw new Error('missing day round'); + }, +}); + +export const week = timeShifterFiller({ + canonicalLength: 7 * 24 * 3600000, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + dt.setUTCDate(dt.getUTCDate() - adjustDay(dt.getUTCDay())); + } else { + const zd = fromDate(dt, tz); + return startOfWeek( + zd.set({ hour: 0, second: 0, minute: 0, millisecond: 0 }), + 'fr-FR', // We want the week to start on Monday + ).toDate(); + } + return dt; + }, + shift: (dt, tz, step) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCDate(dt.getUTCDate() + step * 7); + return dt; + } else { + return fromDate(dt, tz).add({ weeks: step }).toDate(); + } + }, + round: () => { + throw new Error('missing week round'); + }, +}); + +function monthShift(dt: Date, tz: string, step: number) { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCMonth(dt.getUTCMonth() + step); + return dt; + } else { + return fromDate(dt, tz).add({ months: step }).toDate(); + } +} + +export const month = timeShifterFiller({ + canonicalLength: 30 * 24 * 3600000, + siblings: 12, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + dt.setUTCDate(1); + return dt; + } else { + return fromDate(dt, tz) + .set({ day: 1, hour: 0, second: 0, minute: 0, millisecond: 0 }) + .toDate(); + } + }, + round: (dt, roundTo, tz) => { + if (isUTC(tz)) { + const cur = dt.getUTCMonth(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCMonth(adj); + } else { + const cur = fromDate(dt, tz).month - 1; // Needs to be zero indexed + const adj = floorTo(cur, roundTo); + if (cur !== adj) return monthShift(dt, tz, adj - cur); + } + return dt; + }, + shift: monthShift, +}); + +function yearShift(dt: Date, tz: string, step: number) { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCFullYear(dt.getUTCFullYear() + step); + return dt; + } else { + return fromDate(dt, tz).add({ years: step }).toDate(); + } +} + +export const year = timeShifterFiller({ + canonicalLength: 365 * 24 * 3600000, + siblings: 1000, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + dt.setUTCMonth(0, 1); + return dt; + } else { + return fromDate(dt, tz) + .set({ month: 1, day: 1, hour: 0, second: 0, minute: 0, millisecond: 0 }) + .toDate(); + } + }, + round: (dt, roundTo, tz) => { + if (isUTC(tz)) { + const cur = dt.getUTCFullYear(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCFullYear(adj); + } else { + const cur = fromDate(dt, tz).year; + const adj = floorTo(cur, roundTo); + if (cur !== adj) return yearShift(dt, tz, adj - cur); + } + return dt; + }, + shift: yearShift, +}); + +export interface Shifters { + second: TimeShifter; + minute: TimeShifter; + hour: TimeShifter; + day: TimeShifter; + week: TimeShifter; + month: TimeShifter; + year: TimeShifter; + + [key: string]: TimeShifter; +} + +export const shifters: Shifters = { + second, + minute, + hour, + day, + week, + month, + year, +}; diff --git a/web-console/src/utils/date.spec.ts b/web-console/src/utils/date.spec.ts index 843c144244ef..b219ee17af06 100644 --- a/web-console/src/utils/date.spec.ts +++ b/web-console/src/utils/date.spec.ts @@ -17,7 +17,6 @@ */ import { - ceilToUtcDay, dateToIsoDateString, intervalToLocalDateRange, localDateRangeToInterval, @@ -60,12 +59,4 @@ describe('date', () => { expect(localDateRangeToInterval(intervalToLocalDateRange(interval))).toEqual(interval); }); }); - - describe('ceilToUtcDay', () => { - it('works', () => { - expect(ceilToUtcDay(new Date('2021-02-03T12:03:02.001Z'))).toEqual( - new Date('2021-02-04T00:00:00Z'), - ); - }); - }); }); diff --git a/web-console/src/utils/date.ts b/web-console/src/utils/date.ts index 1d5aa4f61e95..08f1d1df43ba 100644 --- a/web-console/src/utils/date.ts +++ b/web-console/src/utils/date.ts @@ -99,9 +99,10 @@ export function localDateRangeToInterval(localRange: DateRange): string { }`; } -export function ceilToUtcDay(date: Date): Date { - date = new Date(date.valueOf()); - date.setUTCHours(0, 0, 0, 0); - date.setUTCDate(date.getUTCDate() + 1); - return date; +export function maxDate(a: Date, b: Date): Date { + return a > b ? a : b; +} + +export function minDate(a: Date, b: Date): Date { + return a < b ? a : b; } diff --git a/web-console/src/utils/download.ts b/web-console/src/utils/download.ts index 4fb3342e523c..40b0d95e8b91 100644 --- a/web-console/src/utils/download.ts +++ b/web-console/src/utils/download.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; +import type { QueryResult } from 'druid-query-toolkit'; import FileSaver from 'file-saver'; import * as JSONBig from 'json-bigint-native'; diff --git a/web-console/src/utils/druid-query.spec.ts b/web-console/src/utils/druid-query.spec.ts index ee867ff47ea5..a940b8ac7bb6 100644 --- a/web-console/src/utils/druid-query.spec.ts +++ b/web-console/src/utils/druid-query.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; +import { sane } from 'druid-query-toolkit'; import { DruidError, getDruidErrorMessage } from './druid-query'; diff --git a/web-console/src/utils/druid-query.ts b/web-console/src/utils/druid-query.ts index 8102db89ca3a..ee1ba1ef4a8c 100644 --- a/web-console/src/utils/druid-query.ts +++ b/web-console/src/utils/druid-query.ts @@ -16,9 +16,9 @@ * limitations under the License. */ -import { C } from '@druid-toolkit/query'; import type { AxiosResponse, CancelToken } from 'axios'; import axios from 'axios'; +import { C } from 'druid-query-toolkit'; import { Api } from '../singletons'; @@ -358,6 +358,12 @@ export async function queryDruidSqlDart( return sqlResultResp.data; } +export async function getApiArray(url: string, cancelToken?: CancelToken): Promise { + const result = (await Api.instance.get(url, { cancelToken })).data; + if (!Array.isArray(result)) throw new Error('unexpected result'); + return result; +} + export interface QueryExplanation { query: any; signature: { name: string; type: string }[]; diff --git a/web-console/src/utils/duration/duration.spec.ts b/web-console/src/utils/duration/duration.spec.ts new file mode 100755 index 000000000000..8b20ac0a6b2a --- /dev/null +++ b/web-console/src/utils/duration/duration.spec.ts @@ -0,0 +1,505 @@ +/* + * 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 { Duration } from './duration'; + +describe('Duration', () => { + const TZ_LA = 'America/Los_Angeles'; + const TZ_JUNEAU = 'America/Juneau'; + + describe('errors', () => { + it('throws error if invalid duration', () => { + expect(() => new Duration('')).toThrow("Can not parse duration ''"); + + expect(() => new Duration('P00')).toThrow("Can not parse duration 'P00'"); + + expect(() => new Duration('P')).toThrow('Duration can not be empty'); + + expect(() => new Duration('P0YT0H')).toThrow('Duration can not be empty'); + + expect(() => new Duration('P0W').shift(new Date(), TZ_LA)).toThrow( + 'Duration can not have empty weeks', + ); + + expect(() => new Duration('P0Y0MT0H0M0S').shift(new Date(), TZ_LA)).toThrow( + 'Duration can not be empty', + ); + }); + + it('throws error if fromJS is not given a string', () => { + expect(() => new Duration(new Date() as any)).toThrow('Duration can not be empty'); + }); + }); + + describe('#toString', () => { + it('gives back the correct string', () => { + let durationStr: string; + + durationStr = 'P3Y'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + + durationStr = 'P2W'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + }); + + it('eliminates 0', () => { + expect(new Duration('P0DT15H').toString()).toEqual('PT15H'); + }); + }); + + describe('fromCanonicalLength', () => { + it('handles zero', () => { + expect(() => { + Duration.fromCanonicalLength(0); + }).toThrow('length must be positive'); + }); + + it('works 1', () => { + expect(Duration.fromCanonicalLength(86400000).toString()).toEqual('P1D'); + }); + + it('works 2', () => { + const len = + new Date('2018-03-01T00:00:00Z').valueOf() - new Date('2016-02-22T00:00:00Z').valueOf(); + expect(Duration.fromCanonicalLength(len).toString()).toEqual('P2Y8D'); + }); + + it('works 3', () => { + const len = + new Date('2018-09-15T00:00:00Z').valueOf() - new Date('2018-09-04T00:00:00Z').valueOf(); + expect(Duration.fromCanonicalLength(len).toString()).toEqual('P11D'); + }); + + it('works with months', () => { + expect(Duration.fromCanonicalLength(2592000000).toString()).toEqual('P1M'); + expect(Duration.fromCanonicalLength(2678400000).toString()).toEqual('P1M1D'); + }); + + it('works without months', () => { + expect(Duration.fromCanonicalLength(2592000000, true).toString()).toEqual('P30D'); + expect(Duration.fromCanonicalLength(2678400000, true).toString()).toEqual('P31D'); + }); + }); + + describe('construct from span', () => { + it('parses days over DST', () => { + expect( + Duration.fromRange( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-11-05T00:00:00-08:00'), + TZ_LA, + ).toString(), + ).toEqual('P7D'); + + expect( + Duration.fromRange( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-11-12T00:00:00-08:00'), + TZ_LA, + ).toString(), + ).toEqual('P14D'); + }); + + it('parses complex case', () => { + expect( + Duration.fromRange( + new Date('2012-10-29T00:00:00-07:00'), + new Date(new Date('2012-11-05T00:00:00-08:00').valueOf() - 1000), + TZ_LA, + ).toString(), + ).toEqual('P6DT24H59M59S'); + + expect( + Duration.fromRange( + new Date('2012-01-01T00:00:00-08:00'), + new Date('2013-03-04T04:05:06-08:00'), + TZ_LA, + ).toString(), + ).toEqual('P1Y2M3DT4H5M6S'); + }); + }); + + describe('#isFloorable', () => { + const floorable = 'P1Y P5Y P10Y P100Y P1M P2M P3M P4M P1D'.split(' '); + for (const v of floorable) { + it(`works on floorable ${v}`, () => { + expect(new Duration(v).isFloorable()).toEqual(true); + }); + } + + const unfloorable = 'P1Y1M P5M P2D P3D'.split(' '); + for (const v of unfloorable) { + it(`works on not floorable ${v}`, () => { + expect(new Duration(v).isFloorable()).toEqual(false); + }); + } + }); + + describe('#floor', () => { + it('throws error if complex duration', () => { + expect(() => new Duration('P1Y2D').floor(new Date(), TZ_LA)).toThrow( + 'Can not floor on a complex duration', + ); + + expect(() => new Duration('P3DT15H').floor(new Date(), TZ_LA)).toThrow( + 'Can not floor on a complex duration', + ); + + expect(() => new Duration('PT5H').floor(new Date(), TZ_LA)).toThrow( + 'Can not floor on a hour duration that does not divide into 24', + ); + }); + + it('works for year', () => { + const p1y = new Duration('P1Y'); + expect(p1y.floor(new Date('2013-09-29T01:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-01-01T00:00:00.000-08:00'), + ); + }); + + it('works for PT2M', () => { + const pt2h = new Duration('PT2M'); + expect(pt2h.floor(new Date('2013-09-29T03:03:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-29T03:02:00.000-07:00'), + ); + }); + + it('works for P2H', () => { + const pt2h = new Duration('PT2H'); + expect(pt2h.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-29T02:00:00.000-07:00'), + ); + }); + + it('works for PT12H', () => { + const pt12h = new Duration('PT12H'); + expect(pt12h.floor(new Date('2015-09-12T13:05:00-08:00'), TZ_JUNEAU)).toEqual( + new Date('2015-09-12T12:00:00-08:00'), + ); + }); + + it('works for P1W', () => { + const p1w = new Duration('P1W'); + + expect(p1w.floor(new Date('2013-09-29T01:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-23T07:00:00.000Z'), + ); + + expect(p1w.floor(new Date('2013-10-03T01:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-30T00:00:00.000-07:00'), + ); + }); + + it('works for P3M', () => { + const p3m = new Duration('P3M'); + expect(p3m.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-07-01T00:00:00.000-07:00'), + ); + + expect(p3m.floor(new Date('2013-02-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-01-01T00:00:00.000-08:00'), + ); + }); + + it('works for P4Y', () => { + const p4y = new Duration('P4Y'); + expect(p4y.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2012-01-01T00:00:00.000-08:00'), + ); + }); + }); + + describe('#shift', () => { + it('works for weeks', () => { + let p1w = new Duration('P1W'); + expect(p1w.shift(new Date('2012-10-29T00:00:00-07:00'), TZ_LA)).toEqual( + new Date('2012-11-05T00:00:00-08:00'), + ); + + p1w = new Duration('P1W'); + expect(p1w.shift(new Date('2012-10-29T00:00:00-07:00'), TZ_LA, 2)).toEqual( + new Date('2012-11-12T00:00:00-08:00'), + ); + + const p2w = new Duration('P2W'); + expect(p2w.shift(new Date('2012-10-29T05:16:17-07:00'), TZ_LA)).toEqual( + new Date('2012-11-12T05:16:17-08:00'), + ); + }); + + it('works for general complex case', () => { + const pComplex = new Duration('P1Y2M3DT4H5M6S'); + expect(pComplex.shift(new Date('2012-01-01T00:00:00-08:00'), TZ_LA)).toEqual( + new Date('2013-03-04T04:05:06-08:00'), + ); + }); + }); + + describe('#materialize', () => { + it('works for weeks', () => { + const p1w = new Duration('P1W'); + + expect( + p1w.materialize( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-12-01T00:00:00-08:00'), + TZ_LA, + ), + ).toEqual([ + new Date('2012-10-29T07:00:00.000Z'), + new Date('2012-11-05T08:00:00.000Z'), + new Date('2012-11-12T08:00:00.000Z'), + new Date('2012-11-19T08:00:00.000Z'), + new Date('2012-11-26T08:00:00.000Z'), + ]); + + expect( + p1w.materialize( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-12-01T00:00:00-08:00'), + TZ_LA, + 2, + ), + ).toEqual([ + new Date('2012-10-29T07:00:00.000Z'), + new Date('2012-11-12T08:00:00.000Z'), + new Date('2012-11-26T08:00:00.000Z'), + ]); + }); + }); + + describe('#isAligned', () => { + it('works for weeks', () => { + const p1w = new Duration('P1W'); + expect(p1w.isAligned(new Date('2012-10-29T00:00:00-07:00'), TZ_LA)).toEqual(true); + expect(p1w.isAligned(new Date('2012-10-29T00:00:00-07:00'), 'Etc/UTC')).toEqual(false); + }); + }); + + describe('#dividesBy', () => { + const divisible = 'P5Y/P1Y P1D/P1D P1M/P1D P1W/P1D P1D/PT6H PT3H/PT1H'.split(' '); + for (const v of divisible) { + it(`works for ${v} (true)`, () => { + const p = v.split('/'); + expect(new Duration(p[0]).dividesBy(new Duration(p[1]))).toEqual(true); + }); + } + + const undivisible = 'P1D/P1M PT5H/PT1H'.split(' '); + for (const v of undivisible) { + it(`works for ${v} (false)`, () => { + const p = v.split('/'); + expect(new Duration(p[0]).dividesBy(new Duration(p[1]))).toEqual(false); + }); + } + }); + + describe('#getCanonicalLength', () => { + it('gives back the correct canonical length', () => { + let durationStr: string; + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(94608000000); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(1209600000); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(18000000); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(313200000); + }); + }); + + describe('#add()', () => { + it('works with a simple duration', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('P1D'); + + expect(d1.add(d2).toString()).toEqual('P2D'); + }); + + it('works with heterogeneous spans', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('P1Y'); + + expect(d1.add(d2).toString()).toEqual('P1Y1D'); + }); + + it('works with weeks', () => { + let d1 = new Duration('P1W'); + let d2 = new Duration('P2W'); + expect(d1.add(d2).toString()).toEqual('P3W'); + + d1 = new Duration('P6D'); + d2 = new Duration('P1D'); + expect(d1.add(d2).toString()).toEqual('P1W'); + }); + }); + + describe('#subtract()', () => { + it('works with a simple duration', () => { + const d1 = new Duration('P1DT2H'); + const d2 = new Duration('PT1H'); + + expect(d1.subtract(d2).toString()).toEqual('P1DT1H'); + }); + + it('works with a less simple duration', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('PT1H'); + + expect(d1.subtract(d2).toString()).toEqual('PT23H'); + }); + + it('works with weeks', () => { + const d1 = new Duration('P1W'); + const d2 = new Duration('P1D'); + + expect(d1.subtract(d2).toString()).toEqual('P6D'); + }); + + it('throws an error if result is going to be negative', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('P2D'); + + expect(() => d1.subtract(d2)).toThrow(); + }); + }); + + describe('#multiply()', () => { + it('works with a simple duration', () => { + const d = new Duration('P1D'); + expect(d.multiply(5).toString()).toEqual('P5D'); + }); + + it('works with a less simple duration', () => { + const d = new Duration('P1DT2H'); + expect(d.multiply(2).toString()).toEqual('P2DT4H'); + }); + + it('works with weeks', () => { + const d = new Duration('P1W'); + expect(d.multiply(5).toString()).toEqual('P5W'); + }); + + it('throws an error if result is going to be negative', () => { + const d = new Duration('P1D'); + expect(() => d.multiply(-1)).toThrow('Multiplier must be positive non-zero'); + }); + + it('gets description properly', () => { + const d = new Duration('P2D'); + expect(d.multiply(2).getDescription(true)).toEqual('4 Days'); + }); + }); + + describe('#getDescription()', () => { + it('gives back the correct description', () => { + let durationStr: string; + + durationStr = 'P1D'; + expect(new Duration(durationStr).getDescription()).toEqual('day'); + + durationStr = 'P1DT2H'; + expect(new Duration(durationStr).getDescription()).toEqual('1 day, 2 hours'); + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getDescription()).toEqual('3 years'); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getDescription()).toEqual('2 weeks'); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getDescription()).toEqual('5 hours'); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getDescription()).toEqual('3 days, 15 hours'); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getDescription(true)).toEqual('3 Days, 15 Hours'); + }); + }); + + describe('#getSingleSpan()', () => { + it('gives back the correct span', () => { + let durationStr: string; + + durationStr = 'P1D'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('day'); + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('year'); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('week'); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('hour'); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpan()).toBeUndefined(); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpan()).toBeUndefined(); + }); + }); + + describe('#getSingleSpanValue()', () => { + it('gives back the correct span value', () => { + let durationStr: string; + + durationStr = 'P1D'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(1); + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(3); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(2); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(5); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpanValue()).toBeUndefined(); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpanValue()).toBeUndefined(); + }); + }); + + describe('#limitToDays', () => { + it('works', () => { + expect(new Duration('P6D').limitToDays().toString()).toEqual('P6D'); + + expect(new Duration('P1M').limitToDays().toString()).toEqual('P30D'); + + expect(new Duration('P1Y').limitToDays().toString()).toEqual('P365D'); + + expect(new Duration('P1Y2M').limitToDays().toString()).toEqual('P425D'); + }); + }); +}); diff --git a/web-console/src/utils/duration/duration.ts b/web-console/src/utils/duration/duration.ts new file mode 100755 index 000000000000..bc7236821f12 --- /dev/null +++ b/web-console/src/utils/duration/duration.ts @@ -0,0 +1,388 @@ +/* + * 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 { second, shifters } from '../date-floor-shift-ceil/date-floor-shift-ceil'; +import { capitalizeFirst, pluralIfNeeded } from '../general'; + +export const TZ_UTC = 'Etc/UTC'; + +export type DurationSpan = 'year' | 'month' | 'week' | 'day' | 'hour' | 'minute' | 'second'; + +const SPANS_WITH_WEEK: DurationSpan[] = [ + 'year', + 'month', + 'week', + 'day', + 'hour', + 'minute', + 'second', +]; +const SPANS_WITHOUT_WEEK: DurationSpan[] = ['year', 'month', 'day', 'hour', 'minute', 'second']; +const SPANS_WITHOUT_WEEK_OR_MONTH: DurationSpan[] = ['year', 'day', 'hour', 'minute', 'second']; +const SPANS_UP_TO_DAY: DurationSpan[] = ['day', 'hour', 'minute', 'second']; + +export type DurationValue = Partial>; + +const periodWeekRegExp = /^P(\d+)W$/; +const periodRegExp = /^P(?:(\d+)Y)?(?:(\d+)M)?(?:(\d+)D)?(?:T(?:(\d+)H)?(?:(\d+)M)?(?:(\d+)S)?)?$/; +// P (year ) (month ) (day ) T(hour ) (minute ) (second ) + +function getSpansFromString(durationStr: string): DurationValue { + const spans: DurationValue = {}; + let matches: RegExpExecArray | null; + if ((matches = periodWeekRegExp.exec(durationStr))) { + spans.week = Number(matches[1]); + if (!spans.week) throw new Error('Duration can not have empty weeks'); + } else if ((matches = periodRegExp.exec(durationStr))) { + const nums = matches.map(Number); + for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { + const span = SPANS_WITHOUT_WEEK[i]; + const value = nums[i + 1]; + if (value) spans[span] = value; + } + } else { + throw new Error("Can not parse duration '" + durationStr + "'"); + } + return spans; +} + +function getSpansFromStartEnd(start: Date, end: Date, timezone: string): DurationValue { + start = second.floor(start, timezone); + end = second.floor(end, timezone); + if (end <= start) throw new Error('start must come before end'); + + const spans: DurationValue = {}; + let iterator: Date = start; + for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { + const span = SPANS_WITHOUT_WEEK[i]; + let spanCount = 0; + + // Shortcut + const length = end.valueOf() - iterator.valueOf(); + const canonicalLength: number = shifters[span].canonicalLength; + if (length < canonicalLength / 4) continue; + const numberToFit = Math.min(0, Math.floor(length / canonicalLength) - 1); + let iteratorMove: Date; + if (numberToFit > 0) { + // try to skip by numberToFit + iteratorMove = shifters[span].shift(iterator, timezone, numberToFit); + if (iteratorMove <= end) { + spanCount += numberToFit; + iterator = iteratorMove; + } + } + + while (true) { + iteratorMove = shifters[span].shift(iterator, timezone, 1); + if (iteratorMove <= end) { + iterator = iteratorMove; + spanCount++; + } else { + break; + } + } + + if (spanCount) { + spans[span] = spanCount; + } + } + return spans; +} + +function removeZeros(spans: DurationValue): DurationValue { + const newSpans: DurationValue = {}; + for (let i = 0; i < SPANS_WITH_WEEK.length; i++) { + const span = SPANS_WITH_WEEK[i]; + if (Number(spans[span]) > 0) { + newSpans[span] = spans[span]; + } + } + return newSpans; +} + +function fitIntoSpans(length: number, spansToCheck: DurationSpan[]): DurationValue { + const spans: DurationValue = {}; + + let lengthLeft = length; + for (let i = 0; i < spansToCheck.length; i++) { + const span = spansToCheck[i]; + const spanLength = shifters[span].canonicalLength; + const count = Math.floor(lengthLeft / spanLength); + + if (count) { + lengthLeft -= spanLength * count; + spans[span] = count; + } + } + + return spans; +} + +/** + * Represents an ISO duration like P1DT3H + */ +export class Duration { + public readonly singleSpan?: DurationSpan; + public readonly spans: Readonly; + + static fromCanonicalLength(length: number, skipMonths = false): Duration { + if (length <= 0) throw new Error('length must be positive'); + let spans = fitIntoSpans(length, skipMonths ? SPANS_WITHOUT_WEEK_OR_MONTH : SPANS_WITHOUT_WEEK); + + if ( + length % shifters['week'].canonicalLength === 0 && // Weeks fits + (Object.keys(spans).length > 1 || // We already have a more complex span + spans['day']) // or... we only have days and it might be simpler to express as weeks + ) { + spans = { week: length / shifters['week'].canonicalLength }; + } + + return new Duration(spans); + } + + static fromCanonicalLengthUpToDays(length: number): Duration { + if (length <= 0) throw new Error('length must be positive'); + return new Duration(fitIntoSpans(length, SPANS_UP_TO_DAY)); + } + + static fromRange(start: Date, end: Date, timezone: string): Duration { + return new Duration(getSpansFromStartEnd(start, end, timezone)); + } + + static pickSmallestGranularityThatFits( + granularities: Duration[], + span: number, + maxEntities: number, + ): Duration { + for (const granularity of granularities) { + if (span / granularity.getCanonicalLength() < maxEntities) return granularity; + } + return granularities[granularities.length - 1]; + } + + constructor(spans: DurationValue | string) { + const effectiveSpans: DurationValue = + typeof spans === 'string' ? getSpansFromString(spans) : removeZeros(spans); + + const usedSpans = Object.keys(effectiveSpans) as DurationSpan[]; + if (!usedSpans.length) throw new Error('Duration can not be empty'); + if (usedSpans.length === 1) { + this.singleSpan = usedSpans[0]; + } else if (effectiveSpans.week) { + throw new Error("Can not mix 'week' and other spans"); + } + this.spans = effectiveSpans; + } + + public toString() { + const strArr: string[] = ['P']; + const spans = this.spans; + if (spans.week) { + strArr.push(String(spans.week), 'W'); + } else { + let addedT = false; + for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { + const span = SPANS_WITHOUT_WEEK[i]; + const value = spans[span]; + if (!value) continue; + if (!addedT && i >= 3) { + strArr.push('T'); + addedT = true; + } + strArr.push(String(value), span[0].toUpperCase()); + } + } + return strArr.join(''); + } + + public add(duration: Duration): Duration { + return Duration.fromCanonicalLength(this.getCanonicalLength() + duration.getCanonicalLength()); + } + + public subtract(duration: Duration): Duration { + const newCanonicalDuration = this.getCanonicalLength() - duration.getCanonicalLength(); + if (newCanonicalDuration < 0) throw new Error('A duration can not be negative.'); + return Duration.fromCanonicalLength(newCanonicalDuration); + } + + public multiply(multiplier: number): Duration { + if (multiplier <= 0) throw new Error('Multiplier must be positive non-zero'); + if (multiplier === 1) return this; + const newCanonicalDuration = this.getCanonicalLength() * multiplier; + return Duration.fromCanonicalLength(newCanonicalDuration); + } + + public valueOf() { + return this.spans; + } + + public equals(other: Duration | undefined): boolean { + return other instanceof Duration && this.toString() === other.toString(); + } + + public isSimple(): boolean { + const { singleSpan } = this; + if (!singleSpan) return false; + return this.spans[singleSpan] === 1; + } + + public isFloorable(): boolean { + const { singleSpan } = this; + if (!singleSpan) return false; + const span = Number(this.spans[singleSpan]); + if (span === 1) return true; + const { siblings } = shifters[singleSpan]; + if (!siblings) return false; + return siblings % span === 0; + } + + /** + * Floors the date according to this duration. + * @param date The date to floor + * @param timezone The timezone within which to floor + */ + public floor(date: Date, timezone: string): Date { + const { singleSpan } = this; + if (!singleSpan) throw new Error('Can not floor on a complex duration'); + const span = this.spans[singleSpan]!; + const mover = shifters[singleSpan]; + let dt = mover.floor(date, timezone); + if (span !== 1) { + if (!mover.siblings) { + throw new Error(`Can not floor on a ${singleSpan} duration that is not 1`); + } + if (mover.siblings % span !== 0) { + throw new Error( + `Can not floor on a ${singleSpan} duration that does not divide into ${mover.siblings}`, + ); + } + dt = mover.round(dt, span, timezone); + } + return dt; + } + + /** + * Moves the given date by 'step' times of the duration + * Negative step value will move back in time. + * @param date The date to move + * @param timezone The timezone within which to make the move + * @param step The number of times to step by the duration + */ + public shift(date: Date, timezone: string, step = 1): Date { + const spans = this.spans; + for (const span of SPANS_WITH_WEEK) { + const value = spans[span]; + if (value) date = shifters[span].shift(date, timezone, step * value); + } + return date; + } + + public ceil(date: Date, timezone: string): Date { + const floored = this.floor(date, timezone); + if (floored.valueOf() === date.valueOf()) return date; // Just like ceil(3) is 3 and not 4 + return this.shift(floored, timezone, 1); + } + + public round(date: Date, timezone: string): Date { + const floorDate = this.floor(date, timezone); + const ceilDate = this.ceil(date, timezone); + const distanceToFloor = Math.abs(date.valueOf() - floorDate.valueOf()); + const distanceToCeil = Math.abs(date.valueOf() - ceilDate.valueOf()); + return distanceToFloor < distanceToCeil ? floorDate : ceilDate; + } + + /** + * Materializes all the values of this duration form start to end + * @param start The date to start on + * @param end The date to start on + * @param timezone The timezone within which to materialize + * @param step The number of times to step by the duration + */ + public materialize(start: Date, end: Date, timezone: string, step = 1): Date[] { + const values: Date[] = []; + let iter = this.floor(start, timezone); + while (iter <= end) { + values.push(iter); + iter = this.shift(iter, timezone, step); + } + return values; + } + + /** + * Checks to see if date is aligned to this duration within the timezone (floors to itself) + * @param date The date to check + * @param timezone The timezone within which to make the check + */ + public isAligned(date: Date, timezone: string): boolean { + return this.floor(date, timezone).valueOf() === date.valueOf(); + } + + /** + * Check to see if this duration can be divided by the given duration + * @param smaller The smaller duration to divide by + */ + public dividesBy(smaller: Duration): boolean { + const myCanonicalLength = this.getCanonicalLength(); + const smallerCanonicalLength = smaller.getCanonicalLength(); + return ( + myCanonicalLength % smallerCanonicalLength === 0 && + this.isFloorable() && + smaller.isFloorable() + ); + } + + public getCanonicalLength(): number { + const spans = this.spans; + let length = 0; + for (const span of SPANS_WITH_WEEK) { + const value = spans[span]; + if (value) length += value * shifters[span].canonicalLength; + } + return length; + } + + public getDescription(capitalize?: boolean): string { + const spans = this.spans; + const description: string[] = []; + for (const span of SPANS_WITH_WEEK) { + const value = spans[span]; + const spanTitle = capitalize ? capitalizeFirst(span) : span; + if (value) { + if (value === 1 && this.singleSpan) { + description.push(spanTitle); + } else { + description.push(pluralIfNeeded(value, spanTitle)); + } + } + } + return description.join(', '); + } + + public getSingleSpan(): string | undefined { + return this.singleSpan; + } + + public getSingleSpanValue(): number | undefined { + if (!this.singleSpan) return; + return this.spans[this.singleSpan]; + } + + public limitToDays(): Duration { + return Duration.fromCanonicalLengthUpToDays(this.getCanonicalLength()); + } +} diff --git a/web-console/src/utils/general.spec.ts b/web-console/src/utils/general.spec.ts index 4b97cb19acb0..39deab9420aa 100644 --- a/web-console/src/utils/general.spec.ts +++ b/web-console/src/utils/general.spec.ts @@ -29,8 +29,8 @@ import { hashJoaat, moveElement, moveToIndex, - objectHash, offsetToRowColumn, + OVERLAY_OPEN_SELECTOR, parseCsvLine, swapElements, } from './general'; @@ -178,12 +178,6 @@ describe('general', () => { }); }); - describe('objectHash', () => { - it('works', () => { - expect(objectHash({ hello: 'world1' })).toEqual('cc14ad13'); - }); - }); - describe('offsetToRowColumn', () => { it('works', () => { const str = 'Hello\nThis is a test\nstring.'; @@ -217,4 +211,10 @@ describe('general', () => { expect(caseInsensitiveEquals(undefined, '')).toEqual(false); }); }); + + describe('OVERLAY_OPEN_SELECTOR', () => { + it('is what it is', () => { + expect(OVERLAY_OPEN_SELECTOR).toEqual('.bp5-portal .bp5-overlay-open'); + }); + }); }); diff --git a/web-console/src/utils/general.tsx b/web-console/src/utils/general.tsx index b742013b2e83..d128d47f3462 100644 --- a/web-console/src/utils/general.tsx +++ b/web-console/src/utils/general.tsx @@ -59,6 +59,10 @@ export function isSimpleArray(a: any): a is (string | number | boolean)[] { ); } +export function arraysEqualByElement(xs: T[], ys: T[]): boolean { + return xs.length === ys.length && xs.every((x, i) => x === ys[i]); +} + export function wait(ms: number): Promise { return new Promise(resolve => { setTimeout(resolve, ms); @@ -251,6 +255,14 @@ export function uniq(array: readonly string[]): string[] { }); } +export function allSameValue(xs: readonly T[]): T | undefined { + const sameValue: T | undefined = xs[0]; + for (let i = 1; i < xs.length; i++) { + if (sameValue !== xs[i]) return; + } + return sameValue; +} + // ---------------------------- export function formatEmpty(str: string): string { @@ -373,6 +385,14 @@ export function formatDurationHybrid(ms: NumberLike): string { } } +export function timezoneOffsetInMinutesToString(offsetInMinutes: number, padHour: boolean): string { + const sign = offsetInMinutes < 0 ? '-' : '+'; + const absOffset = Math.abs(offsetInMinutes); + const h = Math.floor(absOffset / 60); + const m = absOffset % 60; + return `${sign}${padHour ? pad2(h) : h}:${pad2(m)}`; +} + function pluralize(word: string): string { // Ignoring irregular plurals. if (/(s|x|z|ch|sh)$/.test(word)) { @@ -612,12 +632,10 @@ export function hashJoaat(str: string): number { return (hash & 4294967295) >>> 0; } -export function objectHash(obj: any): string { - return hashJoaat(JSONBig.stringify(obj)).toString(16).padStart(8); -} +export const OVERLAY_OPEN_SELECTOR = `.${Classes.PORTAL} .${Classes.OVERLAY_OPEN}`; -export function hasPopoverOpen(): boolean { - return Boolean(document.querySelector(`${Classes.PORTAL} ${Classes.OVERLAY} ${Classes.POPOVER}`)); +export function hasOverlayOpen(): boolean { + return Boolean(document.querySelector(OVERLAY_OPEN_SELECTOR)); } export function checkedCircleIcon(checked: boolean): IconName { diff --git a/web-console/src/utils/index.tsx b/web-console/src/utils/index.tsx index edea5ad0a520..096f0dfe0634 100644 --- a/web-console/src/utils/index.tsx +++ b/web-console/src/utils/index.tsx @@ -19,10 +19,12 @@ export * from './base64-url'; export * from './column-metadata'; export * from './date'; +export * from './date-floor-shift-ceil/date-floor-shift-ceil'; export * from './download'; export * from './download-query-detail-archive'; export * from './druid-lookup'; export * from './druid-query'; +export * from './duration/duration'; export * from './formatter'; export * from './general'; export * from './local-storage-backed-visibility'; diff --git a/web-console/src/utils/query-action.ts b/web-console/src/utils/query-action.ts index 45ade6961388..3f17f22b67d2 100644 --- a/web-console/src/utils/query-action.ts +++ b/web-console/src/utils/query-action.ts @@ -16,6 +16,6 @@ * limitations under the License. */ -import type { SqlQuery } from '@druid-toolkit/query'; +import type { SqlQuery } from 'druid-query-toolkit'; export type QueryAction = (query: SqlQuery) => SqlQuery; diff --git a/web-console/src/utils/query-manager/query-manager.ts b/web-console/src/utils/query-manager/query-manager.ts index 5b141c7c89db..1023b23412ad 100644 --- a/web-console/src/utils/query-manager/query-manager.ts +++ b/web-console/src/utils/query-manager/query-manager.ts @@ -288,7 +288,7 @@ export class QueryManager { return this.lastQuery; } - public getLastIntermediateQuery(): any { + public getLastIntermediateQuery(): unknown { return this.lastIntermediateQuery; } diff --git a/web-console/src/utils/sampler.ts b/web-console/src/utils/sampler.ts index 7fd3d67364cc..5333013c12b4 100644 --- a/web-console/src/utils/sampler.ts +++ b/web-console/src/utils/sampler.ts @@ -16,8 +16,8 @@ * limitations under the License. */ -import { dedupe, F, SqlExpression, SqlFunction } from '@druid-toolkit/query'; import type { CancelToken } from 'axios'; +import { dedupe, F, SqlExpression, SqlFunction } from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import type { diff --git a/web-console/src/utils/sql.spec.ts b/web-console/src/utils/sql.spec.ts index 9e5533ee4e62..57e7bce95fe8 100644 --- a/web-console/src/utils/sql.spec.ts +++ b/web-console/src/utils/sql.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; +import { sane } from 'druid-query-toolkit'; import { findAllSqlQueriesInText, findSqlQueryPrefix } from './sql'; diff --git a/web-console/src/utils/sql.ts b/web-console/src/utils/sql.ts index 1cdf243819d2..6c168002078a 100644 --- a/web-console/src/utils/sql.ts +++ b/web-console/src/utils/sql.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { SqlBase } from '@druid-toolkit/query'; +import type { SqlBase } from 'druid-query-toolkit'; import { SqlColumn, SqlExpression, @@ -24,7 +24,7 @@ import { SqlLiteral, SqlQuery, SqlStar, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import type { RowColumn } from './general'; import { offsetToRowColumn } from './general'; diff --git a/web-console/src/views/explore-view/models/stage.ts b/web-console/src/utils/stage.ts similarity index 80% rename from web-console/src/views/explore-view/models/stage.ts rename to web-console/src/utils/stage.ts index a1d01258f0a2..7b65cfebd9cb 100644 --- a/web-console/src/views/explore-view/models/stage.ts +++ b/web-console/src/utils/stage.ts @@ -16,6 +16,13 @@ * limitations under the License. */ +export interface Margin { + top: number; + right: number; + bottom: number; + left: number; +} + export class Stage { public readonly width: number; public readonly height: number; @@ -28,4 +35,11 @@ export class Stage { public equals(other: Stage | undefined): boolean { return Boolean(other && this.width === other.width && this.height === other.height); } + + public applyMargin(margin: Margin): Stage { + return new Stage( + this.width - margin.left - margin.right, + this.height - margin.top - margin.bottom, + ); + } } diff --git a/web-console/src/utils/table-helpers.ts b/web-console/src/utils/table-helpers.ts index 45e8758bf6f8..a4ba4fce7ddf 100644 --- a/web-console/src/utils/table-helpers.ts +++ b/web-console/src/utils/table-helpers.ts @@ -16,9 +16,10 @@ * limitations under the License. */ -import type { QueryResult, SqlExpression } from '@druid-toolkit/query'; -import { C } from '@druid-toolkit/query'; -import type { Filter } from 'react-table'; +import { ascending, descending, sort } from 'd3-array'; +import type { QueryResult, SqlExpression } from 'druid-query-toolkit'; +import { C } from 'druid-query-toolkit'; +import type { Filter, SortingRule } from 'react-table'; import { filterMap, formatNumber, isNumberLike, oneOf } from './general'; import { deepSet } from './object-change'; @@ -67,19 +68,24 @@ export function getNumericColumnBraces( return numericColumnBraces; } -export interface Sorted { - id: string; - desc: boolean; -} - export interface TableState { page: number; pageSize: number; filtered: Filter[]; - sorted: Sorted[]; + sorted: SortingRule[]; } -export function sortedToOrderByClause(sorted: Sorted[]): string | undefined { +export function sortedToOrderByClause(sorted: SortingRule[]): string | undefined { if (!sorted.length) return; return 'ORDER BY ' + sorted.map(sort => `${C(sort.id)} ${sort.desc ? 'DESC' : 'ASC'}`).join(', '); } + +export function applySorting(xs: any[], sorted: SortingRule[]): any[] { + const firstSortingRule = sorted[0]; + if (!firstSortingRule) return xs; + const { id, desc } = firstSortingRule; + return sort( + xs, + desc ? (d1, d2) => descending(d1[id], d2[id]) : (d1, d2) => ascending(d1[id], d2[id]), + ); +} diff --git a/web-console/src/utils/types.ts b/web-console/src/utils/types.ts index 7f48655cb1c4..a3e38a2c8218 100644 --- a/web-console/src/utils/types.ts +++ b/web-console/src/utils/types.ts @@ -18,7 +18,7 @@ import type { IconName } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { Column } from '@druid-toolkit/query'; +import type { Column } from 'druid-query-toolkit'; export function columnToSummary(column: Column): string { const lines: string[] = [column.name]; diff --git a/web-console/src/utils/values-query.spec.tsx b/web-console/src/utils/values-query.spec.tsx index 7bc093bc3e82..89f4a4d2f14f 100644 --- a/web-console/src/utils/values-query.spec.tsx +++ b/web-console/src/utils/values-query.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { QueryResult, sane } from '@druid-toolkit/query'; +import { QueryResult, sane } from 'druid-query-toolkit'; import { queryResultToValuesQuery } from './values-query'; diff --git a/web-console/src/utils/values-query.tsx b/web-console/src/utils/values-query.tsx index 1b5e62b44c23..69130d380e75 100644 --- a/web-console/src/utils/values-query.tsx +++ b/web-console/src/utils/values-query.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { Column, QueryResult, SqlExpression } from '@druid-toolkit/query'; +import type { Column, QueryResult, SqlExpression } from 'druid-query-toolkit'; import { C, F, @@ -28,7 +28,7 @@ import { SqlRecord, SqlType, SqlValues, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import { oneOf } from './general'; diff --git a/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap b/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap index 6da41f6eb763..5f471fa1e390 100644 --- a/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap +++ b/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap @@ -109,6 +109,7 @@ exports[`DatasourcesView matches snapshot 1`] = ` /> { + datasources = datasourcesResp.map((d: any): DatasourceQueryResultRow => { const totalDataSize = deepGet(d, 'properties.segments.size') || -1; const segmentsToLoad = Number(loadstatus[d.name] || 0); const availableSegments = Number(deepGet(d, 'properties.segments.count')); @@ -521,9 +531,10 @@ GROUP BY 1, 2`; if (capabilities.hasOverlordAccess()) { auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => { try { - const taskList = ( - await Api.instance.get(`/druid/indexer/v1/tasks?state=running`, { cancelToken }) - ).data; + const taskList = await getApiArray( + `/druid/indexer/v1/tasks?state=running`, + cancelToken, + ); const runningTasksByDatasource = groupByAsMap( taskList, @@ -562,10 +573,10 @@ GROUP BY 1, 2`; if (showUnused) { try { unused = ( - await Api.instance.get( + await getApiArray( '/druid/coordinator/v1/metadata/datasources?includeUnused', ) - ).data.filter(d => !seen[d]); + ).filter(d => !seen[d]); } catch { AppToaster.show({ icon: IconNames.ERROR, @@ -578,7 +589,7 @@ GROUP BY 1, 2`; // Rules auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => { try { - const rules: Record = ( + const rules = ( await Api.instance.get>('/druid/coordinator/v1/rules', { cancelToken, }) @@ -589,7 +600,7 @@ GROUP BY 1, 2`; ...ds, rules: rules[ds.datasource] || [], })), - defaultRules: rules[DEFAULT_RULES_KEY], + defaultRules: rules[RuleUtil.DEFAULT_RULES_KEY], }; } catch { AppToaster.show({ @@ -657,8 +668,19 @@ GROUP BY 1, 2`; } private readonly refresh = (auto: boolean): void => { - if (auto && hasPopoverOpen()) return; + if (auto && hasOverlayOpen()) return; this.datasourceQueryManager.rerunLastQuery(auto); + + const { showSegmentTimeline } = this.state; + if (showSegmentTimeline) { + // Create a new capabilities object to force the segment timeline to re-render + this.setState(({ showSegmentTimeline }) => ({ + showSegmentTimeline: { + ...showSegmentTimeline, + capabilities: this.props.capabilities.clone(), + }, + })); + } }; private fetchDatasourceData() { @@ -846,9 +868,9 @@ GROUP BY 1, 2`; { - if (!lastDatasourcesQuery) return; + if (typeof lastDatasourcesQuery !== 'string') return; goToQuery({ queryString: lastDatasourcesQuery }); }} /> @@ -921,6 +943,7 @@ GROUP BY 1, 2`; retentionDialogOpenOn: { datasource: '_default', rules: defaultRules, + defaultRules, }, }; }); @@ -1030,10 +1053,13 @@ GROUP BY 1, 2`; icon: IconNames.AUTOMATIC_UPDATES, title: 'Edit retention rules', onAction: () => { + const defaultRules = this.state.datasourcesAndDefaultRulesState.data?.defaultRules; + if (!defaultRules) return; this.setState({ retentionDialogOpenOn: { datasource, rules: rules || [], + defaultRules, }, }); }, @@ -1097,9 +1123,8 @@ GROUP BY 1, 2`; private renderRetentionDialog() { const { capabilities } = this.props; - const { retentionDialogOpenOn, datasourcesAndDefaultRulesState } = this.state; - const defaultRules = datasourcesAndDefaultRulesState.data?.defaultRules; - if (!retentionDialogOpenOn || !defaultRules) return; + const { retentionDialogOpenOn } = this.state; + if (!retentionDialogOpenOn) return; return ( this.setState({ retentionDialogOpenOn: undefined })} onSave={this.saveRules} /> @@ -1139,8 +1164,9 @@ GROUP BY 1, 2`; } private renderDatasourcesTable() { - const { goToSegments, goToTasks, capabilities, filters, onFiltersChange } = this.props; - const { datasourcesAndDefaultRulesState, showUnused, visibleColumns } = this.state; + const { goToTasks, capabilities, filters, onFiltersChange } = this.props; + const { datasourcesAndDefaultRulesState, showUnused, visibleColumns, showSegmentTimeline } = + this.state; let { datasources, defaultRules } = datasourcesAndDefaultRulesState.data || { datasources: [] }; @@ -1194,12 +1220,19 @@ GROUP BY 1, 2`; show: visibleColumns.shown('Datasource name'), accessor: 'datasource', width: 150, - Cell: row => ( + Cell: ({ value, original }) => ( this.onDetail(row.original)} + onClick={() => this.onDetail(original)} hoverIcon={IconNames.SEARCH_TEMPLATE} + tooltip="Show detail" > - {row.value} + {showSegmentTimeline ? ( + <> + {value} + + ) : ( + value + )} ), }, @@ -1225,7 +1258,12 @@ GROUP BY 1, 2`; const hasZeroReplicationRule = RuleUtil.hasZeroReplicaRule(rules, defaultRules); const descriptor = hasZeroReplicationRule ? 'pre-cached' : 'available'; const segmentsEl = ( - goToSegments(datasource)}> + + this.setState({ showSegmentTimeline: { capabilities, datasource } }) + } + data-tooltip="Show in segment timeline" + > {pluralIfNeeded(num_segments, 'segment')} ); @@ -1318,7 +1356,7 @@ GROUP BY 1, 2`; goToTasks(original.datasource)} hoverIcon={IconNames.ARROW_TOP_RIGHT} - title="Go to tasks" + tooltip="Go to tasks" > {formatRunningTasks(runningTasks)} @@ -1498,6 +1536,7 @@ GROUP BY 1, 2`; if (!compaction) return; return ( { if (!compaction) return; @@ -1614,6 +1653,7 @@ GROUP BY 1, 2`; return ( { if (!defaultRules) return; @@ -1621,6 +1661,7 @@ GROUP BY 1, 2`; retentionDialogOpenOn: { datasource, rules, + defaultRules, }, }); }} @@ -1668,7 +1709,7 @@ GROUP BY 1, 2`; } render() { - const { capabilities } = this.props; + const { capabilities, goToSegments } = this.props; const { showUnused, visibleColumns, @@ -1681,9 +1722,7 @@ GROUP BY 1, 2`;
{ - this.refresh(auto); - }} + onRefresh={this.refresh} localStorageKey={LocalStorageKeys.DATASOURCES_REFRESH_RATE} /> {this.renderBulkDatasourceActions()} @@ -1694,9 +1733,13 @@ GROUP BY 1, 2`; disabled={!capabilities.hasCoordinatorAccess()} /> this.setState({ showSegmentTimeline: !showSegmentTimeline })} + onChange={() => + this.setState({ + showSegmentTimeline: showSegmentTimeline ? undefined : { capabilities }, + }) + } disabled={!capabilities.hasSqlOrCoordinatorAccess()} /> - {showSegmentTimeline && } + {showSegmentTimeline && ( + { + return ( +
- )} -
{openOn.text}
- {!minimal && ( -
- )} -
, - document.body, - ); -}; diff --git a/web-console/src/components/record-table-pane/record-table-pane.tsx b/web-console/src/components/record-table-pane/record-table-pane.tsx index 803ee783c2cd..e44eab355602 100644 --- a/web-console/src/components/record-table-pane/record-table-pane.tsx +++ b/web-console/src/components/record-table-pane/record-table-pane.tsx @@ -18,8 +18,8 @@ import { Button, Icon, Popover } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; +import type { Column, QueryResult } from '@druid-toolkit/query'; import classNames from 'classnames'; -import type { Column, QueryResult } from 'druid-query-toolkit'; import React, { useEffect, useState } from 'react'; import type { RowRenderProps } from 'react-table'; import ReactTable from 'react-table'; diff --git a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap index 7246fe89490f..2769c8af739e 100644 --- a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap +++ b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap @@ -70,7 +70,7 @@ exports[`RuleEditor matches snapshot no tier in rule 1`] = ` style="transform: translateY(0); transition: none;" >
- + + + +`; diff --git a/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap b/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap index 206c2be703bd..ae30df246c2b 100644 --- a/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap +++ b/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap @@ -4,309 +4,191 @@ exports[`SegmentTimeline matches snapshot 1`] = `
-
+
- +
+
- - -
+ + Total size + + + +
+
+
- - - +
+
+
- - - - - - - - + +
- - - + Show all + + + +
+
-
`; diff --git a/web-console/src/components/segment-timeline/bar-group.tsx b/web-console/src/components/segment-timeline/bar-group.tsx new file mode 100644 index 000000000000..d0cf867e2b2d --- /dev/null +++ b/web-console/src/components/segment-timeline/bar-group.tsx @@ -0,0 +1,75 @@ +/* + * 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 { AxisScale } from 'd3-axis'; +import React from 'react'; + +import { BarUnit } from './bar-unit'; +import type { BarUnitData, HoveredBarInfo } from './stacked-bar-chart'; + +interface BarGroupProps { + dataToRender: BarUnitData[]; + changeActiveDatasource: (dataSource: string) => void; + formatTick: (e: number) => string; + xScale: AxisScale; + yScale: AxisScale; + barWidth: number; + onHoverBar?: (e: any) => void; + offHoverBar?: () => void; + hoverOn?: HoveredBarInfo | null; +} + +export class BarGroup extends React.Component { + shouldComponentUpdate(nextProps: BarGroupProps): boolean { + return nextProps.hoverOn === this.props.hoverOn; + } + + render() { + const { dataToRender, changeActiveDatasource, xScale, yScale, onHoverBar, barWidth } = + this.props; + if (dataToRender === undefined) return null; + + return dataToRender.map((entry: BarUnitData, i: number) => { + const y0 = yScale(entry.y0 || 0) || 0; + const x = xScale(new Date(entry.x + 'T00:00:00Z')); + const y = yScale((entry.y0 || 0) + entry.y) || 0; + const height = Math.max(y0 - y, 0); + const barInfo: HoveredBarInfo = { + xCoordinate: x, + yCoordinate: y, + height, + datasource: entry.datasource, + xValue: entry.x, + yValue: entry.y, + dailySize: entry.dailySize, + }; + return ( + changeActiveDatasource(entry.datasource)} + onHover={() => onHoverBar && onHoverBar(barInfo)} + /> + ); + }); + } +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart.scss b/web-console/src/components/segment-timeline/bar-unit.spec.tsx similarity index 68% rename from web-console/src/components/segment-timeline/segment-bar-chart.scss rename to web-console/src/components/segment-timeline/bar-unit.spec.tsx index 130997119e3e..d5926dcf69f0 100644 --- a/web-console/src/components/segment-timeline/segment-bar-chart.scss +++ b/web-console/src/components/segment-timeline/bar-unit.spec.tsx @@ -16,6 +16,18 @@ * limitations under the License. */ -.segment-bar-chart { - position: relative; -} +import { render } from '@testing-library/react'; + +import { BarUnit } from './bar-unit'; + +describe('BarUnit', () => { + it('matches snapshot', () => { + const barGroup = ( + + + + ); + const { container } = render(barGroup); + expect(container.firstChild).toMatchSnapshot(); + }); +}); diff --git a/web-console/src/views/workbench-view/timezone-menu-items/timezone-menu-items.spec.tsx b/web-console/src/components/segment-timeline/bar-unit.tsx similarity index 58% rename from web-console/src/views/workbench-view/timezone-menu-items/timezone-menu-items.spec.tsx rename to web-console/src/components/segment-timeline/bar-unit.tsx index 5a1a2b9ce51b..8591f68cc2e3 100644 --- a/web-console/src/views/workbench-view/timezone-menu-items/timezone-menu-items.spec.tsx +++ b/web-console/src/components/segment-timeline/bar-unit.tsx @@ -16,26 +16,30 @@ * limitations under the License. */ -import { shallow } from '../../../utils/shallow-renderer'; +interface BarChartUnitProps { + x: number | undefined; + y: number; + width: number; + height: number; + style?: any; + onClick?: () => void; + onHover?: () => void; + offHover?: () => void; +} -import { TimezoneMenuItems } from './timezone-menu-items'; - -jest.useFakeTimers('modern').setSystemTime(Date.parse('2024-06-08T12:34:56Z')); - -describe('TimezoneMenuItems', () => { - it('ensure UTC', () => { - expect(new Date().getTimezoneOffset()).toBe(0); - }); - - it('matches snapshot', () => { - const comp = shallow( - {}} - defaultSqlTimeZone="Etc/UTC" - />, - ); - - expect(comp).toMatchSnapshot(); - }); -}); +export function BarUnit(props: BarChartUnitProps) { + const { x, y, width, height, style, onClick, onHover, offHover } = props; + return ( + + ); +} diff --git a/web-console/src/components/segment-timeline/chart-axis.tsx b/web-console/src/components/segment-timeline/chart-axis.tsx index b8ee4e9cbb11..bc333d33b773 100644 --- a/web-console/src/components/segment-timeline/chart-axis.tsx +++ b/web-console/src/components/segment-timeline/chart-axis.tsx @@ -16,22 +16,22 @@ * limitations under the License. */ -import type { Axis } from 'd3-axis'; import { select } from 'd3-selection'; +import React from 'react'; interface ChartAxisProps { - className?: string; transform?: string; - axis: Axis; + scale: any; + className?: string; } -export const ChartAxis = function ChartAxis(props: ChartAxisProps) { - const { transform, axis, className } = props; +export const ChartAxis = React.memo(function ChartAxis(props: ChartAxisProps) { + const { transform, scale, className } = props; return ( select(node).call(axis as any)} + ref={node => select(node).call(scale)} /> ); -}; +}); diff --git a/web-console/src/components/segment-timeline/common.ts b/web-console/src/components/segment-timeline/common.ts deleted file mode 100644 index 48aa5ce48d46..000000000000 --- a/web-console/src/components/segment-timeline/common.ts +++ /dev/null @@ -1,87 +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 { sum } from 'd3-array'; - -import type { Duration } from '../../utils'; -import { formatBytes, formatInteger } from '../../utils'; - -export type IntervalStat = 'segments' | 'size' | 'rows'; - -export const INTERVAL_STATS: IntervalStat[] = ['segments', 'size', 'rows']; - -export function getIntervalStatTitle(intervalStat: IntervalStat): string { - switch (intervalStat) { - case 'segments': - return 'Num. segments'; - - case 'size': - return 'Size'; - - case 'rows': - return 'Rows'; - - default: - return intervalStat; - } -} - -export function aggregateSegmentStats( - xs: readonly Record[], -): Record { - return { - segments: sum(xs, s => s.segments), - size: sum(xs, s => s.size), - rows: sum(xs, s => s.rows), - }; -} - -export function formatIntervalStat(stat: IntervalStat, n: number) { - switch (stat) { - case 'segments': - case 'rows': - return formatInteger(n); - - case 'size': - return formatBytes(n); - - default: - return ''; - } -} - -export interface IntervalRow extends Record { - start: Date; - end: Date; - datasource: string; - realtime: boolean; - originalTimeSpan: Duration; -} - -export interface TrimmedIntervalRow extends IntervalRow { - shownDays: number; - normalized: Record; -} - -export interface IntervalBar extends TrimmedIntervalRow { - offset: Record; -} - -export function formatIsoDateOnly(date: Date): string { - return date.toISOString().slice(0, 10); -} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart-render.scss b/web-console/src/components/segment-timeline/segment-bar-chart-render.scss deleted file mode 100644 index 762772a0f5c5..000000000000 --- a/web-console/src/components/segment-timeline/segment-bar-chart-render.scss +++ /dev/null @@ -1,169 +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'; - -.segment-bar-chart-render { - position: relative; - overflow: hidden; - - @keyframes pulseOpacity { - 0% { - opacity: 0.8; - } - 100% { - opacity: 0.95; - } - } - - svg { - position: absolute; - - .chart-axis text { - user-select: none; - } - - .hover-highlight { - fill: white; - fill-opacity: 0.1; - } - - .hovered-bar { - fill: none; - stroke: #ffffff; - stroke-width: 1.5px; - } - - .selection { - fill: transparent; - stroke: #ffffff; - stroke-width: 1px; - opacity: 0.8; - - &.done { - opacity: 1; - } - } - - .shifter { - fill: white; - fill-opacity: 0.2; - filter: blur(1px); - } - - .time-shift-indicator { - fill: white; - fill-opacity: 0.001; - cursor: grab; - - &:hover { - fill-opacity: 0.1; - } - - &.shifting { - fill-opacity: 0.2; - cursor: grabbing; - } - } - - .gridline-x { - line { - stroke-dasharray: 5, 5; - opacity: 0.5; - } - } - - .now-line { - stroke: $orange4; - stroke-dasharray: 2, 2; - opacity: 0.7; - } - - .bar-unit { - &.realtime { - animation: pulseOpacity 3s alternate infinite; - } - } - } - - .rule-tape { - position: absolute; - top: 5px; - height: 15px; - font-size: 10px; - - .rule-error { - @include pin-full(); - background-color: $red3; - color: $white; - } - - .load-rule { - position: absolute; - overflow: hidden; - padding-left: 2px; - border-left: 1px solid $dark-gray2; - border-right: 1px solid $dark-gray2; - top: 0; - height: 100%; - text-overflow: ellipsis; - - &.load { - background-color: $green1; - - &:nth-child(even) { - background-color: $green3; - } - } - - &.drop { - background-color: $dark-gray5; - - &:nth-child(even) { - background-color: $gray1; - } - } - - &.broadcast { - background-color: $indigo1; - - &:nth-child(even) { - background-color: $indigo3; - } - } - } - } - - .empty-placeholder { - @include pin-full; - display: flex; - align-items: center; - justify-content: center; - font-size: 20px; - user-select: none; - pointer-events: none; - } -} - -.segment-bar-chart-bubble { - .button-bar { - padding-top: 5px; - display: flex; - gap: 5px; - } -} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx b/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx deleted file mode 100644 index 6f2817a202eb..000000000000 --- a/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx +++ /dev/null @@ -1,793 +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, Intent } from '@blueprintjs/core'; -import type { NonNullDateRange } from '@blueprintjs/datetime'; -import { IconNames } from '@blueprintjs/icons'; -import IntervalTree from '@flatten-js/interval-tree'; -import classNames from 'classnames'; -import { max, sort, sum } from 'd3-array'; -import { axisBottom, axisLeft } from 'd3-axis'; -import { scaleLinear, scaleUtc } from 'd3-scale'; -import type { MouseEvent as ReactMouseEvent, ReactNode } from 'react'; -import { useMemo, useRef, useState } from 'react'; - -import type { Rule } from '../../druid-models'; -import { getDatasourceColor, RuleUtil } from '../../druid-models'; -import { useClock, useGlobalEventListener } from '../../hooks'; -import { - allSameValue, - arraysEqualByElement, - clamp, - day, - Duration, - formatBytes, - formatNumber, - groupBy, - groupByAsMap, - minute, - month, - pluralIfNeeded, - TZ_UTC, - uniq, -} from '../../utils'; -import type { Margin, Stage } from '../../utils/stage'; -import type { PortalBubbleOpenOn } from '../portal-bubble/portal-bubble'; -import { PortalBubble } from '../portal-bubble/portal-bubble'; - -import { ChartAxis } from './chart-axis'; -import type { IntervalBar, IntervalRow, IntervalStat, TrimmedIntervalRow } from './common'; -import { aggregateSegmentStats, formatIntervalStat, formatIsoDateOnly } from './common'; - -import './segment-bar-chart-render.scss'; - -const CHART_MARGIN: Margin = { top: 20, right: 0, bottom: 25, left: 70 }; -const MIN_BAR_WIDTH = 4; -const POSSIBLE_GRANULARITIES = [ - new Duration('PT15M'), - new Duration('PT1H'), - new Duration('PT6H'), - new Duration('P1D'), - new Duration('P1M'), - new Duration('P1Y'), -]; - -const EXTEND_X_SCALE_DOMAIN_BY = 1; - -function formatStartDuration(start: Date, duration: Duration): string { - let sliceLength; - const { singleSpan } = duration; - switch (singleSpan) { - case 'year': - sliceLength = 4; - break; - - case 'month': - sliceLength = 7; - break; - - case 'day': - sliceLength = 10; - break; - - case 'hour': - sliceLength = 13; - break; - - case 'minute': - sliceLength = 16; - break; - - default: - sliceLength = 19; - break; - } - - return `${start.toISOString().slice(0, sliceLength)}/${duration}`; -} - -// --------------------------------------- -// Load rule stuff - -function loadRuleToBaseType(loadRule: Rule): string { - const m = /^(load|drop|broadcast)/.exec(loadRule.type); - return m ? m[1] : 'load'; -} - -const NEGATIVE_INFINITY_DATE = new Date(Date.UTC(1000, 0, 1)); -const POSITIVE_INFINITY_DATE = new Date(Date.UTC(3000, 0, 1)); - -function loadRuleToDateRange(loadRule: Rule): NonNullDateRange { - switch (loadRule.type) { - case 'loadByInterval': - case 'dropByInterval': - case 'broadcastByInterval': - return String(loadRule.interval) - .split('/') - .map(d => new Date(d)) as NonNullDateRange; - - case 'loadByPeriod': - case 'dropByPeriod': - case 'broadcastByPeriod': - return [ - new Duration(loadRule.period || 'P1D').shift(new Date(), TZ_UTC, -1), - loadRule.includeFuture ? POSITIVE_INFINITY_DATE : new Date(), - ]; - - case 'dropBeforeByPeriod': - return [ - NEGATIVE_INFINITY_DATE, - new Duration(loadRule.period || 'P1D').shift(new Date(), TZ_UTC, -1), - ]; - - default: - return [NEGATIVE_INFINITY_DATE, POSITIVE_INFINITY_DATE]; - } -} - -// --------------------------------------- - -function offsetDateRange(dateRange: NonNullDateRange, offset: number): NonNullDateRange { - return [new Date(dateRange[0].valueOf() + offset), new Date(dateRange[1].valueOf() + offset)]; -} - -function stackIntervalRows(trimmedIntervalRows: TrimmedIntervalRow[]): { - intervalBars: IntervalBar[]; - intervalTree: IntervalTree; -} { - // Total size of the datasource will be user as an ordering tiebreaker - const datasourceToTotalSize = groupByAsMap( - trimmedIntervalRows, - intervalRow => intervalRow.datasource, - intervalRows => sum(intervalRows, intervalRow => intervalRow.size), - ); - - const sortedIntervalRows = sort(trimmedIntervalRows, (a, b) => { - const shownDaysDiff = b.shownDays - a.shownDays; - if (shownDaysDiff) return shownDaysDiff; - - const timeSpanDiff = - b.originalTimeSpan.getCanonicalLength() - a.originalTimeSpan.getCanonicalLength(); - if (timeSpanDiff) return timeSpanDiff; - - const totalSizeDiff = datasourceToTotalSize[b.datasource] - datasourceToTotalSize[a.datasource]; - if (totalSizeDiff) return totalSizeDiff; - - return Number(a.realtime) - Number(b.realtime); - }); - - const intervalTree = new IntervalTree(); - const intervalBars = sortedIntervalRows.map(intervalRow => { - const startMs = intervalRow.start.valueOf(); - const endMs = intervalRow.end.valueOf(); - const intervalRowsBelow = intervalTree.search([startMs + 1, startMs + 2]) as IntervalBar[]; - const intervalBar: IntervalBar = { - ...intervalRow, - offset: aggregateSegmentStats(intervalRowsBelow.map(i => i.normalized)), - }; - intervalTree.insert([startMs, endMs], intervalBar); - return intervalBar; - }); - - return { - intervalBars, - intervalTree, - }; -} - -interface BubbleInfo { - start: Date; - end: Date; - timeLabel: string; - intervalBars: IntervalBar[]; -} - -interface SelectionRange { - start: Date; - end: Date; - done?: boolean; -} - -export interface DatasourceRules { - loadRules: Rule[]; - defaultLoadRules: Rule[]; -} - -export interface SegmentBarChartRenderProps { - intervalRows: IntervalRow[]; - datasourceRules: DatasourceRules | undefined; - datasourceRulesError: string | undefined; - - stage: Stage; - dateRange: NonNullDateRange; - changeDateRange(dateRange: NonNullDateRange): void; - shownIntervalStat: IntervalStat; - shownDatasource: string | undefined; - changeShownDatasource(datasource: string | undefined): void; - getIntervalActionButton?( - start: Date, - end: Date, - datasource?: string, - realtime?: boolean, - ): ReactNode; -} - -export const SegmentBarChartRender = function SegmentBarChartRender( - props: SegmentBarChartRenderProps, -) { - const { - intervalRows, - datasourceRules, - datasourceRulesError, - - stage, - shownIntervalStat, - dateRange, - changeDateRange, - shownDatasource, - changeShownDatasource, - getIntervalActionButton, - } = props; - const [mouseDownAt, setMouseDownAt] = useState< - { time: Date; action: 'select' | 'shift' } | undefined - >(); - const [selection, setSelection] = useState(); - - function setSelectionIfNeeded(newSelection: SelectionRange) { - if ( - selection && - selection.start.valueOf() === newSelection.start.valueOf() && - selection.end.valueOf() === newSelection.end.valueOf() && - selection.done === newSelection.done - ) { - return; - } - setSelection(newSelection); - } - - const [bubbleInfo, setBubbleInfo] = useState(); - - function setBubbleInfoIfNeeded(newBubbleInfo: BubbleInfo) { - if ( - bubbleInfo && - bubbleInfo.start.valueOf() === newBubbleInfo.start.valueOf() && - bubbleInfo.end.valueOf() === newBubbleInfo.end.valueOf() && - bubbleInfo.timeLabel === newBubbleInfo.timeLabel && - arraysEqualByElement(bubbleInfo.intervalBars, newBubbleInfo.intervalBars) - ) { - return; - } - setBubbleInfo(newBubbleInfo); - } - - const [shiftOffset, setShiftOffset] = useState(); - - const now = useClock(minute.canonicalLength); - const svgRef = useRef(null); - - const trimGranularity = useMemo(() => { - return Duration.pickSmallestGranularityThatFits( - POSSIBLE_GRANULARITIES, - dateRange[1].valueOf() - dateRange[0].valueOf(), - Math.floor(stage.width / MIN_BAR_WIDTH), - ).toString(); - }, [dateRange, stage.width]); - - const { intervalBars, intervalTree } = useMemo(() => { - const shownIntervalRows = intervalRows.filter( - ({ start, end, datasource }) => - start <= dateRange[1] && - dateRange[0] < end && - (!shownDatasource || datasource === shownDatasource), - ); - const averageRowSizeByDatasource = groupByAsMap( - shownIntervalRows.filter(intervalRow => intervalRow.size > 0 && intervalRow.rows > 0), - intervalRow => intervalRow.datasource, - intervalRows => sum(intervalRows, d => d.size) / sum(intervalRows, d => d.rows), - ); - - const trimDuration = new Duration(trimGranularity); - const trimmedIntervalRows = shownIntervalRows.map(intervalRow => { - const { start, end, segments, size, rows } = intervalRow; - const startTrimmed = trimDuration.floor(start, TZ_UTC); - let endTrimmed = trimDuration.ceil(end, TZ_UTC); - - // Special handling to catch WEEK intervals when trimming to month. - if (trimGranularity === 'P1M' && intervalRow.originalTimeSpan.toString() === 'P7D') { - endTrimmed = trimDuration.shift(startTrimmed, TZ_UTC); - } - - const shownDays = (endTrimmed.valueOf() - startTrimmed.valueOf()) / day.canonicalLength; - const shownSize = - size === 0 ? rows * averageRowSizeByDatasource[intervalRow.datasource] : size; - return { - ...intervalRow, - start: startTrimmed, - end: endTrimmed, - shownDays, - size: shownSize, - normalized: { - size: shownSize / shownDays, - rows: rows / shownDays, - segments: segments / shownDays, - }, - }; - }); - - const fullyGroupedSegmentRows = groupBy( - trimmedIntervalRows, - trimmedIntervalRow => - [ - trimmedIntervalRow.start.toISOString(), - trimmedIntervalRow.end.toISOString(), - trimmedIntervalRow.originalTimeSpan, - trimmedIntervalRow.datasource, - trimmedIntervalRow.realtime, - ].join('/'), - (trimmedIntervalRows): TrimmedIntervalRow => { - const firstIntervalRow = trimmedIntervalRows[0]; - return { - ...firstIntervalRow, - ...aggregateSegmentStats(trimmedIntervalRows), - normalized: aggregateSegmentStats(trimmedIntervalRows.map(t => t.normalized)), - }; - }, - ); - - return stackIntervalRows(fullyGroupedSegmentRows); - }, [intervalRows, trimGranularity, dateRange, shownDatasource]); - - const innerStage = stage.applyMargin(CHART_MARGIN); - - const baseTimeScale = scaleUtc() - .domain(dateRange) - .range([EXTEND_X_SCALE_DOMAIN_BY, innerStage.width - EXTEND_X_SCALE_DOMAIN_BY]); - const timeScale = shiftOffset - ? baseTimeScale.copy().domain(offsetDateRange(dateRange, shiftOffset)) - : baseTimeScale; - - const maxNormalizedStat = max( - intervalBars, - d => d.normalized[shownIntervalStat] + d.offset[shownIntervalStat], - ); - const statScale = scaleLinear() - .rangeRound([innerStage.height, 0]) - .domain([0, (maxNormalizedStat ?? 1) * 1.05]); - - const formatTickRate = (n: number) => { - switch (shownIntervalStat) { - case 'segments': - return formatNumber(n); // + ' seg/day'; - - case 'rows': - return formatNumber(n); // + ' row/day'; - - case 'size': - return formatBytes(n); - } - }; - - function handleMouseDown(e: ReactMouseEvent) { - const svg = svgRef.current; - if (!svg) return; - e.preventDefault(); - - if (selection) { - setSelection(undefined); - } else { - const rect = svg.getBoundingClientRect(); - const x = e.clientX - rect.x - CHART_MARGIN.left; - const y = e.clientY - rect.y - CHART_MARGIN.top; - const time = baseTimeScale.invert(x); - const action = y > innerStage.height || e.shiftKey ? 'shift' : 'select'; - setBubbleInfo(undefined); - setMouseDownAt({ - time, - action, - }); - } - } - - useGlobalEventListener('mousemove', (e: MouseEvent) => { - const svg = svgRef.current; - if (!svg) return; - const rect = svg.getBoundingClientRect(); - const x = e.clientX - rect.x - CHART_MARGIN.left; - const y = e.clientY - rect.y - CHART_MARGIN.top; - - if (mouseDownAt) { - e.preventDefault(); - - const b = baseTimeScale.invert(x); - if (mouseDownAt.action === 'shift' || e.shiftKey) { - setShiftOffset(mouseDownAt.time.valueOf() - b.valueOf()); - } else { - if (mouseDownAt.time < b) { - setSelectionIfNeeded({ - start: day.floor(mouseDownAt.time, TZ_UTC), - end: day.ceil(b, TZ_UTC), - }); - } else { - setSelectionIfNeeded({ - start: day.floor(b, TZ_UTC), - end: day.ceil(mouseDownAt.time, TZ_UTC), - }); - } - } - } else if (!selection) { - if ( - 0 <= x && - x <= innerStage.width && - 0 <= y && - y <= innerStage.height + CHART_MARGIN.bottom - ) { - const time = baseTimeScale.invert(x); - const shifter = - new Duration(trimGranularity).getCanonicalLength() > day.canonicalLength * 25 - ? month - : day; - const start = shifter.floor(time, TZ_UTC); - const end = shifter.ceil(time, TZ_UTC); - - let intervalBars: IntervalBar[] = []; - if (y <= innerStage.height) { - const bars = intervalTree.search([ - time.valueOf() + 1, - time.valueOf() + 2, - ]) as IntervalBar[]; - - if (bars.length) { - const stat = statScale.invert(y); - const hoverBar = bars.find( - bar => - bar.offset[shownIntervalStat] <= stat && - stat < bar.offset[shownIntervalStat] + bar.normalized[shownIntervalStat], - ); - intervalBars = hoverBar ? [hoverBar] : bars; - } - } - setBubbleInfoIfNeeded({ - start, - end, - timeLabel: start.toISOString().slice(0, shifter === day ? 10 : 7), - intervalBars, - }); - } else { - setBubbleInfo(undefined); - } - } - }); - - useGlobalEventListener('mouseup', (e: MouseEvent) => { - if (!mouseDownAt) return; - e.preventDefault(); - setMouseDownAt(undefined); - - const svg = svgRef.current; - if (!svg) return; - const rect = svg.getBoundingClientRect(); - const x = e.clientX - rect.x - CHART_MARGIN.left; - const y = e.clientY - rect.y - CHART_MARGIN.top; - - if (shiftOffset || selection) { - setShiftOffset(undefined); - if (mouseDownAt.action === 'shift' || e.shiftKey) { - if (shiftOffset) { - changeDateRange(offsetDateRange(dateRange, shiftOffset)); - } - } else { - if (selection) { - setSelection({ ...selection, done: true }); - } - } - } else if (0 <= x && x <= innerStage.width && 0 <= y && y <= innerStage.height) { - const time = baseTimeScale.invert(x); - - const bars = intervalTree.search([time.valueOf() + 1, time.valueOf() + 2]) as IntervalBar[]; - - if (bars.length) { - const stat = statScale.invert(y); - const hoverBar = bars.find( - bar => - bar.offset[shownIntervalStat] <= stat && - stat < bar.offset[shownIntervalStat] + bar.normalized[shownIntervalStat], - ); - if (hoverBar) { - changeShownDatasource(shownDatasource ? undefined : hoverBar.datasource); - } - } - } - }); - - useGlobalEventListener('keydown', (e: KeyboardEvent) => { - if (e.key === 'Escape' && mouseDownAt) { - setMouseDownAt(undefined); - setSelection(undefined); - } - }); - - function startEndToXWidth({ start, end }: { start: Date; end: Date }) { - const xStart = clamp(timeScale(start), 0, innerStage.width); - const xEnd = clamp(timeScale(end), 0, innerStage.width); - - return { - x: xStart, - width: Math.max(xEnd - xStart - 1, 1), - }; - } - - function segmentBarToRect(intervalBar: IntervalBar) { - const y0 = statScale(intervalBar.offset[shownIntervalStat]); - const y = statScale( - intervalBar.normalized[shownIntervalStat] + intervalBar.offset[shownIntervalStat], - ); - - return { - ...startEndToXWidth(intervalBar), - y: y, - height: y0 - y, - }; - } - - let hoveredOpenOn: PortalBubbleOpenOn | undefined; - if (svgRef.current) { - const rect = svgRef.current.getBoundingClientRect(); - - if (bubbleInfo) { - const hoveredIntervalBars = bubbleInfo.intervalBars; - - let title: string | undefined; - let text: ReactNode; - if (hoveredIntervalBars.length === 0) { - title = bubbleInfo.timeLabel; - text = ''; - } else if (hoveredIntervalBars.length === 1) { - const hoveredIntervalBar = hoveredIntervalBars[0]; - title = `${formatStartDuration( - hoveredIntervalBar.start, - hoveredIntervalBar.originalTimeSpan, - )}${hoveredIntervalBar.realtime ? ' (realtime)' : ''}`; - text = ( - <> - {!shownDatasource &&
{`Datasource: ${hoveredIntervalBar.datasource}`}
} -
{`Size: ${ - hoveredIntervalBar.realtime - ? 'estimated for realtime' - : formatIntervalStat('size', hoveredIntervalBar.size) - }`}
-
{`Rows: ${formatIntervalStat('rows', hoveredIntervalBar.rows)}`}
-
{`Segments: ${formatIntervalStat('segments', hoveredIntervalBar.segments)}`}
- - ); - } else { - const datasources = uniq(hoveredIntervalBars.map(b => b.datasource)); - const agg = aggregateSegmentStats(hoveredIntervalBars); - title = bubbleInfo.timeLabel; - text = ( - <> - {!shownDatasource && ( -
{`Totals for ${pluralIfNeeded(datasources.length, 'datasource')}`}
- )} -
{`Size: ${formatIntervalStat('size', agg.size)}`}
-
{`Rows: ${formatIntervalStat('rows', agg.rows)}`}
-
{`Segments: ${formatIntervalStat('segments', agg.segments)}`}
- - ); - } - - hoveredOpenOn = { - x: - rect.x + - CHART_MARGIN.left + - timeScale(new Date((bubbleInfo.start.valueOf() + bubbleInfo.end.valueOf()) / 2)), - y: rect.y + CHART_MARGIN.top, - title, - text, - }; - } else if (selection) { - const selectedBars = intervalTree.search([ - selection.start.valueOf() + 1, - selection.end.valueOf() - 1, - ]) as IntervalBar[]; - const datasources = uniq(selectedBars.map(b => b.datasource)); - const realtime = allSameValue(selectedBars.map(b => b.realtime)); - const agg = aggregateSegmentStats(selectedBars); - hoveredOpenOn = { - x: - rect.x + - CHART_MARGIN.left + - timeScale(new Date((selection.start.valueOf() + selection.end.valueOf()) / 2)), - y: rect.y + CHART_MARGIN.top, - title: `${formatIsoDateOnly(selection.start)} → ${formatIsoDateOnly(selection.end)}`, - text: ( - <> - {selectedBars.length ? ( - <> - {!shownDatasource && ( -
{`Totals for ${pluralIfNeeded(datasources.length, 'datasource')}`}
- )} -
{`Size: ${formatIntervalStat('size', agg.size)}`}
-
{`Rows: ${formatIntervalStat('rows', agg.rows)}`}
-
{`Segments: ${formatIntervalStat('segments', agg.segments)}`}
- - ) : ( -
No segments in this interval
- )} - {selection.done && ( -
-
- )} - - ), - }; - } - } - - function renderLoadRule(loadRule: Rule, i: number, isDefault: boolean) { - const [start, end] = loadRuleToDateRange(loadRule); - const { x, width } = startEndToXWidth({ start, end }); - const title = RuleUtil.ruleToString(loadRule) + (isDefault ? ' (cluster default)' : ''); - return ( -
- {title} -
- ); - } - - const nowX = timeScale(now); - return ( -
- - - v !== 0)) - .tickSize(-innerStage.width) - .tickFormat(() => '') - .tickSizeOuter(0)} - /> - - - formatTickRate(e.valueOf()))} - /> - - {bubbleInfo && ( - - )} - {0 < nowX && nowX < innerStage.width && ( - - )} - {intervalBars.map((intervalBar, i) => { - return ( - - ); - })} - {bubbleInfo?.intervalBars.length === 1 && - bubbleInfo.intervalBars.map((intervalBar, i) => ( - - ))} - {selection && ( - - )} - {!!shiftOffset && ( - 0 ? timeScale(dateRange[1]) : 0} - y={0} - height={innerStage.height} - width={ - shiftOffset > 0 - ? innerStage.width - timeScale(dateRange[1]) - : timeScale(dateRange[0]) - } - /> - )} - - - - {(datasourceRules || datasourceRulesError) && ( -
- {datasourceRules?.defaultLoadRules.map((rule, index) => - renderLoadRule(rule, index, true), - )} - {datasourceRules?.loadRules.map((rule, index) => renderLoadRule(rule, index, false))} - {datasourceRulesError && ( -
Rule loading error: {datasourceRulesError}
- )} -
- )} - {!intervalRows.length && ( -
-
There are no segments in the selected range
-
- )} - setSelection(undefined) : undefined} - mute - direction="up" - /> -
- ); -}; diff --git a/web-console/src/components/segment-timeline/segment-bar-chart.tsx b/web-console/src/components/segment-timeline/segment-bar-chart.tsx deleted file mode 100644 index 297ea4afe617..000000000000 --- a/web-console/src/components/segment-timeline/segment-bar-chart.tsx +++ /dev/null @@ -1,162 +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 { C, F, L, N, sql, SqlExpression, SqlQuery } from 'druid-query-toolkit'; -import { useMemo } from 'react'; - -import { END_OF_TIME_DATE, type Rule, RuleUtil, START_OF_TIME_DATE } from '../../druid-models'; -import type { Capabilities } from '../../helpers'; -import { useQueryManager } from '../../hooks'; -import { Api } from '../../singletons'; -import { Duration, filterMap, getApiArray, queryDruidSql, TZ_UTC } from '../../utils'; -import { Loader } from '../loader/loader'; - -import type { IntervalRow } from './common'; -import type { SegmentBarChartRenderProps } from './segment-bar-chart-render'; -import { SegmentBarChartRender } from './segment-bar-chart-render'; - -import './segment-bar-chart.scss'; - -export interface SegmentBarChartProps - extends Omit< - SegmentBarChartRenderProps, - 'intervalRows' | 'datasourceRules' | 'datasourceRulesError' - > { - capabilities: Capabilities; -} - -export const SegmentBarChart = function SegmentBarChart(props: SegmentBarChartProps) { - const { capabilities, dateRange, shownDatasource, ...otherProps } = props; - - const intervalsQuery = useMemo( - () => ({ capabilities, dateRange, shownDatasource: shownDatasource }), - [capabilities, dateRange, shownDatasource], - ); - - const [intervalRowsState] = useQueryManager({ - query: intervalsQuery, - processQuery: async ({ capabilities, dateRange, shownDatasource }, cancelToken) => { - if (capabilities.hasSql()) { - const query = SqlQuery.from(N('sys').table('segments')) - .changeWhereExpression( - SqlExpression.and( - sql`"start" <= '${dateRange[1].toISOString()}' AND '${dateRange[0].toISOString()}' < "end"`, - C('start').unequal(START_OF_TIME_DATE), - C('end').unequal(END_OF_TIME_DATE), - C('is_overshadowed').equal(0), - shownDatasource ? C('datasource').equal(L(shownDatasource)) : undefined, - ), - ) - .addSelect(C('start'), { addToGroupBy: 'end' }) - .addSelect(C('end'), { addToGroupBy: 'end' }) - .addSelect(C('datasource'), { addToGroupBy: 'end' }) - .addSelect(C('is_realtime').as('realtime'), { addToGroupBy: 'end' }) - .addSelect(F.count().as('segments')) - .addSelect(F.sum(C('size')).as('size')) - .addSelect(F.sum(C('num_rows')).as('rows')) - .toString(); - - return (await queryDruidSql({ query }, cancelToken)).map(sr => { - const start = new Date(sr.start); - const end = new Date(sr.end); - - return { - ...sr, - start, - end, - realtime: Boolean(sr.realtime), - originalTimeSpan: Duration.fromRange(start, end, TZ_UTC), - } as IntervalRow; - }); - } else { - return filterMap( - await getApiArray( - `/druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&includeRealtimeSegments&${ - shownDatasource ? `datasources=${Api.encodePath(shownDatasource)}` : '' - }`, - cancelToken, - ), - (segment: any) => { - if (segment.overshadowed) return; // We have to include overshadowed segments to get the realtime segments in this API - const [startStr, endStr] = segment.interval.split('/'); - if (startStr === START_OF_TIME_DATE && endStr === END_OF_TIME_DATE) return; - const start = new Date(startStr); - const end = new Date(endStr); - if (!(start <= dateRange[1] && dateRange[0] < end)) return; - - return { - start, - end, - datasource: segment.dataSource, - realtime: Boolean(segment.realtime), - originalTimeSpan: Duration.fromRange(start, end, TZ_UTC), - segments: 1, - size: segment.size, - rows: segment.num_rows || 0, // segment.num_rows is really null on this API :-( - } as IntervalRow; - }, - ); - } - }, - }); - - const [allLoadRulesState] = useQueryManager({ - query: shownDatasource ? '' : undefined, - processQuery: async (_, cancelToken) => { - return ( - await Api.instance.get>('/druid/coordinator/v1/rules', { - cancelToken, - }) - ).data; - }, - }); - - const datasourceRules = useMemo(() => { - const allLoadRules = allLoadRulesState.data; - if (!allLoadRules || !shownDatasource) return; - return { - loadRules: (allLoadRules[shownDatasource] || []).toReversed(), - defaultLoadRules: (allLoadRules[RuleUtil.DEFAULT_RULES_KEY] || []).toReversed(), - }; - }, [allLoadRulesState.data, shownDatasource]); - - if (intervalRowsState.error) { - return ( -
- {`Error when loading data: ${intervalRowsState.getErrorMessage()}`} -
- ); - } - - const intervalRows = intervalRowsState.getSomeData(); - return ( - <> - {intervalRows && ( - - )} - {intervalRowsState.loading && } - - ); -}; diff --git a/web-console/src/components/segment-timeline/segment-timeline.scss b/web-console/src/components/segment-timeline/segment-timeline.scss index 4224c1cab89d..aa437052d617 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.scss +++ b/web-console/src/components/segment-timeline/segment-timeline.scss @@ -16,20 +16,12 @@ * limitations under the License. */ -@import '../../variables'; - .segment-timeline { - .control-bar { - @include card-like; - height: 34px; - display: flex; - align-items: start; - padding: 5px; - gap: 10px; + display: grid; + grid-template-columns: 1fr 220px; - & > .expander { - flex: 1; - } + .loader { + width: 85%; } .loading-error { @@ -39,16 +31,14 @@ transform: translate(-50%, -50%); } - .chart-container { + .no-data-text { position: absolute; - top: 34px; - width: 100%; - bottom: 0; - overflow: hidden; + left: 30vw; + top: 15vh; + font-size: 20px; + } - .segment-bar-chart, - .segment-bar-chart-render { - @include pin-full; - } + .side-control { + padding-top: 20px; } } diff --git a/web-console/src/components/segment-timeline/segment-timeline.spec.tsx b/web-console/src/components/segment-timeline/segment-timeline.spec.tsx index ebc83ae143c7..4f95842801a4 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.spec.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.spec.tsx @@ -16,42 +16,40 @@ * limitations under the License. */ +import { sane } from '@druid-toolkit/query'; import { render } from '@testing-library/react'; import { Capabilities } from '../../helpers'; -import { QueryState } from '../../utils'; import { SegmentTimeline } from './segment-timeline'; -jest.useFakeTimers('modern').setSystemTime(Date.parse('2024-11-19T12:34:56Z')); - -jest.mock('../../hooks', () => { - return { - useQueryManager: (options: any) => { - if (options.initQuery instanceof Capabilities) { - // This is a query for data sources - return [new QueryState({ data: ['ds1', 'ds2'] })]; - } - - if (options.query === null) { - // This is a query for the data source time range - return [ - new QueryState({ - data: [new Date('2024-11-01 00:00:00Z'), new Date('2024-11-18 00:00:00Z')], - }), - ]; - } - - return new QueryState({ error: new Error('not covered') }); - }, - }; -}); +jest.useFakeTimers('modern').setSystemTime(Date.parse('2021-06-08T12:34:56Z')); describe('SegmentTimeline', () => { + it('.getSqlQuery', () => { + expect( + SegmentTimeline.getSqlQuery([ + new Date('2020-01-01T00:00:00Z'), + new Date('2021-02-01T00:00:00Z'), + ]), + ).toEqual(sane` + SELECT + "start", "end", "datasource", + COUNT(*) AS "count", + SUM("size") AS "size" + FROM sys.segments + WHERE + '2020-01-01T00:00:00.000Z' <= "start" AND + "end" <= '2021-02-01T00:00:00.000Z' AND + is_published = 1 AND + is_overshadowed = 0 + GROUP BY 1, 2, 3 + ORDER BY "start" DESC + `); + }); + it('matches snapshot', () => { - const segmentTimeline = ( - - ); + const segmentTimeline = ; const { container } = render(segmentTimeline); expect(container.firstChild).toMatchSnapshot(); }); diff --git a/web-console/src/components/segment-timeline/segment-timeline.tsx b/web-console/src/components/segment-timeline/segment-timeline.tsx index 2774c73276eb..8aee0c66d477 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.tsx @@ -16,361 +16,628 @@ * limitations under the License. */ -import { - Button, - ButtonGroup, - Intent, - Menu, - MenuItem, - Popover, - Position, - ResizeSensor, -} from '@blueprintjs/core'; -import type { NonNullDateRange } from '@blueprintjs/datetime'; -import { DateRangePicker3 } from '@blueprintjs/datetime2'; +import { Button, FormGroup, MenuItem, ResizeSensor, SegmentedControl } from '@blueprintjs/core'; +import type { DateRange, NonNullDateRange } from '@blueprintjs/datetime'; +import { DateRangeInput3 } from '@blueprintjs/datetime2'; import { IconNames } from '@blueprintjs/icons'; +import type { ItemPredicate, ItemRenderer } from '@blueprintjs/select'; import { Select } from '@blueprintjs/select'; -import { C, L, N, SqlExpression, SqlQuery } from 'druid-query-toolkit'; -import { useEffect, useMemo, useState } from 'react'; +import type { AxisScale } from 'd3-axis'; +import { scaleLinear, scaleUtc } from 'd3-scale'; +import enUS from 'date-fns/locale/en-US'; +import React from 'react'; -import { END_OF_TIME_DATE, START_OF_TIME_DATE } from '../../druid-models'; import type { Capabilities } from '../../helpers'; -import { useQueryManager } from '../../hooks'; +import { Api } from '../../singletons'; import { - checkedCircleIcon, - day, - Duration, - getApiArray, + ceilToUtcDay, + formatBytes, + formatInteger, isNonNullRange, localToUtcDateRange, - maxDate, queryDruidSql, - TZ_UTC, + QueryManager, + uniq, utcToLocalDateRange, } from '../../utils'; -import { Stage } from '../../utils/stage'; import { Loader } from '../loader/loader'; -import type { IntervalStat } from './common'; -import { formatIsoDateOnly, getIntervalStatTitle, INTERVAL_STATS } from './common'; -import type { SegmentBarChartProps } from './segment-bar-chart'; -import { SegmentBarChart } from './segment-bar-chart'; +import type { BarUnitData } from './stacked-bar-chart'; +import { StackedBarChart } from './stacked-bar-chart'; import './segment-timeline.scss'; -const DEFAULT_SHOWN_DURATION = new Duration('P1Y'); -const SHOWN_DURATION_OPTIONS: Duration[] = [ - new Duration('P1D'), - new Duration('P1W'), - new Duration('P1M'), - new Duration('P3M'), - new Duration('P1Y'), - new Duration('P5Y'), - new Duration('P10Y'), -]; - -function getDateRange(shownDuration: Duration): NonNullDateRange { - const end = day.ceil(new Date(), TZ_UTC); - return [shownDuration.shift(end, TZ_UTC, -1), end]; +interface SegmentTimelineProps { + capabilities: Capabilities; } -function formatDateRange(dateRange: NonNullDateRange): string { - return `${formatIsoDateOnly(dateRange[0])} → ${formatIsoDateOnly(dateRange[1])}`; +type ActiveDataType = 'sizeData' | 'countData'; + +interface SegmentTimelineState { + chartHeight: number; + chartWidth: number; + data?: Record; + datasources: string[]; + stackedData?: Record; + singleDatasourceData?: Record>; + activeDatasource: string | null; + activeDataType: ActiveDataType; + dataToRender: BarUnitData[]; + loading: boolean; + error?: Error; + xScale: AxisScale | null; + yScale: AxisScale | null; + dateRange: NonNullDateRange; + selectedDateRange?: DateRange; } -function dateRangesEqual(dr1: NonNullDateRange, dr2: NonNullDateRange): boolean { - return dr1[0].valueOf() === dr2[0].valueOf() && dr2[1].valueOf() === dr2[1].valueOf(); +interface BarChartScales { + xScale: AxisScale; + yScale: AxisScale; } -interface SegmentTimelineProps extends Pick { - capabilities: Capabilities; - datasource: string | undefined; +interface IntervalRow { + start: string; + end: string; + datasource: string; + count: number; + size: number; } -export const SegmentTimeline = function SegmentTimeline(props: SegmentTimelineProps) { - const { capabilities, datasource, ...otherProps } = props; - const [stage, setStage] = useState(); - const [activeSegmentStat, setActiveSegmentStat] = useState('size'); - const [shownDatasource, setShownDatasource] = useState(datasource); - const [dateRange, setDateRange] = useState(); - const [showCustomDatePicker, setShowCustomDatePicker] = useState(false); - - useEffect(() => { - setShownDatasource(datasource); - }, [datasource]); - - const defaultDateRange = useMemo(() => { - return getDateRange(DEFAULT_SHOWN_DURATION); - }, []); - - const [datasourcesState] = useQueryManager({ - initQuery: capabilities, - processQuery: async (capabilities, cancelToken) => { - if (capabilities.hasSql()) { - const tables = await queryDruidSql<{ TABLE_NAME: string }>( - { - query: `SELECT TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE = 'TABLE'`, - }, - cancelToken, - ); +const DEFAULT_TIME_SPAN_MONTHS = 3; + +function getDefaultDateRange(): NonNullDateRange { + const start = ceilToUtcDay(new Date()); + const end = new Date(start.valueOf()); + start.setUTCMonth(start.getUTCMonth() - DEFAULT_TIME_SPAN_MONTHS); + return [start, end]; +} + +export class SegmentTimeline extends React.PureComponent< + SegmentTimelineProps, + SegmentTimelineState +> { + static COLORS = [ + '#b33040', + '#d25c4d', + '#f2b447', + '#d9d574', + '#4FAA7E', + '#57ceff', + '#789113', + '#098777', + '#b33040', + '#d2757b', + '#f29063', + '#d9a241', + '#80aa61', + '#c4ff9e', + '#915412', + '#87606c', + ]; + + static getColor(index: number): string { + return SegmentTimeline.COLORS[index % SegmentTimeline.COLORS.length]; + } + + static getSqlQuery(dateRange: NonNullDateRange): string { + return `SELECT + "start", "end", "datasource", + COUNT(*) AS "count", + SUM("size") AS "size" +FROM sys.segments +WHERE + '${dateRange[0].toISOString()}' <= "start" AND + "end" <= '${dateRange[1].toISOString()}' AND + is_published = 1 AND + is_overshadowed = 0 +GROUP BY 1, 2, 3 +ORDER BY "start" DESC`; + } + + static processRawData(data: IntervalRow[]) { + if (data === null) return []; - return tables.map(d => d.TABLE_NAME); + const countData: Record = {}; + const sizeData: Record = {}; + data.forEach(entry => { + const start = entry.start; + const day = start.split('T')[0]; + const datasource = entry.datasource; + const count = entry.count; + const segmentSize = entry.size; + if (countData[day] === undefined) { + countData[day] = { + day, + [datasource]: count, + total: count, + }; + sizeData[day] = { + day, + [datasource]: segmentSize, + total: segmentSize, + }; } else { - return await getApiArray(`/druid/coordinator/v1/datasources`, cancelToken); + const countDataEntry: number | undefined = countData[day][datasource]; + countData[day][datasource] = count + (countDataEntry === undefined ? 0 : countDataEntry); + const sizeDataEntry: number | undefined = sizeData[day][datasource]; + sizeData[day][datasource] = segmentSize + (sizeDataEntry === undefined ? 0 : sizeDataEntry); + countData[day].total += count; + sizeData[day].total += segmentSize; } - }, - }); - - const [initDatasourceDateRangeState] = useQueryManager({ - query: dateRange ? undefined : shownDatasource ?? null, - processQuery: async (datasource, cancelToken) => { - let queriedStart: Date; - let queriedEnd: Date; - if (capabilities.hasSql()) { - const baseQuery = SqlQuery.from(N('sys').table('segments')) - .changeWhereExpression( - SqlExpression.and( - C('start').unequal(START_OF_TIME_DATE), - C('end').unequal(END_OF_TIME_DATE), - C('is_overshadowed').equal(0), - datasource ? C('datasource').equal(L(datasource)) : undefined, - ), - ) - .changeLimitValue(1); + }); - const endQuery = baseQuery - .addSelect(C('end'), { addToOrderBy: 'end', direction: 'DESC' }) - .toString(); + const countDataArray = Object.keys(countData) + .reverse() + .map((time: any) => { + return countData[time]; + }); - const endRes = await queryDruidSql<{ end: string }>({ query: endQuery }, cancelToken).catch( - () => [], - ); - if (endRes.length !== 1) { - return getDateRange(DEFAULT_SHOWN_DURATION); + const sizeDataArray = Object.keys(sizeData) + .reverse() + .map((time: any) => { + return sizeData[time]; + }); + + return { countData: countDataArray, sizeData: sizeDataArray }; + } + + static calculateStackedData( + data: Record, + datasources: string[], + ): Record { + const newStackedData: Record = {}; + Object.keys(data).forEach((type: any) => { + const stackedData: any = data[type].map((d: any) => { + let y0 = 0; + return datasources.map((datasource: string, i) => { + const barUnitData = { + x: d.day, + y: d[datasource] === undefined ? 0 : d[datasource], + y0, + datasource, + color: SegmentTimeline.getColor(i), + dailySize: d.total, + }; + y0 += d[datasource] === undefined ? 0 : d[datasource]; + return barUnitData; + }); + }); + newStackedData[type] = stackedData.flat(); + }); + + return newStackedData; + } + + static calculateSingleDatasourceData( + data: Record, + datasources: string[], + ): Record> { + const singleDatasourceData: Record> = {}; + Object.keys(data).forEach(dataType => { + singleDatasourceData[dataType] = {}; + datasources.forEach((datasource, i) => { + const currentData = data[dataType]; + if (currentData.length === 0) return; + const dataResult = currentData.map((d: any) => { + let y = 0; + if (d[datasource] !== undefined) { + y = d[datasource]; + } + return { + x: d.day, + y, + datasource, + color: SegmentTimeline.getColor(i), + dailySize: d.total, + }; + }); + if (!dataResult.every((d: any) => d.y === 0)) { + singleDatasourceData[dataType][datasource] = dataResult; } + }); + }); - queriedEnd = day.ceil(new Date(endRes[0].end), TZ_UTC); + return singleDatasourceData; + } + + private readonly dataQueryManager: QueryManager< + { capabilities: Capabilities; dateRange: NonNullDateRange }, + any + >; + + private readonly chartMargin = { top: 40, right: 15, bottom: 20, left: 60 }; + + constructor(props: SegmentTimelineProps) { + super(props); + const dateRange = getDefaultDateRange(); + + this.state = { + chartWidth: 1, // Dummy init values to be replaced + chartHeight: 1, // after first render + data: {}, + datasources: [], + stackedData: {}, + singleDatasourceData: {}, + dataToRender: [], + activeDatasource: null, + activeDataType: 'sizeData', + loading: true, + xScale: null, + yScale: null, + dateRange, + }; + + this.dataQueryManager = new QueryManager({ + processQuery: async ({ capabilities, dateRange }, cancelToken) => { + let intervals: IntervalRow[]; + let datasources: string[]; + if (capabilities.hasSql()) { + intervals = await queryDruidSql( + { + query: SegmentTimeline.getSqlQuery(dateRange), + }, + cancelToken, + ); + datasources = uniq(intervals.map(r => r.datasource).sort()); + } else if (capabilities.hasCoordinatorAccess()) { + const startIso = dateRange[0].toISOString(); - const startQuery = baseQuery - .addSelect(C('start'), { addToOrderBy: 'end', direction: 'ASC' }) - .toString(); + datasources = ( + await Api.instance.get(`/druid/coordinator/v1/datasources`, { cancelToken }) + ).data; + intervals = ( + await Promise.all( + datasources.map(async datasource => { + const intervalMap = ( + await Api.instance.get( + `/druid/coordinator/v1/datasources/${Api.encodePath( + datasource, + )}/intervals?simple`, + { cancelToken }, + ) + ).data; - const startRes = await queryDruidSql<{ start: string }>( - { query: startQuery }, - cancelToken, - ).catch(() => []); - if (startRes.length !== 1) { - return [DEFAULT_SHOWN_DURATION.shift(queriedEnd, TZ_UTC, -1), queriedEnd]; // Should not really get here + return Object.keys(intervalMap) + .map(interval => { + const [start, end] = interval.split('/'); + const { count, size } = intervalMap[interval]; + return { + start, + end, + datasource, + count, + size, + }; + }) + .filter(a => startIso < a.start); + }), + ) + ) + .flat() + .sort((a, b) => b.start.localeCompare(a.start)); + } else { + throw new Error(`must have SQL or coordinator access`); } - queriedStart = day.floor(new Date(startRes[0].start), TZ_UTC); - } else { - // Don't bother querying if there is no SQL - return getDateRange(DEFAULT_SHOWN_DURATION); + const data = SegmentTimeline.processRawData(intervals); + const stackedData = SegmentTimeline.calculateStackedData(data, datasources); + const singleDatasourceData = SegmentTimeline.calculateSingleDatasourceData( + data, + datasources, + ); + return { data, datasources, stackedData, singleDatasourceData }; + }, + onStateChange: ({ data, loading, error }) => { + this.setState({ + data: data ? data.data : undefined, + datasources: data ? data.datasources : [], + stackedData: data ? data.stackedData : undefined, + singleDatasourceData: data ? data.singleDatasourceData : undefined, + loading, + error, + }); + }, + }); + } + + componentDidMount(): void { + const { capabilities } = this.props; + const { dateRange } = this.state; + + if (isNonNullRange(dateRange)) { + this.dataQueryManager.runQuery({ capabilities, dateRange }); + } + } + + componentWillUnmount(): void { + this.dataQueryManager.terminate(); + } + + componentDidUpdate(_prevProps: SegmentTimelineProps, prevState: SegmentTimelineState): void { + const { activeDatasource, activeDataType, singleDatasourceData, stackedData } = this.state; + if ( + prevState.data !== this.state.data || + prevState.activeDataType !== this.state.activeDataType || + prevState.activeDatasource !== this.state.activeDatasource || + prevState.chartWidth !== this.state.chartWidth || + prevState.chartHeight !== this.state.chartHeight + ) { + const scales: BarChartScales | undefined = this.calculateScales(); + const dataToRender: BarUnitData[] | undefined = activeDatasource + ? singleDatasourceData + ? singleDatasourceData[activeDataType][activeDatasource] + : undefined + : stackedData + ? stackedData[activeDataType] + : undefined; + + if (scales && dataToRender) { + this.setState({ + dataToRender, + xScale: scales.xScale, + yScale: scales.yScale, + }); } + } + } + + private calculateScales(): BarChartScales | undefined { + const { + chartWidth, + chartHeight, + data, + activeDataType, + activeDatasource, + singleDatasourceData, + dateRange, + } = this.state; + if (!data || !Object.keys(data).length || !isNonNullRange(dateRange)) return; + const activeData = data[activeDataType]; - return [ - maxDate(queriedStart, DEFAULT_SHOWN_DURATION.shift(queriedEnd, TZ_UTC, -1)), - queriedEnd, + let yDomain: number[] = [ + 0, + activeData.length === 0 + ? 0 + : activeData.reduce((max: any, d: any) => (max.total > d.total ? max : d)).total, + ]; + + if ( + activeDatasource !== null && + singleDatasourceData![activeDataType][activeDatasource] !== undefined + ) { + yDomain = [ + 0, + singleDatasourceData![activeDataType][activeDatasource].reduce((max: any, d: any) => + max.y > d.y ? max : d, + ).y, ]; - }, - }); - - const effectiveDateRange = - dateRange || - initDatasourceDateRangeState.data || - (initDatasourceDateRangeState.isLoading() ? undefined : defaultDateRange); - - let previousDateRange: NonNullDateRange | undefined; - let zoomedOutDateRange: NonNullDateRange | undefined; - let nextDateRange: NonNullDateRange | undefined; - if (effectiveDateRange) { - const d = Duration.fromRange(effectiveDateRange[0], effectiveDateRange[1], TZ_UTC); - const shiftStartBack = d.shift(effectiveDateRange[0], TZ_UTC, -1); - const shiftEndForward = d.shift(effectiveDateRange[1], TZ_UTC); - const now = day.ceil(new Date(), TZ_UTC); - previousDateRange = [shiftStartBack, effectiveDateRange[0]]; - zoomedOutDateRange = [shiftStartBack, shiftEndForward < now ? shiftEndForward : now]; - nextDateRange = [effectiveDateRange[1], shiftEndForward]; + } + + const xScale: AxisScale = scaleUtc() + .domain(dateRange) + .range([0, chartWidth - this.chartMargin.left - this.chartMargin.right]); + + const yScale: AxisScale = scaleLinear() + .rangeRound([chartHeight - this.chartMargin.top - this.chartMargin.bottom, 0]) + .domain(yDomain); + + return { + xScale, + yScale, + }; } - return ( -
-
- - - items={datasourcesState.data || []} - disabled={datasourcesState.isError()} - onItemSelect={setShownDatasource} - itemRenderer={(val, { handleClick, handleFocus, modifiers }) => { - if (!modifiers.matchesPredicate) return null; - return ( - - ); - }} - noResults={} - itemPredicate={(query, val, _index, exactMatch) => { - const normalizedTitle = val.toLowerCase(); - const normalizedQuery = query.toLowerCase(); - - if (exactMatch) { - return normalizedTitle === normalizedQuery; - } else { - return normalizedTitle.includes(normalizedQuery); - } - }} - > -
- { - const rect = entries[0].contentRect; - setStage(new Stage(rect.width, rect.height)); - }} - > -
- {stage && effectiveDateRange && ( - + + { + const newUtcDateRange = localToUtcDateRange(newDateRange); + if (!isNonNullRange(newUtcDateRange)) return; + this.setState({ dateRange: newUtcDateRange, selectedDateRange: undefined }, () => { + this.dataQueryManager.runQuery({ capabilities, dateRange: newUtcDateRange }); + }); + }} + fill + locale={enUS} /> - )} - {initDatasourceDateRangeState.isLoading() && } + + + +
-
-
- ); -}; +
+ ); + } +} diff --git a/web-console/src/druid-models/datasource/datasource.ts b/web-console/src/components/segment-timeline/stacked-bar-chart.scss similarity index 64% rename from web-console/src/druid-models/datasource/datasource.ts rename to web-console/src/components/segment-timeline/stacked-bar-chart.scss index 313d711b35c2..26e5f5186b5f 100644 --- a/web-console/src/druid-models/datasource/datasource.ts +++ b/web-console/src/components/segment-timeline/stacked-bar-chart.scss @@ -16,32 +16,35 @@ * limitations under the License. */ -import { hashJoaat } from '../../utils'; +.stacked-bar-chart { + position: relative; + overflow: hidden; -const COLORS = [ - '#1f77b4', - '#aec7e8', - '#ff7f0e', - '#ffbb78', - '#2ca02c', - '#98df8a', - '#d62728', - '#ff9896', - '#9467bd', - '#c5b0d5', - '#8c564b', - '#c49c94', - '#e377c2', - '#f7b6d2', - '#7f7f7f', - '#c7c7c7', - '#bcbd22', - '#dbdb8d', - '#17becf', - '#9edae5', -]; + .bar-chart-tooltip { + position: absolute; + left: 100px; + right: 0; -export function getDatasourceColor(datasource: string) { - const hash = hashJoaat(datasource); - return COLORS[hash % COLORS.length]; + div { + display: inline-block; + width: 230px; + } + } + + svg { + position: absolute; + + .hovered-bar { + fill: transparent; + stroke: #ffffff; + stroke-width: 1.5px; + } + + .gridline-x { + line { + stroke-dasharray: 5, 5; + opacity: 0.5; + } + } + } } diff --git a/web-console/src/components/segment-timeline/stacked-bar-chart.tsx b/web-console/src/components/segment-timeline/stacked-bar-chart.tsx new file mode 100644 index 000000000000..8018aaee5f6f --- /dev/null +++ b/web-console/src/components/segment-timeline/stacked-bar-chart.tsx @@ -0,0 +1,174 @@ +/* + * 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 { AxisScale } from 'd3-axis'; +import { axisBottom, axisLeft } from 'd3-axis'; +import React, { useState } from 'react'; + +import { BarGroup } from './bar-group'; +import { ChartAxis } from './chart-axis'; + +import './stacked-bar-chart.scss'; + +export interface BarUnitData { + x: number; + y: number; + y0?: number; + width: number; + datasource: string; + color: string; + dailySize?: number; +} + +export interface BarChartMargin { + top: number; + right: number; + bottom: number; + left: number; +} + +export interface HoveredBarInfo { + xCoordinate?: number; + yCoordinate?: number; + height?: number; + width?: number; + datasource?: string; + xValue?: number; + yValue?: number; + dailySize?: number; +} + +interface StackedBarChartProps { + svgWidth: number; + svgHeight: number; + margin: BarChartMargin; + activeDataType?: string; + dataToRender: BarUnitData[]; + changeActiveDatasource: (e: string | null) => void; + formatTick: (e: number) => string; + xScale: AxisScale; + yScale: AxisScale; + barWidth: number; +} + +export const StackedBarChart = React.forwardRef(function StackedBarChart( + props: StackedBarChartProps, + ref, +) { + const { + activeDataType, + svgWidth, + svgHeight, + margin, + formatTick, + xScale, + yScale, + dataToRender, + changeActiveDatasource, + barWidth, + } = props; + const [hoverOn, setHoverOn] = useState(); + + const width = svgWidth - margin.left - margin.right; + const height = svgHeight - margin.top - margin.bottom; + + function renderBarChart() { + return ( + + setHoverOn(undefined)} + > + '') + .tickSizeOuter(0)} + /> + setHoverOn(e)} + hoverOn={hoverOn} + barWidth={barWidth} + /> + + formatTick(e))} + /> + {hoverOn && ( + { + setHoverOn(undefined); + changeActiveDatasource(hoverOn.datasource ?? null); + }} + > + + + )} + + + ); + } + + return ( +
+ {hoverOn && ( +
+
Datasource: {hoverOn.datasource}
+
Time: {hoverOn.xValue}
+
+ {`${ + activeDataType === 'countData' ? 'Daily total count:' : 'Daily total size:' + } ${formatTick(hoverOn.dailySize!)}`} +
+
+ {`${activeDataType === 'countData' ? 'Count:' : 'Size:'} ${formatTick( + hoverOn.yValue!, + )}`} +
+
+ )} + {renderBarChart()} +
+ ); +}); diff --git a/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx b/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx index a7fa38fd445c..74cb55682f30 100644 --- a/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx +++ b/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx @@ -25,7 +25,7 @@ import type { IngestionSpec } from '../../druid-models'; import { cleanSpec } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { Api } from '../../singletons'; -import { deepSet, getApiArray } from '../../utils'; +import { deepSet } from '../../utils'; import { Loader } from '../loader/loader'; import { ShowValue } from '../show-value/show-value'; @@ -49,12 +49,11 @@ export const SupervisorHistoryPanel = React.memo(function SupervisorHistoryPanel const [historyState] = useQueryManager({ initQuery: supervisorId, processQuery: async (supervisorId, cancelToken) => { - return ( - await getApiArray( - `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/history`, - cancelToken, - ) - ).map(vs => deepSet(vs, 'spec', cleanSpec(vs.spec))); + const resp = await Api.instance.get( + `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/history`, + { cancelToken }, + ); + return resp.data.map((vs: SupervisorHistoryEntry) => deepSet(vs, 'spec', cleanSpec(vs.spec))); }, }); diff --git a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx index 38c69e297e0c..aec706a50fc0 100644 --- a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx +++ b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx @@ -28,7 +28,7 @@ export interface TableClickableCellProps { className?: string; onClick: MouseEventHandler; hoverIcon?: IconName; - tooltip?: string; + title?: string; disabled?: boolean; children?: ReactNode; } @@ -36,13 +36,12 @@ export interface TableClickableCellProps { export const TableClickableCell = React.memo(function TableClickableCell( props: TableClickableCellProps, ) { - const { className, onClick, hoverIcon, disabled, children, tooltip, ...rest } = props; + const { className, onClick, hoverIcon, disabled, children, ...rest } = props; return (
{children} diff --git a/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx b/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx index c03f0038cc7f..631fa224aaf4 100644 --- a/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx +++ b/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx @@ -27,7 +27,7 @@ import { Deferred } from '../deferred/deferred'; import './table-filterable-cell.scss'; -const FILTER_MODES: FilterMode[] = ['=', '!=', '<', '>=']; +const FILTER_MODES: FilterMode[] = ['=', '!=', '<=', '>=']; const FILTER_MODES_NO_COMPARISONS: FilterMode[] = ['=', '!=']; export interface TableFilterableCellProps { diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 5dfbba77101a..95336713dbb6 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -182,24 +182,12 @@ export class ConsoleApplication extends React.PureComponent< changeTabWithFilter('datasources', [{ id: 'datasource', value: `=${datasource}` }]); }; - private readonly goToSegments = ({ - start, - end, - datasource, - realtime, - }: { - start?: Date; - end?: Date; - datasource?: string; - realtime?: boolean; - }) => { + private readonly goToSegments = (datasource: string, onlyUnavailable = false) => { changeTabWithFilter( 'segments', compact([ - start && { id: 'start', value: `>=${start.toISOString()}` }, - end && { id: 'end', value: `<${end.toISOString()}` }, - datasource && { id: 'datasource', value: `=${datasource}` }, - typeof realtime === 'boolean' ? { id: 'is_realtime', value: `=${realtime}` } : undefined, + { id: 'datasource', value: `=${datasource}` }, + onlyUnavailable ? { id: 'is_available', value: '=false' } : undefined, ]), ); }; diff --git a/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx b/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx index 8ee0ffdbfbc3..9e19e043c713 100644 --- a/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx +++ b/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx @@ -24,7 +24,7 @@ import { Loader, ShowValue } from '../../components'; import type { CompactionConfig } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { Api } from '../../singletons'; -import { formatInteger, formatPercent, getApiArray } from '../../utils'; +import { formatInteger, formatPercent } from '../../utils'; import { DiffDialog } from '../diff-dialog/diff-dialog'; import './compaction-history-dialog.scss'; @@ -65,10 +65,11 @@ export const CompactionHistoryDialog = React.memo(function CompactionHistoryDial initQuery: datasource, processQuery: async (datasource, cancelToken) => { try { - return await getApiArray( + const resp = await Api.instance.get( `/druid/coordinator/v1/config/compaction/${Api.encodePath(datasource)}/history?count=20`, - cancelToken, + { cancelToken }, ); + return resp.data; } catch (e) { if (e.response?.status === 404) return []; throw e; diff --git a/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx b/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx index ab4fed8ae1ad..ec964f5507ee 100644 --- a/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx @@ -27,7 +27,7 @@ import { COORDINATOR_DYNAMIC_CONFIG_FIELDS } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { getLink } from '../../links'; import { Api, AppToaster } from '../../singletons'; -import { getApiArray, getDruidErrorMessage } from '../../utils'; +import { getDruidErrorMessage } from '../../utils'; import { SnitchDialog } from '..'; import './coordinator-dynamic-config-dialog.scss'; @@ -47,7 +47,10 @@ export const CoordinatorDynamicConfigDialog = React.memo(function CoordinatorDyn const [historyRecordsState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - return await getApiArray(`/druid/coordinator/v1/config/history?count=100`, cancelToken); + const historyResp = await Api.instance.get(`/druid/coordinator/v1/config/history?count=100`, { + cancelToken, + }); + return historyResp.data; }, }); diff --git a/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx b/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx index 3dfb2f7879d7..82187c866f44 100644 --- a/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx +++ b/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { L } from 'druid-query-toolkit'; +import { L } from '@druid-toolkit/query'; import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx b/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx index c48cbfa51989..1ca92787c1cb 100644 --- a/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx +++ b/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from 'druid-query-toolkit'; -import { QueryRunner, T } from 'druid-query-toolkit'; +import type { QueryResult } from '@druid-toolkit/query'; +import { QueryRunner, T } from '@druid-toolkit/query'; import React from 'react'; import { Loader, RecordTablePane } from '../../../components'; diff --git a/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx b/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx index 122cc4566b85..c36445d17731 100644 --- a/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx +++ b/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { N } from 'druid-query-toolkit'; +import { N } from '@druid-toolkit/query'; import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx b/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx index 5b1233c6384e..ba30118b0a32 100644 --- a/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx @@ -27,7 +27,7 @@ import { OVERLORD_DYNAMIC_CONFIG_FIELDS } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { getLink } from '../../links'; import { Api, AppToaster } from '../../singletons'; -import { getApiArray, getDruidErrorMessage } from '../../utils'; +import { getDruidErrorMessage } from '../../utils'; import { SnitchDialog } from '..'; import './overlord-dynamic-config-dialog.scss'; @@ -47,7 +47,10 @@ export const OverlordDynamicConfigDialog = React.memo(function OverlordDynamicCo const [historyRecordsState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - return await getApiArray(`/druid/indexer/v1/worker/history?count=100`, cancelToken); + const historyResp = await Api.instance.get(`/druid/indexer/v1/worker/history?count=100`, { + cancelToken, + }); + return historyResp.data; }, }); diff --git a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap index 8bba057fac2d..19c90c7de335 100644 --- a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap +++ b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap @@ -188,7 +188,7 @@ exports[`RetentionDialog matches snapshot 1`] = ` style="transform: translateY(0); transition: none;" >
d.tier); } else if (capabilities.hasCoordinatorAccess()) { - return filterMap( - await getApiArray('/druid/coordinator/v1/servers?simple', cancelToken), - (s: any) => (s.type === 'historical' ? s.tier : undefined), + const allServiceResp = await Api.instance.get('/druid/coordinator/v1/servers?simple', { + cancelToken, + }); + return filterMap(allServiceResp.data, (s: any) => + s.type === 'historical' ? s.tier : undefined, ); } else { throw new Error(`must have sql or coordinator access`); @@ -82,10 +84,11 @@ ORDER BY 1`, const [historyQueryState] = useQueryManager({ initQuery: props.datasource, processQuery: async (datasource, cancelToken) => { - return await getApiArray( + const historyResp = await Api.instance.get( `/druid/coordinator/v1/rules/${Api.encodePath(datasource)}/history?count=200`, - cancelToken, + { cancelToken }, ); + return historyResp.data; }, }); diff --git a/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx b/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx index 2925482d9901..4c2487ce0cc6 100644 --- a/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx +++ b/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from 'druid-query-toolkit'; -import { QueryRunner } from 'druid-query-toolkit'; +import type { QueryResult } from '@druid-toolkit/query'; +import { QueryRunner } from '@druid-toolkit/query'; import React from 'react'; import { Loader, RecordTablePane } from '../../../components'; diff --git a/web-console/src/dialogs/status-dialog/status-dialog.tsx b/web-console/src/dialogs/status-dialog/status-dialog.tsx index 672fc40910f3..311f3e056646 100644 --- a/web-console/src/dialogs/status-dialog/status-dialog.tsx +++ b/web-console/src/dialogs/status-dialog/status-dialog.tsx @@ -50,7 +50,8 @@ export const StatusDialog = React.memo(function StatusDialog(props: StatusDialog const [responseState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - return (await Api.instance.get(`/status`, { cancelToken })).data; + const resp = await Api.instance.get(`/status`, { cancelToken }); + return resp.data; }, }); diff --git a/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx b/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx index d10fc00eb5ce..009d83260604 100644 --- a/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx +++ b/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx @@ -106,12 +106,11 @@ export const SupervisorResetOffsetsDialog = React.memo(function SupervisorResetO const [statusResp] = useQueryManager({ initQuery: supervisorId, processQuery: async (supervisorId, cancelToken) => { - return ( - await Api.instance.get( - `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/status`, - { cancelToken }, - ) - ).data; + const statusResp = await Api.instance.get( + `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/status`, + { cancelToken }, + ); + return statusResp.data; }, }); diff --git a/web-console/src/druid-models/execution/execution.ts b/web-console/src/druid-models/execution/execution.ts index 1237b49e62da..f4a73eb97207 100644 --- a/web-console/src/druid-models/execution/execution.ts +++ b/web-console/src/druid-models/execution/execution.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { Column, QueryResult, SqlExpression, SqlQuery, SqlWithQuery } from 'druid-query-toolkit'; +import { Column, QueryResult, SqlExpression, SqlQuery, SqlWithQuery } from '@druid-toolkit/query'; import { maybeGetClusterCapacity } from '../../helpers'; import { diff --git a/web-console/src/druid-models/external-config/external-config.ts b/web-console/src/druid-models/external-config/external-config.ts index 6fa74eb038bd..d6541747ce5a 100644 --- a/web-console/src/druid-models/external-config/external-config.ts +++ b/web-console/src/druid-models/external-config/external-config.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { SqlQuery } from 'druid-query-toolkit'; +import type { SqlQuery } from '@druid-toolkit/query'; import { C, F, @@ -28,7 +28,7 @@ import { SqlLiteral, SqlStar, SqlType, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import * as JSONBig from 'json-bigint-native'; import { nonEmptyArray } from '../../utils'; diff --git a/web-console/src/druid-models/index.ts b/web-console/src/druid-models/index.ts index e31eedeea738..dfeeeeaac837 100644 --- a/web-console/src/druid-models/index.ts +++ b/web-console/src/druid-models/index.ts @@ -21,7 +21,6 @@ export * from './compaction-config/compaction-config'; export * from './compaction-status/compaction-status'; export * from './coordinator-dynamic-config/coordinator-dynamic-config'; export * from './dart/dart-query-entry'; -export * from './datasource/datasource'; export * from './dimension-spec/dimension-spec'; export * from './druid-engine/druid-engine'; export * from './execution/execution'; @@ -33,12 +32,10 @@ export * from './ingest-query-pattern/ingest-query-pattern'; export * from './ingestion-spec/ingestion-spec'; export * from './input-format/input-format'; export * from './input-source/input-source'; -export * from './load-rule/load-rule'; export * from './lookup-spec/lookup-spec'; export * from './metric-spec/metric-spec'; export * from './overlord-dynamic-config/overlord-dynamic-config'; export * from './query-context/query-context'; -export * from './segment/segment'; export * from './stages/stages'; export * from './supervisor-status/supervisor-status'; export * from './task/task'; diff --git a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts index cf108c2b3890..996385fc08ff 100644 --- a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts +++ b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane, SqlQuery } from 'druid-query-toolkit'; +import { sane, SqlQuery } from '@druid-toolkit/query'; import { fitIngestQueryPattern, ingestQueryPatternToQuery } from './ingest-query-pattern'; diff --git a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts index d2009ad2e5ee..f0ce8a0eb288 100644 --- a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts +++ b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts @@ -26,7 +26,7 @@ import { SqlTable, SqlWithPart, T, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import { filterMap, oneOf } from '../../utils'; import type { ExternalConfig } from '../external-config/external-config'; diff --git a/web-console/src/druid-models/segment/segment.ts b/web-console/src/druid-models/segment/segment.ts deleted file mode 100644 index 1bff0711fcdf..000000000000 --- a/web-console/src/druid-models/segment/segment.ts +++ /dev/null @@ -1,50 +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 { Duration } from '../../utils'; - -export const START_OF_TIME_DATE = '-146136543-09-08T08:23:32.096Z'; -export const END_OF_TIME_DATE = '146140482-04-24T15:36:27.903Z'; - -export function computeSegmentTimeSpan(start: string, end: string): string { - if (start === START_OF_TIME_DATE && end === END_OF_TIME_DATE) { - return 'All'; - } - - const startDate = new Date(start); - if (isNaN(startDate.valueOf())) { - return 'Invalid start'; - } - - const endDate = new Date(end); - if (isNaN(endDate.valueOf())) { - return 'Invalid end'; - } - - return Duration.fromRange(startDate, endDate, 'Etc/UTC').getDescription(true); -} - -export interface ShardSpec { - type: string; - partitionNum?: number; - partitions?: number; - dimensions?: string[]; - partitionDimensions?: string[]; - start?: string[]; - end?: string[]; -} diff --git a/web-console/src/druid-models/task/task.ts b/web-console/src/druid-models/task/task.ts index e1743c116897..3445cc5fb45f 100644 --- a/web-console/src/druid-models/task/task.ts +++ b/web-console/src/druid-models/task/task.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { C } from 'druid-query-toolkit'; +import { C } from '@druid-toolkit/query'; import type { Counters, StageDefinition } from '../stages/stages'; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts index 8456b0d065d5..02bb3e399b02 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from 'druid-query-toolkit'; +import { sane } from '@druid-toolkit/query'; import { WorkbenchQuery } from './workbench-query'; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.ts b/web-console/src/druid-models/workbench-query/workbench-query.ts index 9b1ed42b516b..da37127f1d2d 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.ts @@ -22,7 +22,7 @@ import type { SqlClusteredByClause, SqlExpression, SqlPartitionedByClause, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import { C, F, @@ -30,7 +30,7 @@ import { SqlOrderByClause, SqlOrderByExpression, SqlQuery, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import Hjson from 'hjson'; import * as JSONBig from 'json-bigint-native'; import { v4 as uuidv4 } from 'uuid'; diff --git a/web-console/src/entry.scss b/web-console/src/entry.scss index 46ea9d298d0f..8d368a4146b7 100644 --- a/web-console/src/entry.scss +++ b/web-console/src/entry.scss @@ -56,7 +56,6 @@ body { position: absolute; height: 100%; width: 100%; - z-index: 0; .console-application { position: absolute; diff --git a/web-console/src/entry.tsx b/web-console/src/entry.tsx index 55f7abfc59a2..42cfe08ad13c 100644 --- a/web-console/src/entry.tsx +++ b/web-console/src/entry.tsx @@ -20,7 +20,7 @@ import 'regenerator-runtime/runtime'; import './bootstrap/ace'; import { OverlaysProvider } from '@blueprintjs/core'; -import { QueryRunner } from 'druid-query-toolkit'; +import { QueryRunner } from '@druid-toolkit/query'; import { createRoot } from 'react-dom/client'; import { bootstrapJsonParse } from './bootstrap/json-parser'; diff --git a/web-console/src/helpers/capabilities.ts b/web-console/src/helpers/capabilities.ts index 3255ec93be73..013f9368c58c 100644 --- a/web-console/src/helpers/capabilities.ts +++ b/web-console/src/helpers/capabilities.ts @@ -211,10 +211,6 @@ export class Capabilities { }; } - public clone(): Capabilities { - return new Capabilities(this.valueOf()); - } - public getMode(): CapabilitiesMode { if (!this.hasSql()) return 'no-sql'; if (!this.hasCoordinatorAccess()) return 'no-proxy'; diff --git a/web-console/src/helpers/execution/general.ts b/web-console/src/helpers/execution/general.ts index 0e78834172b1..a90c02a63061 100644 --- a/web-console/src/helpers/execution/general.ts +++ b/web-console/src/helpers/execution/general.ts @@ -16,8 +16,8 @@ * limitations under the License. */ +import type { QueryResult } from '@druid-toolkit/query'; import type { CancelToken } from 'axios'; -import type { QueryResult } from 'druid-query-toolkit'; import type { Execution } from '../../druid-models'; import { IntermediateQueryState } from '../../utils'; diff --git a/web-console/src/helpers/execution/sql-task-execution.ts b/web-console/src/helpers/execution/sql-task-execution.ts index 68cf0763b425..f4dd45a2cb91 100644 --- a/web-console/src/helpers/execution/sql-task-execution.ts +++ b/web-console/src/helpers/execution/sql-task-execution.ts @@ -16,8 +16,8 @@ * limitations under the License. */ +import { QueryResult } from '@druid-toolkit/query'; import type { AxiosResponse, CancelToken } from 'axios'; -import { QueryResult } from 'druid-query-toolkit'; import type { AsyncStatusResponse, MsqTaskPayloadResponse, QueryContext } from '../../druid-models'; import { Execution } from '../../druid-models'; diff --git a/web-console/src/helpers/spec-conversion.ts b/web-console/src/helpers/spec-conversion.ts index 9fc95d3b60f5..00bc3d5f832c 100644 --- a/web-console/src/helpers/spec-conversion.ts +++ b/web-console/src/helpers/spec-conversion.ts @@ -25,7 +25,7 @@ import { SqlExpression, SqlType, T, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import * as JSONBig from 'json-bigint-native'; import type { diff --git a/web-console/src/hooks/use-clock.ts b/web-console/src/hooks/use-clock.ts index a909a2c1f740..5d1264421957 100644 --- a/web-console/src/hooks/use-clock.ts +++ b/web-console/src/hooks/use-clock.ts @@ -16,9 +16,7 @@ * limitations under the License. */ -import { useState } from 'react'; - -import { useInterval } from './use-interval'; +import { useEffect, useState } from 'react'; function getNowToSecond(): Date { const now = new Date(); @@ -26,12 +24,18 @@ function getNowToSecond(): Date { return now; } -export function useClock(updateInterval = 1000) { +export function useClock() { const [now, setNow] = useState(getNowToSecond); - useInterval(() => { - setNow(getNowToSecond()); - }, updateInterval); + useEffect(() => { + const checkInterval = setInterval(() => { + setNow(getNowToSecond()); + }, 1000); + + return () => { + clearInterval(checkInterval); + }; + }, []); return now; } diff --git a/web-console/src/react-table/react-table-utils.spec.ts b/web-console/src/react-table/react-table-utils.spec.ts index 1dd245b49f70..0a1bbf3f9ad5 100644 --- a/web-console/src/react-table/react-table-utils.spec.ts +++ b/web-console/src/react-table/react-table-utils.spec.ts @@ -75,9 +75,5 @@ describe('react-table-utils', () => { { id: 'x', value: '~y' }, { id: 'z', value: '=w&' }, ]); - expect(stringToTableFilters('x<3&y<=3')).toEqual([ - { id: 'x', value: '<3' }, - { id: 'y', value: '<=3' }, - ]); }); }); diff --git a/web-console/src/react-table/react-table-utils.ts b/web-console/src/react-table/react-table-utils.ts index 7732c90b231c..ce9497a1209a 100644 --- a/web-console/src/react-table/react-table-utils.ts +++ b/web-console/src/react-table/react-table-utils.ts @@ -18,7 +18,8 @@ import type { IconName } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import { C, F, SqlExpression } from 'druid-query-toolkit'; +import type { SqlExpression } from '@druid-toolkit/query'; +import { C, F } from '@druid-toolkit/query'; import type { Filter } from 'react-table'; import { addOrUpdate, caseInsensitiveContains, filterMap } from '../utils'; @@ -31,9 +32,9 @@ export const STANDARD_TABLE_PAGE_SIZE_OPTIONS = [50, 100, 200]; export const SMALL_TABLE_PAGE_SIZE = 25; export const SMALL_TABLE_PAGE_SIZE_OPTIONS = [25, 50, 100]; -export type FilterMode = '~' | '=' | '!=' | '<' | '<=' | '>' | '>='; +export type FilterMode = '~' | '=' | '!=' | '<=' | '>='; -export const FILTER_MODES: FilterMode[] = ['~', '=', '!=', '<', '<=', '>', '>=']; +export const FILTER_MODES: FilterMode[] = ['~', '=', '!=', '<=', '>=']; export const FILTER_MODES_NO_COMPARISON: FilterMode[] = ['~', '=', '!=']; export function filterModeToIcon(mode: FilterMode): IconName { @@ -44,12 +45,8 @@ export function filterModeToIcon(mode: FilterMode): IconName { return IconNames.EQUALS; case '!=': return IconNames.NOT_EQUAL_TO; - case '<': - return IconNames.LESS_THAN; case '<=': return IconNames.LESS_THAN_OR_EQUAL_TO; - case '>': - return IconNames.GREATER_THAN; case '>=': return IconNames.GREATER_THAN_OR_EQUAL_TO; default: @@ -65,12 +62,8 @@ export function filterModeToTitle(mode: FilterMode): string { return 'Equals'; case '!=': return 'Not equals'; - case '<': - return 'Less than'; case '<=': return 'Less than or equal'; - case '>': - return 'Greater than'; case '>=': return 'Greater than or equal'; default: @@ -96,7 +89,7 @@ export function parseFilterModeAndNeedle( filter: Filter, loose = false, ): FilterModeAndNeedle | undefined { - const m = /^(~|=|!=|<(?!=)|<=|>(?!=)|>=)?(.*)$/.exec(String(filter.value)); + const m = /^(~|=|!=|<=|>=)?(.*)$/.exec(String(filter.value)); if (!m) return; if (!loose && !m[2]) return; const mode = (m[1] as FilterMode) || '~'; @@ -119,28 +112,21 @@ export function booleanCustomTableFilter(filter: Filter, value: unknown): boolea const modeAndNeedle = parseFilterModeAndNeedle(filter); if (!modeAndNeedle) return true; const { mode, needle } = modeAndNeedle; - const strValue = String(value); switch (mode) { case '=': - return strValue === needle; + return String(value) === needle; case '!=': - return strValue !== needle; - - case '<': - return strValue < needle; + return String(value) !== needle; case '<=': - return strValue <= needle; - - case '>': - return strValue > needle; + return String(value) <= needle; case '>=': - return strValue >= needle; + return String(value) >= needle; default: - return caseInsensitiveContains(strValue, needle); + return caseInsensitiveContains(String(value), needle); } } @@ -156,15 +142,9 @@ export function sqlQueryCustomTableFilter(filter: Filter): SqlExpression | undef case '!=': return column.unequal(needle); - case '<': - return column.lessThan(needle); - case '<=': return column.lessThanOrEqual(needle); - case '>': - return column.greaterThan(needle); - case '>=': return column.greaterThanOrEqual(needle); @@ -173,10 +153,6 @@ export function sqlQueryCustomTableFilter(filter: Filter): SqlExpression | undef } } -export function sqlQueryCustomTableFilters(filters: Filter[]): SqlExpression { - return SqlExpression.and(...filterMap(filters, sqlQueryCustomTableFilter)); -} - export function tableFiltersToString(tableFilters: Filter[]): string { return tableFilters .map(({ id, value }) => `${id}${value.replace(/[&%]/g, encodeURIComponent)}`) @@ -185,11 +161,9 @@ export function tableFiltersToString(tableFilters: Filter[]): string { export function stringToTableFilters(str: string | undefined): Filter[] { if (!str) return []; - // '~' | '=' | '!=' | '<' | '<=' | '>' | '>='; + // '~' | '=' | '!=' | '<=' | '>='; return filterMap(str.split('&'), clause => { - const m = /^(\w+)((?:~|=|!=|<(?!=)|<=|>(?!=)|>=).*)$/.exec( - clause.replace(/%2[56]/g, decodeURIComponent), - ); + const m = /^(\w+)((?:~|=|!=|<=|>=).*)$/.exec(clause.replace(/%2[56]/g, decodeURIComponent)); if (!m) return; return { id: m[1], value: m[2] }; }); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts deleted file mode 100755 index 5ba63b04468f..000000000000 --- a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts +++ /dev/null @@ -1,169 +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 { shifters } from './date-floor-shift-ceil'; - -function pairwise(array: T[], callback: (t1: T, t2: T) => void) { - for (let i = 0; i < array.length - 1; i++) { - callback(array[i], array[i + 1]); - } -} - -describe('floor, shift, ceil (UTC)', () => { - const tz = 'Etc/UTC'; - - it('moves seconds', () => { - const dates: Date[] = [ - new Date('2012-11-04T00:00:00Z'), - new Date('2012-11-04T00:00:03Z'), - new Date('2012-11-04T00:00:06Z'), - new Date('2012-11-04T00:00:09Z'), - new Date('2012-11-04T00:00:12Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.second.shift(d1, tz, 3)).toEqual(d2)); - }); - - it('rounds minutes', () => { - expect(shifters.minute.round(new Date('2012-11-04T00:29:00Z'), 15, tz)).toEqual( - new Date('2012-11-04T00:15:00Z'), - ); - - expect(shifters.minute.round(new Date('2012-11-04T00:29:00Z'), 4, tz)).toEqual( - new Date('2012-11-04T00:28:00Z'), - ); - }); - - it('moves minutes', () => { - const dates: Date[] = [ - new Date('2012-11-04T00:00:00Z'), - new Date('2012-11-04T00:03:00Z'), - new Date('2012-11-04T00:06:00Z'), - new Date('2012-11-04T00:09:00Z'), - new Date('2012-11-04T00:12:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.minute.shift(d1, tz, 3)).toEqual(d2)); - }); - - it('floors hour correctly', () => { - expect(shifters.hour.floor(new Date('2012-11-04T00:30:00Z'), tz)).toEqual( - new Date('2012-11-04T00:00:00Z'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T01:30:00Z'), tz)).toEqual( - new Date('2012-11-04T01:00:00Z'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T01:30:00Z'), tz)).toEqual( - new Date('2012-11-04T01:00:00Z'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T02:30:00Z'), tz)).toEqual( - new Date('2012-11-04T02:00:00Z'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T03:30:00Z'), tz)).toEqual( - new Date('2012-11-04T03:00:00Z'), - ); - }); - - it('moves hour', () => { - const dates: Date[] = [ - new Date('2012-11-04T00:00:00Z'), - new Date('2012-11-04T01:00:00Z'), - new Date('2012-11-04T02:00:00Z'), - new Date('2012-11-04T03:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('moves day', () => { - const dates: Date[] = [ - new Date('2012-11-03T00:00:00Z'), - new Date('2012-11-04T00:00:00Z'), - new Date('2012-11-05T00:00:00Z'), - new Date('2012-11-06T00:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.day.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('ceils day', () => { - let d1 = new Date('2014-12-11T22:11:57.469Z'); - let d2 = new Date('2014-12-12T00:00:00.000Z'); - expect(shifters.day.ceil(d1, tz)).toEqual(d2); - - d1 = new Date('2014-12-08T00:00:00.000Z'); - d2 = new Date('2014-12-08T00:00:00.000Z'); - expect(shifters.day.ceil(d1, tz)).toEqual(d2); - }); - - it('moves week', () => { - const dates: Date[] = [ - new Date('2012-10-29T00:00:00Z'), - new Date('2012-11-05T00:00:00Z'), - new Date('2012-11-12T00:00:00Z'), - new Date('2012-11-19T00:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.week.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('floors week correctly', () => { - let d1 = new Date('2014-12-11T22:11:57.469Z'); - let d2 = new Date('2014-12-08T00:00:00.000Z'); - expect(shifters.week.floor(d1, tz)).toEqual(d2); - - d1 = new Date('2014-12-07T12:11:57.469Z'); - d2 = new Date('2014-12-01T00:00:00.000Z'); - expect(shifters.week.floor(d1, tz)).toEqual(d2); - }); - - it('ceils week correctly', () => { - let d1 = new Date('2014-12-11T22:11:57.469Z'); - let d2 = new Date('2014-12-15T00:00:00.000Z'); - expect(shifters.week.ceil(d1, tz)).toEqual(d2); - - d1 = new Date('2014-12-07T12:11:57.469Z'); - d2 = new Date('2014-12-08T00:00:00.000Z'); - expect(shifters.week.ceil(d1, tz)).toEqual(d2); - }); - - it('moves month', () => { - const dates: Date[] = [ - new Date('2012-11-01T00:00:00Z'), - new Date('2012-12-01T00:00:00Z'), - new Date('2013-01-01T00:00:00Z'), - new Date('2013-02-01T00:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.month.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('shifts month on the 31st', () => { - const d1 = new Date('2016-03-31T00:00:00.000Z'); - const d2 = new Date('2016-05-01T00:00:00.000Z'); - expect(shifters.month.shift(d1, tz, 1)).toEqual(d2); - }); - - it('moves year', () => { - const dates: Date[] = [ - new Date('2010-01-01T00:00:00Z'), - new Date('2011-01-01T00:00:00Z'), - new Date('2012-01-01T00:00:00Z'), - new Date('2013-01-01T00:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.year.shift(d1, tz, 1)).toEqual(d2)); - }); -}); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts deleted file mode 100755 index 1612c9ce5792..000000000000 --- a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts +++ /dev/null @@ -1,181 +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 { shifters } from './date-floor-shift-ceil'; - -function pairwise(array: T[], callback: (t1: T, t2: T) => void) { - for (let i = 0; i < array.length - 1; i++) { - callback(array[i], array[i + 1]); - } -} - -describe('floor/shift/ceil', () => { - const tz = 'America/Los_Angeles'; - - it('shifts seconds', () => { - const dates: Date[] = [ - new Date('2012-11-04T00:00:00-07:00'), - new Date('2012-11-04T00:00:03-07:00'), - new Date('2012-11-04T00:00:06-07:00'), - new Date('2012-11-04T00:00:09-07:00'), - new Date('2012-11-04T00:00:12-07:00'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.second.shift(d1, tz, 3)).toEqual(d2)); - }); - - it('shifts minutes', () => { - const dates: Date[] = [ - new Date('2012-11-04T00:00:00-07:00'), - new Date('2012-11-04T00:03:00-07:00'), - new Date('2012-11-04T00:06:00-07:00'), - new Date('2012-11-04T00:09:00-07:00'), - new Date('2012-11-04T00:12:00-07:00'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.minute.shift(d1, tz, 3)).toEqual(d2)); - }); - - it('floors hour correctly', () => { - expect(shifters.hour.floor(new Date('2012-11-04T00:30:00-07:00'), tz)).toEqual( - new Date('2012-11-04T00:00:00-07:00'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T01:30:00-07:00'), tz)).toEqual( - new Date('2012-11-04T01:00:00-07:00'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T01:30:00-08:00'), tz)).toEqual( - new Date('2012-11-04T01:00:00-07:00'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T02:30:00-08:00'), tz)).toEqual( - new Date('2012-11-04T02:00:00-08:00'), - ); - - expect(shifters.hour.floor(new Date('2012-11-04T03:30:00-08:00'), tz)).toEqual( - new Date('2012-11-04T03:00:00-08:00'), - ); - }); - - it('shifting 24 hours over DST is not the same as shifting a day', () => { - const start = new Date('2012-11-04T07:00:00Z'); - - const shift1Day = shifters.day.shift(start, tz, 1); - const shift24Hours = shifters.hour.shift(start, tz, 24); - - expect(shift1Day).toEqual(new Date('2012-11-05T08:00:00Z')); - expect(shift24Hours).toEqual(new Date('2012-11-05T07:00:00Z')); - }); - - it('shifts hour over DST 1', () => { - const dates: Date[] = [ - new Date('2012-11-04T00:00:00-07:00'), - new Date('2012-11-04T08:00:00Z'), - new Date('2012-11-04T09:00:00Z'), - new Date('2012-11-04T10:00:00Z'), - new Date('2012-11-04T11:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('floors hour over DST 1', () => { - expect(shifters.hour.floor(new Date('2012-11-04T00:05:00-07:00'), tz)).toEqual( - new Date('2012-11-04T00:00:00-07:00'), - ); - expect(shifters.hour.floor(new Date('2012-11-04T01:05:00-07:00'), tz)).toEqual( - new Date('2012-11-04T01:00:00-07:00'), - ); - expect(shifters.hour.floor(new Date('2012-11-04T02:05:00-07:00'), tz)).toEqual( - new Date('2012-11-04T01:00:00-07:00'), - ); - expect(shifters.hour.floor(new Date('2012-11-04T03:05:00-07:00'), tz)).toEqual( - new Date('2012-11-04T03:00:00-07:00'), - ); - }); - - it('shifts hour over DST 2', () => { - // "2018-03-11T09:00:00Z" - const dates: Date[] = [ - new Date('2018-03-11T01:00:00-07:00'), - new Date('2018-03-11T09:00:00Z'), - new Date('2018-03-11T10:00:00Z'), - new Date('2018-03-11T11:00:00Z'), - new Date('2018-03-11T12:00:00Z'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('shifts day over DST', () => { - const dates: Date[] = [ - new Date('2012-11-03T00:00:00-07:00'), - new Date('2012-11-04T00:00:00-07:00'), - new Date('2012-11-05T00:00:00-08:00'), - new Date('2012-11-06T00:00:00-08:00'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.day.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('shifts week over DST', () => { - const dates: Date[] = [ - new Date('2012-10-29T00:00:00-07:00'), - new Date('2012-11-05T00:00:00-08:00'), - new Date('2012-11-12T00:00:00-08:00'), - new Date('2012-11-19T00:00:00-08:00'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.week.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('floors week correctly', () => { - let d1 = new Date('2014-12-11T22:11:57.469Z'); - let d2 = new Date('2014-12-08T08:00:00.000Z'); - expect(shifters.week.floor(d1, tz)).toEqual(d2); - - d1 = new Date('2014-12-07T12:11:57.469Z'); - d2 = new Date('2014-12-01T08:00:00.000Z'); - expect(shifters.week.floor(d1, tz)).toEqual(d2); - }); - - it('ceils week correctly', () => { - let d1 = new Date('2014-12-11T22:11:57.469Z'); - let d2 = new Date('2014-12-15T08:00:00.000Z'); - expect(shifters.week.ceil(d1, tz)).toEqual(d2); - - d1 = new Date('2014-12-07T12:11:57.469Z'); - d2 = new Date('2014-12-08T08:00:00.000Z'); - expect(shifters.week.ceil(d1, tz)).toEqual(d2); - }); - - it('shifts month over DST', () => { - const dates: Date[] = [ - new Date('2012-11-01T00:00:00-07:00'), - new Date('2012-12-01T00:00:00-08:00'), - new Date('2013-01-01T00:00:00-08:00'), - new Date('2013-02-01T00:00:00-08:00'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.month.shift(d1, tz, 1)).toEqual(d2)); - }); - - it('shifts year', () => { - const dates: Date[] = [ - new Date('2010-01-01T00:00:00-08:00'), - new Date('2011-01-01T00:00:00-08:00'), - new Date('2012-01-01T00:00:00-08:00'), - new Date('2013-01-01T00:00:00-08:00'), - ]; - pairwise(dates, (d1, d2) => expect(shifters.year.shift(d1, tz, 1)).toEqual(d2)); - }); -}); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts deleted file mode 100755 index 3306b05267d9..000000000000 --- a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts +++ /dev/null @@ -1,296 +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 { fromDate, startOfWeek } from '@internationalized/date'; - -export type AlignFn = (dt: Date, tz: string) => Date; - -export type ShiftFn = (dt: Date, tz: string, step: number) => Date; - -export type RoundFn = (dt: Date, roundTo: number, tz: string) => Date; - -export interface TimeShifterNoCeil { - canonicalLength: number; - siblings?: number; - floor: AlignFn; - round: RoundFn; - shift: ShiftFn; -} - -export interface TimeShifter extends TimeShifterNoCeil { - ceil: AlignFn; -} - -function isUTC(tz: string): boolean { - return tz === 'Etc/UTC'; -} - -function adjustDay(day: number): number { - return (day + 6) % 7; -} - -function floorTo(n: number, roundTo: number): number { - return Math.floor(n / roundTo) * roundTo; -} - -function timeShifterFiller(tm: TimeShifterNoCeil): TimeShifter { - const { floor, shift } = tm; - return { - ...tm, - ceil: (dt: Date, tz: string) => { - const floored = floor(dt, tz); - if (floored.valueOf() === dt.valueOf()) return dt; // Just like ceil(3) is 3 and not 4 - return shift(floored, tz, 1); - }, - }; -} - -export const second = timeShifterFiller({ - canonicalLength: 1000, - siblings: 60, - floor: (dt, _tz) => { - // Seconds do not actually need a timezone because all timezones align on seconds... for now... - dt = new Date(dt.valueOf()); - dt.setUTCMilliseconds(0); - return dt; - }, - round: (dt, roundTo, _tz) => { - const cur = dt.getUTCSeconds(); - const adj = floorTo(cur, roundTo); - if (cur !== adj) dt.setUTCSeconds(adj); - return dt; - }, - shift: (dt, _tz, step) => { - dt = new Date(dt.valueOf()); - dt.setUTCSeconds(dt.getUTCSeconds() + step); - return dt; - }, -}); - -export const minute = timeShifterFiller({ - canonicalLength: 60000, - siblings: 60, - floor: (dt, _tz) => { - // Minutes do not actually need a timezone because all timezones align on minutes... for now... - dt = new Date(dt.valueOf()); - dt.setUTCSeconds(0, 0); - return dt; - }, - round: (dt, roundTo, _tz) => { - const cur = dt.getUTCMinutes(); - const adj = floorTo(cur, roundTo); - if (cur !== adj) dt.setUTCMinutes(adj); - return dt; - }, - shift: (dt, _tz, step) => { - dt = new Date(dt.valueOf()); - dt.setUTCMinutes(dt.getUTCMinutes() + step); - return dt; - }, -}); - -// Movement by hour is tz independent because in every timezone an hour is 60 min -function hourMove(dt: Date, _tz: string, step: number) { - dt = new Date(dt.valueOf()); - dt.setUTCHours(dt.getUTCHours() + step); - return dt; -} - -export const hour = timeShifterFiller({ - canonicalLength: 3600000, - siblings: 24, - floor: (dt, tz) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCMinutes(0, 0, 0); - return dt; - } else { - return fromDate(dt, tz).set({ second: 0, minute: 0, millisecond: 0 }).toDate(); - } - }, - round: (dt, roundTo, tz) => { - if (isUTC(tz)) { - const cur = dt.getUTCHours(); - const adj = floorTo(cur, roundTo); - if (cur !== adj) dt.setUTCHours(adj); - } else { - const cur = fromDate(dt, tz).hour; - const adj = floorTo(cur, roundTo); - if (cur !== adj) return hourMove(dt, tz, adj - cur); - } - return dt; - }, - shift: hourMove, -}); - -export const day = timeShifterFiller({ - canonicalLength: 24 * 3600000, - floor: (dt, tz) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCHours(0, 0, 0, 0); - return dt; - } else { - return fromDate(dt, tz).set({ hour: 0, second: 0, minute: 0, millisecond: 0 }).toDate(); - } - }, - shift: (dt, tz, step) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCDate(dt.getUTCDate() + step); - return dt; - } else { - return fromDate(dt, tz).add({ days: step }).toDate(); - } - }, - round: () => { - throw new Error('missing day round'); - }, -}); - -export const week = timeShifterFiller({ - canonicalLength: 7 * 24 * 3600000, - floor: (dt, tz) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCHours(0, 0, 0, 0); - dt.setUTCDate(dt.getUTCDate() - adjustDay(dt.getUTCDay())); - } else { - const zd = fromDate(dt, tz); - return startOfWeek( - zd.set({ hour: 0, second: 0, minute: 0, millisecond: 0 }), - 'fr-FR', // We want the week to start on Monday - ).toDate(); - } - return dt; - }, - shift: (dt, tz, step) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCDate(dt.getUTCDate() + step * 7); - return dt; - } else { - return fromDate(dt, tz).add({ weeks: step }).toDate(); - } - }, - round: () => { - throw new Error('missing week round'); - }, -}); - -function monthShift(dt: Date, tz: string, step: number) { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCMonth(dt.getUTCMonth() + step); - return dt; - } else { - return fromDate(dt, tz).add({ months: step }).toDate(); - } -} - -export const month = timeShifterFiller({ - canonicalLength: 30 * 24 * 3600000, - siblings: 12, - floor: (dt, tz) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCHours(0, 0, 0, 0); - dt.setUTCDate(1); - return dt; - } else { - return fromDate(dt, tz) - .set({ day: 1, hour: 0, second: 0, minute: 0, millisecond: 0 }) - .toDate(); - } - }, - round: (dt, roundTo, tz) => { - if (isUTC(tz)) { - const cur = dt.getUTCMonth(); - const adj = floorTo(cur, roundTo); - if (cur !== adj) dt.setUTCMonth(adj); - } else { - const cur = fromDate(dt, tz).month - 1; // Needs to be zero indexed - const adj = floorTo(cur, roundTo); - if (cur !== adj) return monthShift(dt, tz, adj - cur); - } - return dt; - }, - shift: monthShift, -}); - -function yearShift(dt: Date, tz: string, step: number) { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCFullYear(dt.getUTCFullYear() + step); - return dt; - } else { - return fromDate(dt, tz).add({ years: step }).toDate(); - } -} - -export const year = timeShifterFiller({ - canonicalLength: 365 * 24 * 3600000, - siblings: 1000, - floor: (dt, tz) => { - if (isUTC(tz)) { - dt = new Date(dt.valueOf()); - dt.setUTCHours(0, 0, 0, 0); - dt.setUTCMonth(0, 1); - return dt; - } else { - return fromDate(dt, tz) - .set({ month: 1, day: 1, hour: 0, second: 0, minute: 0, millisecond: 0 }) - .toDate(); - } - }, - round: (dt, roundTo, tz) => { - if (isUTC(tz)) { - const cur = dt.getUTCFullYear(); - const adj = floorTo(cur, roundTo); - if (cur !== adj) dt.setUTCFullYear(adj); - } else { - const cur = fromDate(dt, tz).year; - const adj = floorTo(cur, roundTo); - if (cur !== adj) return yearShift(dt, tz, adj - cur); - } - return dt; - }, - shift: yearShift, -}); - -export interface Shifters { - second: TimeShifter; - minute: TimeShifter; - hour: TimeShifter; - day: TimeShifter; - week: TimeShifter; - month: TimeShifter; - year: TimeShifter; - - [key: string]: TimeShifter; -} - -export const shifters: Shifters = { - second, - minute, - hour, - day, - week, - month, - year, -}; diff --git a/web-console/src/utils/date.spec.ts b/web-console/src/utils/date.spec.ts index b219ee17af06..843c144244ef 100644 --- a/web-console/src/utils/date.spec.ts +++ b/web-console/src/utils/date.spec.ts @@ -17,6 +17,7 @@ */ import { + ceilToUtcDay, dateToIsoDateString, intervalToLocalDateRange, localDateRangeToInterval, @@ -59,4 +60,12 @@ describe('date', () => { expect(localDateRangeToInterval(intervalToLocalDateRange(interval))).toEqual(interval); }); }); + + describe('ceilToUtcDay', () => { + it('works', () => { + expect(ceilToUtcDay(new Date('2021-02-03T12:03:02.001Z'))).toEqual( + new Date('2021-02-04T00:00:00Z'), + ); + }); + }); }); diff --git a/web-console/src/utils/date.ts b/web-console/src/utils/date.ts index 08f1d1df43ba..1d5aa4f61e95 100644 --- a/web-console/src/utils/date.ts +++ b/web-console/src/utils/date.ts @@ -99,10 +99,9 @@ export function localDateRangeToInterval(localRange: DateRange): string { }`; } -export function maxDate(a: Date, b: Date): Date { - return a > b ? a : b; -} - -export function minDate(a: Date, b: Date): Date { - return a < b ? a : b; +export function ceilToUtcDay(date: Date): Date { + date = new Date(date.valueOf()); + date.setUTCHours(0, 0, 0, 0); + date.setUTCDate(date.getUTCDate() + 1); + return date; } diff --git a/web-console/src/utils/download.ts b/web-console/src/utils/download.ts index 40b0d95e8b91..4fb3342e523c 100644 --- a/web-console/src/utils/download.ts +++ b/web-console/src/utils/download.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { QueryResult } from 'druid-query-toolkit'; +import type { QueryResult } from '@druid-toolkit/query'; import FileSaver from 'file-saver'; import * as JSONBig from 'json-bigint-native'; diff --git a/web-console/src/utils/druid-query.spec.ts b/web-console/src/utils/druid-query.spec.ts index a940b8ac7bb6..ee867ff47ea5 100644 --- a/web-console/src/utils/druid-query.spec.ts +++ b/web-console/src/utils/druid-query.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from 'druid-query-toolkit'; +import { sane } from '@druid-toolkit/query'; import { DruidError, getDruidErrorMessage } from './druid-query'; diff --git a/web-console/src/utils/druid-query.ts b/web-console/src/utils/druid-query.ts index ee1ba1ef4a8c..8102db89ca3a 100644 --- a/web-console/src/utils/druid-query.ts +++ b/web-console/src/utils/druid-query.ts @@ -16,9 +16,9 @@ * limitations under the License. */ +import { C } from '@druid-toolkit/query'; import type { AxiosResponse, CancelToken } from 'axios'; import axios from 'axios'; -import { C } from 'druid-query-toolkit'; import { Api } from '../singletons'; @@ -358,12 +358,6 @@ export async function queryDruidSqlDart( return sqlResultResp.data; } -export async function getApiArray(url: string, cancelToken?: CancelToken): Promise { - const result = (await Api.instance.get(url, { cancelToken })).data; - if (!Array.isArray(result)) throw new Error('unexpected result'); - return result; -} - export interface QueryExplanation { query: any; signature: { name: string; type: string }[]; diff --git a/web-console/src/utils/duration/duration.spec.ts b/web-console/src/utils/duration/duration.spec.ts deleted file mode 100755 index 8b20ac0a6b2a..000000000000 --- a/web-console/src/utils/duration/duration.spec.ts +++ /dev/null @@ -1,505 +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 { Duration } from './duration'; - -describe('Duration', () => { - const TZ_LA = 'America/Los_Angeles'; - const TZ_JUNEAU = 'America/Juneau'; - - describe('errors', () => { - it('throws error if invalid duration', () => { - expect(() => new Duration('')).toThrow("Can not parse duration ''"); - - expect(() => new Duration('P00')).toThrow("Can not parse duration 'P00'"); - - expect(() => new Duration('P')).toThrow('Duration can not be empty'); - - expect(() => new Duration('P0YT0H')).toThrow('Duration can not be empty'); - - expect(() => new Duration('P0W').shift(new Date(), TZ_LA)).toThrow( - 'Duration can not have empty weeks', - ); - - expect(() => new Duration('P0Y0MT0H0M0S').shift(new Date(), TZ_LA)).toThrow( - 'Duration can not be empty', - ); - }); - - it('throws error if fromJS is not given a string', () => { - expect(() => new Duration(new Date() as any)).toThrow('Duration can not be empty'); - }); - }); - - describe('#toString', () => { - it('gives back the correct string', () => { - let durationStr: string; - - durationStr = 'P3Y'; - expect(new Duration(durationStr).toString()).toEqual(durationStr); - - durationStr = 'P2W'; - expect(new Duration(durationStr).toString()).toEqual(durationStr); - - durationStr = 'PT5H'; - expect(new Duration(durationStr).toString()).toEqual(durationStr); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).toString()).toEqual(durationStr); - }); - - it('eliminates 0', () => { - expect(new Duration('P0DT15H').toString()).toEqual('PT15H'); - }); - }); - - describe('fromCanonicalLength', () => { - it('handles zero', () => { - expect(() => { - Duration.fromCanonicalLength(0); - }).toThrow('length must be positive'); - }); - - it('works 1', () => { - expect(Duration.fromCanonicalLength(86400000).toString()).toEqual('P1D'); - }); - - it('works 2', () => { - const len = - new Date('2018-03-01T00:00:00Z').valueOf() - new Date('2016-02-22T00:00:00Z').valueOf(); - expect(Duration.fromCanonicalLength(len).toString()).toEqual('P2Y8D'); - }); - - it('works 3', () => { - const len = - new Date('2018-09-15T00:00:00Z').valueOf() - new Date('2018-09-04T00:00:00Z').valueOf(); - expect(Duration.fromCanonicalLength(len).toString()).toEqual('P11D'); - }); - - it('works with months', () => { - expect(Duration.fromCanonicalLength(2592000000).toString()).toEqual('P1M'); - expect(Duration.fromCanonicalLength(2678400000).toString()).toEqual('P1M1D'); - }); - - it('works without months', () => { - expect(Duration.fromCanonicalLength(2592000000, true).toString()).toEqual('P30D'); - expect(Duration.fromCanonicalLength(2678400000, true).toString()).toEqual('P31D'); - }); - }); - - describe('construct from span', () => { - it('parses days over DST', () => { - expect( - Duration.fromRange( - new Date('2012-10-29T00:00:00-07:00'), - new Date('2012-11-05T00:00:00-08:00'), - TZ_LA, - ).toString(), - ).toEqual('P7D'); - - expect( - Duration.fromRange( - new Date('2012-10-29T00:00:00-07:00'), - new Date('2012-11-12T00:00:00-08:00'), - TZ_LA, - ).toString(), - ).toEqual('P14D'); - }); - - it('parses complex case', () => { - expect( - Duration.fromRange( - new Date('2012-10-29T00:00:00-07:00'), - new Date(new Date('2012-11-05T00:00:00-08:00').valueOf() - 1000), - TZ_LA, - ).toString(), - ).toEqual('P6DT24H59M59S'); - - expect( - Duration.fromRange( - new Date('2012-01-01T00:00:00-08:00'), - new Date('2013-03-04T04:05:06-08:00'), - TZ_LA, - ).toString(), - ).toEqual('P1Y2M3DT4H5M6S'); - }); - }); - - describe('#isFloorable', () => { - const floorable = 'P1Y P5Y P10Y P100Y P1M P2M P3M P4M P1D'.split(' '); - for (const v of floorable) { - it(`works on floorable ${v}`, () => { - expect(new Duration(v).isFloorable()).toEqual(true); - }); - } - - const unfloorable = 'P1Y1M P5M P2D P3D'.split(' '); - for (const v of unfloorable) { - it(`works on not floorable ${v}`, () => { - expect(new Duration(v).isFloorable()).toEqual(false); - }); - } - }); - - describe('#floor', () => { - it('throws error if complex duration', () => { - expect(() => new Duration('P1Y2D').floor(new Date(), TZ_LA)).toThrow( - 'Can not floor on a complex duration', - ); - - expect(() => new Duration('P3DT15H').floor(new Date(), TZ_LA)).toThrow( - 'Can not floor on a complex duration', - ); - - expect(() => new Duration('PT5H').floor(new Date(), TZ_LA)).toThrow( - 'Can not floor on a hour duration that does not divide into 24', - ); - }); - - it('works for year', () => { - const p1y = new Duration('P1Y'); - expect(p1y.floor(new Date('2013-09-29T01:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-01-01T00:00:00.000-08:00'), - ); - }); - - it('works for PT2M', () => { - const pt2h = new Duration('PT2M'); - expect(pt2h.floor(new Date('2013-09-29T03:03:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-09-29T03:02:00.000-07:00'), - ); - }); - - it('works for P2H', () => { - const pt2h = new Duration('PT2H'); - expect(pt2h.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-09-29T02:00:00.000-07:00'), - ); - }); - - it('works for PT12H', () => { - const pt12h = new Duration('PT12H'); - expect(pt12h.floor(new Date('2015-09-12T13:05:00-08:00'), TZ_JUNEAU)).toEqual( - new Date('2015-09-12T12:00:00-08:00'), - ); - }); - - it('works for P1W', () => { - const p1w = new Duration('P1W'); - - expect(p1w.floor(new Date('2013-09-29T01:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-09-23T07:00:00.000Z'), - ); - - expect(p1w.floor(new Date('2013-10-03T01:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-09-30T00:00:00.000-07:00'), - ); - }); - - it('works for P3M', () => { - const p3m = new Duration('P3M'); - expect(p3m.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-07-01T00:00:00.000-07:00'), - ); - - expect(p3m.floor(new Date('2013-02-29T03:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2013-01-01T00:00:00.000-08:00'), - ); - }); - - it('works for P4Y', () => { - const p4y = new Duration('P4Y'); - expect(p4y.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( - new Date('2012-01-01T00:00:00.000-08:00'), - ); - }); - }); - - describe('#shift', () => { - it('works for weeks', () => { - let p1w = new Duration('P1W'); - expect(p1w.shift(new Date('2012-10-29T00:00:00-07:00'), TZ_LA)).toEqual( - new Date('2012-11-05T00:00:00-08:00'), - ); - - p1w = new Duration('P1W'); - expect(p1w.shift(new Date('2012-10-29T00:00:00-07:00'), TZ_LA, 2)).toEqual( - new Date('2012-11-12T00:00:00-08:00'), - ); - - const p2w = new Duration('P2W'); - expect(p2w.shift(new Date('2012-10-29T05:16:17-07:00'), TZ_LA)).toEqual( - new Date('2012-11-12T05:16:17-08:00'), - ); - }); - - it('works for general complex case', () => { - const pComplex = new Duration('P1Y2M3DT4H5M6S'); - expect(pComplex.shift(new Date('2012-01-01T00:00:00-08:00'), TZ_LA)).toEqual( - new Date('2013-03-04T04:05:06-08:00'), - ); - }); - }); - - describe('#materialize', () => { - it('works for weeks', () => { - const p1w = new Duration('P1W'); - - expect( - p1w.materialize( - new Date('2012-10-29T00:00:00-07:00'), - new Date('2012-12-01T00:00:00-08:00'), - TZ_LA, - ), - ).toEqual([ - new Date('2012-10-29T07:00:00.000Z'), - new Date('2012-11-05T08:00:00.000Z'), - new Date('2012-11-12T08:00:00.000Z'), - new Date('2012-11-19T08:00:00.000Z'), - new Date('2012-11-26T08:00:00.000Z'), - ]); - - expect( - p1w.materialize( - new Date('2012-10-29T00:00:00-07:00'), - new Date('2012-12-01T00:00:00-08:00'), - TZ_LA, - 2, - ), - ).toEqual([ - new Date('2012-10-29T07:00:00.000Z'), - new Date('2012-11-12T08:00:00.000Z'), - new Date('2012-11-26T08:00:00.000Z'), - ]); - }); - }); - - describe('#isAligned', () => { - it('works for weeks', () => { - const p1w = new Duration('P1W'); - expect(p1w.isAligned(new Date('2012-10-29T00:00:00-07:00'), TZ_LA)).toEqual(true); - expect(p1w.isAligned(new Date('2012-10-29T00:00:00-07:00'), 'Etc/UTC')).toEqual(false); - }); - }); - - describe('#dividesBy', () => { - const divisible = 'P5Y/P1Y P1D/P1D P1M/P1D P1W/P1D P1D/PT6H PT3H/PT1H'.split(' '); - for (const v of divisible) { - it(`works for ${v} (true)`, () => { - const p = v.split('/'); - expect(new Duration(p[0]).dividesBy(new Duration(p[1]))).toEqual(true); - }); - } - - const undivisible = 'P1D/P1M PT5H/PT1H'.split(' '); - for (const v of undivisible) { - it(`works for ${v} (false)`, () => { - const p = v.split('/'); - expect(new Duration(p[0]).dividesBy(new Duration(p[1]))).toEqual(false); - }); - } - }); - - describe('#getCanonicalLength', () => { - it('gives back the correct canonical length', () => { - let durationStr: string; - - durationStr = 'P3Y'; - expect(new Duration(durationStr).getCanonicalLength()).toEqual(94608000000); - - durationStr = 'P2W'; - expect(new Duration(durationStr).getCanonicalLength()).toEqual(1209600000); - - durationStr = 'PT5H'; - expect(new Duration(durationStr).getCanonicalLength()).toEqual(18000000); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getCanonicalLength()).toEqual(313200000); - }); - }); - - describe('#add()', () => { - it('works with a simple duration', () => { - const d1 = new Duration('P1D'); - const d2 = new Duration('P1D'); - - expect(d1.add(d2).toString()).toEqual('P2D'); - }); - - it('works with heterogeneous spans', () => { - const d1 = new Duration('P1D'); - const d2 = new Duration('P1Y'); - - expect(d1.add(d2).toString()).toEqual('P1Y1D'); - }); - - it('works with weeks', () => { - let d1 = new Duration('P1W'); - let d2 = new Duration('P2W'); - expect(d1.add(d2).toString()).toEqual('P3W'); - - d1 = new Duration('P6D'); - d2 = new Duration('P1D'); - expect(d1.add(d2).toString()).toEqual('P1W'); - }); - }); - - describe('#subtract()', () => { - it('works with a simple duration', () => { - const d1 = new Duration('P1DT2H'); - const d2 = new Duration('PT1H'); - - expect(d1.subtract(d2).toString()).toEqual('P1DT1H'); - }); - - it('works with a less simple duration', () => { - const d1 = new Duration('P1D'); - const d2 = new Duration('PT1H'); - - expect(d1.subtract(d2).toString()).toEqual('PT23H'); - }); - - it('works with weeks', () => { - const d1 = new Duration('P1W'); - const d2 = new Duration('P1D'); - - expect(d1.subtract(d2).toString()).toEqual('P6D'); - }); - - it('throws an error if result is going to be negative', () => { - const d1 = new Duration('P1D'); - const d2 = new Duration('P2D'); - - expect(() => d1.subtract(d2)).toThrow(); - }); - }); - - describe('#multiply()', () => { - it('works with a simple duration', () => { - const d = new Duration('P1D'); - expect(d.multiply(5).toString()).toEqual('P5D'); - }); - - it('works with a less simple duration', () => { - const d = new Duration('P1DT2H'); - expect(d.multiply(2).toString()).toEqual('P2DT4H'); - }); - - it('works with weeks', () => { - const d = new Duration('P1W'); - expect(d.multiply(5).toString()).toEqual('P5W'); - }); - - it('throws an error if result is going to be negative', () => { - const d = new Duration('P1D'); - expect(() => d.multiply(-1)).toThrow('Multiplier must be positive non-zero'); - }); - - it('gets description properly', () => { - const d = new Duration('P2D'); - expect(d.multiply(2).getDescription(true)).toEqual('4 Days'); - }); - }); - - describe('#getDescription()', () => { - it('gives back the correct description', () => { - let durationStr: string; - - durationStr = 'P1D'; - expect(new Duration(durationStr).getDescription()).toEqual('day'); - - durationStr = 'P1DT2H'; - expect(new Duration(durationStr).getDescription()).toEqual('1 day, 2 hours'); - - durationStr = 'P3Y'; - expect(new Duration(durationStr).getDescription()).toEqual('3 years'); - - durationStr = 'P2W'; - expect(new Duration(durationStr).getDescription()).toEqual('2 weeks'); - - durationStr = 'PT5H'; - expect(new Duration(durationStr).getDescription()).toEqual('5 hours'); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getDescription()).toEqual('3 days, 15 hours'); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getDescription(true)).toEqual('3 Days, 15 Hours'); - }); - }); - - describe('#getSingleSpan()', () => { - it('gives back the correct span', () => { - let durationStr: string; - - durationStr = 'P1D'; - expect(new Duration(durationStr).getSingleSpan()).toEqual('day'); - - durationStr = 'P3Y'; - expect(new Duration(durationStr).getSingleSpan()).toEqual('year'); - - durationStr = 'P2W'; - expect(new Duration(durationStr).getSingleSpan()).toEqual('week'); - - durationStr = 'PT5H'; - expect(new Duration(durationStr).getSingleSpan()).toEqual('hour'); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getSingleSpan()).toBeUndefined(); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getSingleSpan()).toBeUndefined(); - }); - }); - - describe('#getSingleSpanValue()', () => { - it('gives back the correct span value', () => { - let durationStr: string; - - durationStr = 'P1D'; - expect(new Duration(durationStr).getSingleSpanValue()).toEqual(1); - - durationStr = 'P3Y'; - expect(new Duration(durationStr).getSingleSpanValue()).toEqual(3); - - durationStr = 'P2W'; - expect(new Duration(durationStr).getSingleSpanValue()).toEqual(2); - - durationStr = 'PT5H'; - expect(new Duration(durationStr).getSingleSpanValue()).toEqual(5); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getSingleSpanValue()).toBeUndefined(); - - durationStr = 'P3DT15H'; - expect(new Duration(durationStr).getSingleSpanValue()).toBeUndefined(); - }); - }); - - describe('#limitToDays', () => { - it('works', () => { - expect(new Duration('P6D').limitToDays().toString()).toEqual('P6D'); - - expect(new Duration('P1M').limitToDays().toString()).toEqual('P30D'); - - expect(new Duration('P1Y').limitToDays().toString()).toEqual('P365D'); - - expect(new Duration('P1Y2M').limitToDays().toString()).toEqual('P425D'); - }); - }); -}); diff --git a/web-console/src/utils/duration/duration.ts b/web-console/src/utils/duration/duration.ts deleted file mode 100755 index bc7236821f12..000000000000 --- a/web-console/src/utils/duration/duration.ts +++ /dev/null @@ -1,388 +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 { second, shifters } from '../date-floor-shift-ceil/date-floor-shift-ceil'; -import { capitalizeFirst, pluralIfNeeded } from '../general'; - -export const TZ_UTC = 'Etc/UTC'; - -export type DurationSpan = 'year' | 'month' | 'week' | 'day' | 'hour' | 'minute' | 'second'; - -const SPANS_WITH_WEEK: DurationSpan[] = [ - 'year', - 'month', - 'week', - 'day', - 'hour', - 'minute', - 'second', -]; -const SPANS_WITHOUT_WEEK: DurationSpan[] = ['year', 'month', 'day', 'hour', 'minute', 'second']; -const SPANS_WITHOUT_WEEK_OR_MONTH: DurationSpan[] = ['year', 'day', 'hour', 'minute', 'second']; -const SPANS_UP_TO_DAY: DurationSpan[] = ['day', 'hour', 'minute', 'second']; - -export type DurationValue = Partial>; - -const periodWeekRegExp = /^P(\d+)W$/; -const periodRegExp = /^P(?:(\d+)Y)?(?:(\d+)M)?(?:(\d+)D)?(?:T(?:(\d+)H)?(?:(\d+)M)?(?:(\d+)S)?)?$/; -// P (year ) (month ) (day ) T(hour ) (minute ) (second ) - -function getSpansFromString(durationStr: string): DurationValue { - const spans: DurationValue = {}; - let matches: RegExpExecArray | null; - if ((matches = periodWeekRegExp.exec(durationStr))) { - spans.week = Number(matches[1]); - if (!spans.week) throw new Error('Duration can not have empty weeks'); - } else if ((matches = periodRegExp.exec(durationStr))) { - const nums = matches.map(Number); - for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { - const span = SPANS_WITHOUT_WEEK[i]; - const value = nums[i + 1]; - if (value) spans[span] = value; - } - } else { - throw new Error("Can not parse duration '" + durationStr + "'"); - } - return spans; -} - -function getSpansFromStartEnd(start: Date, end: Date, timezone: string): DurationValue { - start = second.floor(start, timezone); - end = second.floor(end, timezone); - if (end <= start) throw new Error('start must come before end'); - - const spans: DurationValue = {}; - let iterator: Date = start; - for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { - const span = SPANS_WITHOUT_WEEK[i]; - let spanCount = 0; - - // Shortcut - const length = end.valueOf() - iterator.valueOf(); - const canonicalLength: number = shifters[span].canonicalLength; - if (length < canonicalLength / 4) continue; - const numberToFit = Math.min(0, Math.floor(length / canonicalLength) - 1); - let iteratorMove: Date; - if (numberToFit > 0) { - // try to skip by numberToFit - iteratorMove = shifters[span].shift(iterator, timezone, numberToFit); - if (iteratorMove <= end) { - spanCount += numberToFit; - iterator = iteratorMove; - } - } - - while (true) { - iteratorMove = shifters[span].shift(iterator, timezone, 1); - if (iteratorMove <= end) { - iterator = iteratorMove; - spanCount++; - } else { - break; - } - } - - if (spanCount) { - spans[span] = spanCount; - } - } - return spans; -} - -function removeZeros(spans: DurationValue): DurationValue { - const newSpans: DurationValue = {}; - for (let i = 0; i < SPANS_WITH_WEEK.length; i++) { - const span = SPANS_WITH_WEEK[i]; - if (Number(spans[span]) > 0) { - newSpans[span] = spans[span]; - } - } - return newSpans; -} - -function fitIntoSpans(length: number, spansToCheck: DurationSpan[]): DurationValue { - const spans: DurationValue = {}; - - let lengthLeft = length; - for (let i = 0; i < spansToCheck.length; i++) { - const span = spansToCheck[i]; - const spanLength = shifters[span].canonicalLength; - const count = Math.floor(lengthLeft / spanLength); - - if (count) { - lengthLeft -= spanLength * count; - spans[span] = count; - } - } - - return spans; -} - -/** - * Represents an ISO duration like P1DT3H - */ -export class Duration { - public readonly singleSpan?: DurationSpan; - public readonly spans: Readonly; - - static fromCanonicalLength(length: number, skipMonths = false): Duration { - if (length <= 0) throw new Error('length must be positive'); - let spans = fitIntoSpans(length, skipMonths ? SPANS_WITHOUT_WEEK_OR_MONTH : SPANS_WITHOUT_WEEK); - - if ( - length % shifters['week'].canonicalLength === 0 && // Weeks fits - (Object.keys(spans).length > 1 || // We already have a more complex span - spans['day']) // or... we only have days and it might be simpler to express as weeks - ) { - spans = { week: length / shifters['week'].canonicalLength }; - } - - return new Duration(spans); - } - - static fromCanonicalLengthUpToDays(length: number): Duration { - if (length <= 0) throw new Error('length must be positive'); - return new Duration(fitIntoSpans(length, SPANS_UP_TO_DAY)); - } - - static fromRange(start: Date, end: Date, timezone: string): Duration { - return new Duration(getSpansFromStartEnd(start, end, timezone)); - } - - static pickSmallestGranularityThatFits( - granularities: Duration[], - span: number, - maxEntities: number, - ): Duration { - for (const granularity of granularities) { - if (span / granularity.getCanonicalLength() < maxEntities) return granularity; - } - return granularities[granularities.length - 1]; - } - - constructor(spans: DurationValue | string) { - const effectiveSpans: DurationValue = - typeof spans === 'string' ? getSpansFromString(spans) : removeZeros(spans); - - const usedSpans = Object.keys(effectiveSpans) as DurationSpan[]; - if (!usedSpans.length) throw new Error('Duration can not be empty'); - if (usedSpans.length === 1) { - this.singleSpan = usedSpans[0]; - } else if (effectiveSpans.week) { - throw new Error("Can not mix 'week' and other spans"); - } - this.spans = effectiveSpans; - } - - public toString() { - const strArr: string[] = ['P']; - const spans = this.spans; - if (spans.week) { - strArr.push(String(spans.week), 'W'); - } else { - let addedT = false; - for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { - const span = SPANS_WITHOUT_WEEK[i]; - const value = spans[span]; - if (!value) continue; - if (!addedT && i >= 3) { - strArr.push('T'); - addedT = true; - } - strArr.push(String(value), span[0].toUpperCase()); - } - } - return strArr.join(''); - } - - public add(duration: Duration): Duration { - return Duration.fromCanonicalLength(this.getCanonicalLength() + duration.getCanonicalLength()); - } - - public subtract(duration: Duration): Duration { - const newCanonicalDuration = this.getCanonicalLength() - duration.getCanonicalLength(); - if (newCanonicalDuration < 0) throw new Error('A duration can not be negative.'); - return Duration.fromCanonicalLength(newCanonicalDuration); - } - - public multiply(multiplier: number): Duration { - if (multiplier <= 0) throw new Error('Multiplier must be positive non-zero'); - if (multiplier === 1) return this; - const newCanonicalDuration = this.getCanonicalLength() * multiplier; - return Duration.fromCanonicalLength(newCanonicalDuration); - } - - public valueOf() { - return this.spans; - } - - public equals(other: Duration | undefined): boolean { - return other instanceof Duration && this.toString() === other.toString(); - } - - public isSimple(): boolean { - const { singleSpan } = this; - if (!singleSpan) return false; - return this.spans[singleSpan] === 1; - } - - public isFloorable(): boolean { - const { singleSpan } = this; - if (!singleSpan) return false; - const span = Number(this.spans[singleSpan]); - if (span === 1) return true; - const { siblings } = shifters[singleSpan]; - if (!siblings) return false; - return siblings % span === 0; - } - - /** - * Floors the date according to this duration. - * @param date The date to floor - * @param timezone The timezone within which to floor - */ - public floor(date: Date, timezone: string): Date { - const { singleSpan } = this; - if (!singleSpan) throw new Error('Can not floor on a complex duration'); - const span = this.spans[singleSpan]!; - const mover = shifters[singleSpan]; - let dt = mover.floor(date, timezone); - if (span !== 1) { - if (!mover.siblings) { - throw new Error(`Can not floor on a ${singleSpan} duration that is not 1`); - } - if (mover.siblings % span !== 0) { - throw new Error( - `Can not floor on a ${singleSpan} duration that does not divide into ${mover.siblings}`, - ); - } - dt = mover.round(dt, span, timezone); - } - return dt; - } - - /** - * Moves the given date by 'step' times of the duration - * Negative step value will move back in time. - * @param date The date to move - * @param timezone The timezone within which to make the move - * @param step The number of times to step by the duration - */ - public shift(date: Date, timezone: string, step = 1): Date { - const spans = this.spans; - for (const span of SPANS_WITH_WEEK) { - const value = spans[span]; - if (value) date = shifters[span].shift(date, timezone, step * value); - } - return date; - } - - public ceil(date: Date, timezone: string): Date { - const floored = this.floor(date, timezone); - if (floored.valueOf() === date.valueOf()) return date; // Just like ceil(3) is 3 and not 4 - return this.shift(floored, timezone, 1); - } - - public round(date: Date, timezone: string): Date { - const floorDate = this.floor(date, timezone); - const ceilDate = this.ceil(date, timezone); - const distanceToFloor = Math.abs(date.valueOf() - floorDate.valueOf()); - const distanceToCeil = Math.abs(date.valueOf() - ceilDate.valueOf()); - return distanceToFloor < distanceToCeil ? floorDate : ceilDate; - } - - /** - * Materializes all the values of this duration form start to end - * @param start The date to start on - * @param end The date to start on - * @param timezone The timezone within which to materialize - * @param step The number of times to step by the duration - */ - public materialize(start: Date, end: Date, timezone: string, step = 1): Date[] { - const values: Date[] = []; - let iter = this.floor(start, timezone); - while (iter <= end) { - values.push(iter); - iter = this.shift(iter, timezone, step); - } - return values; - } - - /** - * Checks to see if date is aligned to this duration within the timezone (floors to itself) - * @param date The date to check - * @param timezone The timezone within which to make the check - */ - public isAligned(date: Date, timezone: string): boolean { - return this.floor(date, timezone).valueOf() === date.valueOf(); - } - - /** - * Check to see if this duration can be divided by the given duration - * @param smaller The smaller duration to divide by - */ - public dividesBy(smaller: Duration): boolean { - const myCanonicalLength = this.getCanonicalLength(); - const smallerCanonicalLength = smaller.getCanonicalLength(); - return ( - myCanonicalLength % smallerCanonicalLength === 0 && - this.isFloorable() && - smaller.isFloorable() - ); - } - - public getCanonicalLength(): number { - const spans = this.spans; - let length = 0; - for (const span of SPANS_WITH_WEEK) { - const value = spans[span]; - if (value) length += value * shifters[span].canonicalLength; - } - return length; - } - - public getDescription(capitalize?: boolean): string { - const spans = this.spans; - const description: string[] = []; - for (const span of SPANS_WITH_WEEK) { - const value = spans[span]; - const spanTitle = capitalize ? capitalizeFirst(span) : span; - if (value) { - if (value === 1 && this.singleSpan) { - description.push(spanTitle); - } else { - description.push(pluralIfNeeded(value, spanTitle)); - } - } - } - return description.join(', '); - } - - public getSingleSpan(): string | undefined { - return this.singleSpan; - } - - public getSingleSpanValue(): number | undefined { - if (!this.singleSpan) return; - return this.spans[this.singleSpan]; - } - - public limitToDays(): Duration { - return Duration.fromCanonicalLengthUpToDays(this.getCanonicalLength()); - } -} diff --git a/web-console/src/utils/general.spec.ts b/web-console/src/utils/general.spec.ts index 39deab9420aa..4b97cb19acb0 100644 --- a/web-console/src/utils/general.spec.ts +++ b/web-console/src/utils/general.spec.ts @@ -29,8 +29,8 @@ import { hashJoaat, moveElement, moveToIndex, + objectHash, offsetToRowColumn, - OVERLAY_OPEN_SELECTOR, parseCsvLine, swapElements, } from './general'; @@ -178,6 +178,12 @@ describe('general', () => { }); }); + describe('objectHash', () => { + it('works', () => { + expect(objectHash({ hello: 'world1' })).toEqual('cc14ad13'); + }); + }); + describe('offsetToRowColumn', () => { it('works', () => { const str = 'Hello\nThis is a test\nstring.'; @@ -211,10 +217,4 @@ describe('general', () => { expect(caseInsensitiveEquals(undefined, '')).toEqual(false); }); }); - - describe('OVERLAY_OPEN_SELECTOR', () => { - it('is what it is', () => { - expect(OVERLAY_OPEN_SELECTOR).toEqual('.bp5-portal .bp5-overlay-open'); - }); - }); }); diff --git a/web-console/src/utils/general.tsx b/web-console/src/utils/general.tsx index d128d47f3462..b742013b2e83 100644 --- a/web-console/src/utils/general.tsx +++ b/web-console/src/utils/general.tsx @@ -59,10 +59,6 @@ export function isSimpleArray(a: any): a is (string | number | boolean)[] { ); } -export function arraysEqualByElement(xs: T[], ys: T[]): boolean { - return xs.length === ys.length && xs.every((x, i) => x === ys[i]); -} - export function wait(ms: number): Promise { return new Promise(resolve => { setTimeout(resolve, ms); @@ -255,14 +251,6 @@ export function uniq(array: readonly string[]): string[] { }); } -export function allSameValue(xs: readonly T[]): T | undefined { - const sameValue: T | undefined = xs[0]; - for (let i = 1; i < xs.length; i++) { - if (sameValue !== xs[i]) return; - } - return sameValue; -} - // ---------------------------- export function formatEmpty(str: string): string { @@ -385,14 +373,6 @@ export function formatDurationHybrid(ms: NumberLike): string { } } -export function timezoneOffsetInMinutesToString(offsetInMinutes: number, padHour: boolean): string { - const sign = offsetInMinutes < 0 ? '-' : '+'; - const absOffset = Math.abs(offsetInMinutes); - const h = Math.floor(absOffset / 60); - const m = absOffset % 60; - return `${sign}${padHour ? pad2(h) : h}:${pad2(m)}`; -} - function pluralize(word: string): string { // Ignoring irregular plurals. if (/(s|x|z|ch|sh)$/.test(word)) { @@ -632,10 +612,12 @@ export function hashJoaat(str: string): number { return (hash & 4294967295) >>> 0; } -export const OVERLAY_OPEN_SELECTOR = `.${Classes.PORTAL} .${Classes.OVERLAY_OPEN}`; +export function objectHash(obj: any): string { + return hashJoaat(JSONBig.stringify(obj)).toString(16).padStart(8); +} -export function hasOverlayOpen(): boolean { - return Boolean(document.querySelector(OVERLAY_OPEN_SELECTOR)); +export function hasPopoverOpen(): boolean { + return Boolean(document.querySelector(`${Classes.PORTAL} ${Classes.OVERLAY} ${Classes.POPOVER}`)); } export function checkedCircleIcon(checked: boolean): IconName { diff --git a/web-console/src/utils/index.tsx b/web-console/src/utils/index.tsx index 096f0dfe0634..edea5ad0a520 100644 --- a/web-console/src/utils/index.tsx +++ b/web-console/src/utils/index.tsx @@ -19,12 +19,10 @@ export * from './base64-url'; export * from './column-metadata'; export * from './date'; -export * from './date-floor-shift-ceil/date-floor-shift-ceil'; export * from './download'; export * from './download-query-detail-archive'; export * from './druid-lookup'; export * from './druid-query'; -export * from './duration/duration'; export * from './formatter'; export * from './general'; export * from './local-storage-backed-visibility'; diff --git a/web-console/src/druid-models/load-rule/load-rule.ts b/web-console/src/utils/load-rule.ts similarity index 97% rename from web-console/src/druid-models/load-rule/load-rule.ts rename to web-console/src/utils/load-rule.ts index 31b4d600585a..a32422bbb6a4 100644 --- a/web-console/src/druid-models/load-rule/load-rule.ts +++ b/web-console/src/utils/load-rule.ts @@ -18,7 +18,7 @@ import { sum } from 'd3-array'; -import { deepMove, deepSet } from '../../utils'; +import { deepMove, deepSet } from './object-change'; export type RuleType = | 'loadForever' @@ -41,7 +41,6 @@ export interface Rule { } export class RuleUtil { - static DEFAULT_RULES_KEY = '_default'; static TYPES: RuleType[] = [ 'loadForever', 'loadByInterval', diff --git a/web-console/src/utils/query-action.ts b/web-console/src/utils/query-action.ts index 3f17f22b67d2..45ade6961388 100644 --- a/web-console/src/utils/query-action.ts +++ b/web-console/src/utils/query-action.ts @@ -16,6 +16,6 @@ * limitations under the License. */ -import type { SqlQuery } from 'druid-query-toolkit'; +import type { SqlQuery } from '@druid-toolkit/query'; export type QueryAction = (query: SqlQuery) => SqlQuery; diff --git a/web-console/src/utils/query-manager/query-manager.ts b/web-console/src/utils/query-manager/query-manager.ts index 1023b23412ad..5b141c7c89db 100644 --- a/web-console/src/utils/query-manager/query-manager.ts +++ b/web-console/src/utils/query-manager/query-manager.ts @@ -288,7 +288,7 @@ export class QueryManager { return this.lastQuery; } - public getLastIntermediateQuery(): unknown { + public getLastIntermediateQuery(): any { return this.lastIntermediateQuery; } diff --git a/web-console/src/utils/sampler.ts b/web-console/src/utils/sampler.ts index 5333013c12b4..7fd3d67364cc 100644 --- a/web-console/src/utils/sampler.ts +++ b/web-console/src/utils/sampler.ts @@ -16,8 +16,8 @@ * limitations under the License. */ +import { dedupe, F, SqlExpression, SqlFunction } from '@druid-toolkit/query'; import type { CancelToken } from 'axios'; -import { dedupe, F, SqlExpression, SqlFunction } from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import type { diff --git a/web-console/src/utils/sql.spec.ts b/web-console/src/utils/sql.spec.ts index 57e7bce95fe8..9e5533ee4e62 100644 --- a/web-console/src/utils/sql.spec.ts +++ b/web-console/src/utils/sql.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from 'druid-query-toolkit'; +import { sane } from '@druid-toolkit/query'; import { findAllSqlQueriesInText, findSqlQueryPrefix } from './sql'; diff --git a/web-console/src/utils/sql.ts b/web-console/src/utils/sql.ts index 6c168002078a..1cdf243819d2 100644 --- a/web-console/src/utils/sql.ts +++ b/web-console/src/utils/sql.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { SqlBase } from 'druid-query-toolkit'; +import type { SqlBase } from '@druid-toolkit/query'; import { SqlColumn, SqlExpression, @@ -24,7 +24,7 @@ import { SqlLiteral, SqlQuery, SqlStar, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import type { RowColumn } from './general'; import { offsetToRowColumn } from './general'; diff --git a/web-console/src/utils/table-helpers.ts b/web-console/src/utils/table-helpers.ts index a4ba4fce7ddf..45e8758bf6f8 100644 --- a/web-console/src/utils/table-helpers.ts +++ b/web-console/src/utils/table-helpers.ts @@ -16,10 +16,9 @@ * limitations under the License. */ -import { ascending, descending, sort } from 'd3-array'; -import type { QueryResult, SqlExpression } from 'druid-query-toolkit'; -import { C } from 'druid-query-toolkit'; -import type { Filter, SortingRule } from 'react-table'; +import type { QueryResult, SqlExpression } from '@druid-toolkit/query'; +import { C } from '@druid-toolkit/query'; +import type { Filter } from 'react-table'; import { filterMap, formatNumber, isNumberLike, oneOf } from './general'; import { deepSet } from './object-change'; @@ -68,24 +67,19 @@ export function getNumericColumnBraces( return numericColumnBraces; } +export interface Sorted { + id: string; + desc: boolean; +} + export interface TableState { page: number; pageSize: number; filtered: Filter[]; - sorted: SortingRule[]; + sorted: Sorted[]; } -export function sortedToOrderByClause(sorted: SortingRule[]): string | undefined { +export function sortedToOrderByClause(sorted: Sorted[]): string | undefined { if (!sorted.length) return; return 'ORDER BY ' + sorted.map(sort => `${C(sort.id)} ${sort.desc ? 'DESC' : 'ASC'}`).join(', '); } - -export function applySorting(xs: any[], sorted: SortingRule[]): any[] { - const firstSortingRule = sorted[0]; - if (!firstSortingRule) return xs; - const { id, desc } = firstSortingRule; - return sort( - xs, - desc ? (d1, d2) => descending(d1[id], d2[id]) : (d1, d2) => ascending(d1[id], d2[id]), - ); -} diff --git a/web-console/src/utils/types.ts b/web-console/src/utils/types.ts index a3e38a2c8218..7f48655cb1c4 100644 --- a/web-console/src/utils/types.ts +++ b/web-console/src/utils/types.ts @@ -18,7 +18,7 @@ import type { IconName } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { Column } from 'druid-query-toolkit'; +import type { Column } from '@druid-toolkit/query'; export function columnToSummary(column: Column): string { const lines: string[] = [column.name]; diff --git a/web-console/src/utils/values-query.spec.tsx b/web-console/src/utils/values-query.spec.tsx index 89f4a4d2f14f..7bc093bc3e82 100644 --- a/web-console/src/utils/values-query.spec.tsx +++ b/web-console/src/utils/values-query.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { QueryResult, sane } from 'druid-query-toolkit'; +import { QueryResult, sane } from '@druid-toolkit/query'; import { queryResultToValuesQuery } from './values-query'; diff --git a/web-console/src/utils/values-query.tsx b/web-console/src/utils/values-query.tsx index 69130d380e75..1b5e62b44c23 100644 --- a/web-console/src/utils/values-query.tsx +++ b/web-console/src/utils/values-query.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { Column, QueryResult, SqlExpression } from 'druid-query-toolkit'; +import type { Column, QueryResult, SqlExpression } from '@druid-toolkit/query'; import { C, F, @@ -28,7 +28,7 @@ import { SqlRecord, SqlType, SqlValues, -} from 'druid-query-toolkit'; +} from '@druid-toolkit/query'; import * as JSONBig from 'json-bigint-native'; import { oneOf } from './general'; diff --git a/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap b/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap index 5f471fa1e390..6da41f6eb763 100644 --- a/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap +++ b/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap @@ -109,7 +109,6 @@ exports[`DatasourcesView matches snapshot 1`] = ` /> { + datasources = datasourcesResp.data.map((d: any): DatasourceQueryResultRow => { const totalDataSize = deepGet(d, 'properties.segments.size') || -1; const segmentsToLoad = Number(loadstatus[d.name] || 0); const availableSegments = Number(deepGet(d, 'properties.segments.count')); @@ -531,10 +521,9 @@ GROUP BY 1, 2`; if (capabilities.hasOverlordAccess()) { auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => { try { - const taskList = await getApiArray( - `/druid/indexer/v1/tasks?state=running`, - cancelToken, - ); + const taskList = ( + await Api.instance.get(`/druid/indexer/v1/tasks?state=running`, { cancelToken }) + ).data; const runningTasksByDatasource = groupByAsMap( taskList, @@ -573,10 +562,10 @@ GROUP BY 1, 2`; if (showUnused) { try { unused = ( - await getApiArray( + await Api.instance.get( '/druid/coordinator/v1/metadata/datasources?includeUnused', ) - ).filter(d => !seen[d]); + ).data.filter(d => !seen[d]); } catch { AppToaster.show({ icon: IconNames.ERROR, @@ -589,7 +578,7 @@ GROUP BY 1, 2`; // Rules auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => { try { - const rules = ( + const rules: Record = ( await Api.instance.get>('/druid/coordinator/v1/rules', { cancelToken, }) @@ -600,7 +589,7 @@ GROUP BY 1, 2`; ...ds, rules: rules[ds.datasource] || [], })), - defaultRules: rules[RuleUtil.DEFAULT_RULES_KEY], + defaultRules: rules[DEFAULT_RULES_KEY], }; } catch { AppToaster.show({ @@ -668,19 +657,8 @@ GROUP BY 1, 2`; } private readonly refresh = (auto: boolean): void => { - if (auto && hasOverlayOpen()) return; + if (auto && hasPopoverOpen()) return; this.datasourceQueryManager.rerunLastQuery(auto); - - const { showSegmentTimeline } = this.state; - if (showSegmentTimeline) { - // Create a new capabilities object to force the segment timeline to re-render - this.setState(({ showSegmentTimeline }) => ({ - showSegmentTimeline: { - ...showSegmentTimeline, - capabilities: this.props.capabilities.clone(), - }, - })); - } }; private fetchDatasourceData() { @@ -868,9 +846,9 @@ GROUP BY 1, 2`; { - if (typeof lastDatasourcesQuery !== 'string') return; + if (!lastDatasourcesQuery) return; goToQuery({ queryString: lastDatasourcesQuery }); }} /> @@ -943,7 +921,6 @@ GROUP BY 1, 2`; retentionDialogOpenOn: { datasource: '_default', rules: defaultRules, - defaultRules, }, }; }); @@ -1053,13 +1030,10 @@ GROUP BY 1, 2`; icon: IconNames.AUTOMATIC_UPDATES, title: 'Edit retention rules', onAction: () => { - const defaultRules = this.state.datasourcesAndDefaultRulesState.data?.defaultRules; - if (!defaultRules) return; this.setState({ retentionDialogOpenOn: { datasource, rules: rules || [], - defaultRules, }, }); }, @@ -1123,8 +1097,9 @@ GROUP BY 1, 2`; private renderRetentionDialog() { const { capabilities } = this.props; - const { retentionDialogOpenOn } = this.state; - if (!retentionDialogOpenOn) return; + const { retentionDialogOpenOn, datasourcesAndDefaultRulesState } = this.state; + const defaultRules = datasourcesAndDefaultRulesState.data?.defaultRules; + if (!retentionDialogOpenOn || !defaultRules) return; return ( this.setState({ retentionDialogOpenOn: undefined })} onSave={this.saveRules} /> @@ -1164,9 +1139,8 @@ GROUP BY 1, 2`; } private renderDatasourcesTable() { - const { goToTasks, capabilities, filters, onFiltersChange } = this.props; - const { datasourcesAndDefaultRulesState, showUnused, visibleColumns, showSegmentTimeline } = - this.state; + const { goToSegments, goToTasks, capabilities, filters, onFiltersChange } = this.props; + const { datasourcesAndDefaultRulesState, showUnused, visibleColumns } = this.state; let { datasources, defaultRules } = datasourcesAndDefaultRulesState.data || { datasources: [] }; @@ -1220,19 +1194,12 @@ GROUP BY 1, 2`; show: visibleColumns.shown('Datasource name'), accessor: 'datasource', width: 150, - Cell: ({ value, original }) => ( + Cell: row => ( this.onDetail(original)} + onClick={() => this.onDetail(row.original)} hoverIcon={IconNames.SEARCH_TEMPLATE} - tooltip="Show detail" > - {showSegmentTimeline ? ( - <> - {value} - - ) : ( - value - )} + {row.value} ), }, @@ -1258,12 +1225,7 @@ GROUP BY 1, 2`; const hasZeroReplicationRule = RuleUtil.hasZeroReplicaRule(rules, defaultRules); const descriptor = hasZeroReplicationRule ? 'pre-cached' : 'available'; const segmentsEl = ( - - this.setState({ showSegmentTimeline: { capabilities, datasource } }) - } - data-tooltip="Show in segment timeline" - > + goToSegments(datasource)}> {pluralIfNeeded(num_segments, 'segment')} ); @@ -1356,7 +1318,7 @@ GROUP BY 1, 2`; goToTasks(original.datasource)} hoverIcon={IconNames.ARROW_TOP_RIGHT} - tooltip="Go to tasks" + title="Go to tasks" > {formatRunningTasks(runningTasks)} @@ -1536,7 +1498,6 @@ GROUP BY 1, 2`; if (!compaction) return; return ( { if (!compaction) return; @@ -1653,7 +1614,6 @@ GROUP BY 1, 2`; return ( { if (!defaultRules) return; @@ -1661,7 +1621,6 @@ GROUP BY 1, 2`; retentionDialogOpenOn: { datasource, rules, - defaultRules, }, }); }} @@ -1709,7 +1668,7 @@ GROUP BY 1, 2`; } render() { - const { capabilities, goToSegments } = this.props; + const { capabilities } = this.props; const { showUnused, visibleColumns, @@ -1722,7 +1681,9 @@ GROUP BY 1, 2`;
{ + this.refresh(auto); + }} localStorageKey={LocalStorageKeys.DATASOURCES_REFRESH_RATE} /> {this.renderBulkDatasourceActions()} @@ -1733,13 +1694,9 @@ GROUP BY 1, 2`; disabled={!capabilities.hasCoordinatorAccess()} /> - this.setState({ - showSegmentTimeline: showSegmentTimeline ? undefined : { capabilities }, - }) - } + onChange={() => this.setState({ showSegmentTimeline: !showSegmentTimeline })} disabled={!capabilities.hasSqlOrCoordinatorAccess()} /> - {showSegmentTimeline && ( - { - return ( -
+ )} +
{openOn.text}
+ {!minimal && ( +
+ )} +
, + document.body, + ); +}; diff --git a/web-console/src/components/record-table-pane/record-table-pane.tsx b/web-console/src/components/record-table-pane/record-table-pane.tsx index e44eab355602..803ee783c2cd 100644 --- a/web-console/src/components/record-table-pane/record-table-pane.tsx +++ b/web-console/src/components/record-table-pane/record-table-pane.tsx @@ -18,8 +18,8 @@ import { Button, Icon, Popover } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { Column, QueryResult } from '@druid-toolkit/query'; import classNames from 'classnames'; +import type { Column, QueryResult } from 'druid-query-toolkit'; import React, { useEffect, useState } from 'react'; import type { RowRenderProps } from 'react-table'; import ReactTable from 'react-table'; diff --git a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap index 2769c8af739e..7246fe89490f 100644 --- a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap +++ b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap @@ -70,7 +70,7 @@ exports[`RuleEditor matches snapshot no tier in rule 1`] = ` style="transform: translateY(0); transition: none;" >
- + - - -`; diff --git a/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap b/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap index ae30df246c2b..206c2be703bd 100644 --- a/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap +++ b/web-console/src/components/segment-timeline/__snapshots__/segment-timeline.spec.tsx.snap @@ -4,191 +4,309 @@ exports[`SegmentTimeline matches snapshot 1`] = `
-
-
- -
-
-
+ + + +
- - -
-
+ + +
- -
+ 1D + + + + + + + + + - -
+ + + + + 2024-11-01 → 2024-11-18 + + + +
+
`; diff --git a/web-console/src/components/segment-timeline/bar-group.tsx b/web-console/src/components/segment-timeline/bar-group.tsx deleted file mode 100644 index d0cf867e2b2d..000000000000 --- a/web-console/src/components/segment-timeline/bar-group.tsx +++ /dev/null @@ -1,75 +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 { AxisScale } from 'd3-axis'; -import React from 'react'; - -import { BarUnit } from './bar-unit'; -import type { BarUnitData, HoveredBarInfo } from './stacked-bar-chart'; - -interface BarGroupProps { - dataToRender: BarUnitData[]; - changeActiveDatasource: (dataSource: string) => void; - formatTick: (e: number) => string; - xScale: AxisScale; - yScale: AxisScale; - barWidth: number; - onHoverBar?: (e: any) => void; - offHoverBar?: () => void; - hoverOn?: HoveredBarInfo | null; -} - -export class BarGroup extends React.Component { - shouldComponentUpdate(nextProps: BarGroupProps): boolean { - return nextProps.hoverOn === this.props.hoverOn; - } - - render() { - const { dataToRender, changeActiveDatasource, xScale, yScale, onHoverBar, barWidth } = - this.props; - if (dataToRender === undefined) return null; - - return dataToRender.map((entry: BarUnitData, i: number) => { - const y0 = yScale(entry.y0 || 0) || 0; - const x = xScale(new Date(entry.x + 'T00:00:00Z')); - const y = yScale((entry.y0 || 0) + entry.y) || 0; - const height = Math.max(y0 - y, 0); - const barInfo: HoveredBarInfo = { - xCoordinate: x, - yCoordinate: y, - height, - datasource: entry.datasource, - xValue: entry.x, - yValue: entry.y, - dailySize: entry.dailySize, - }; - return ( - changeActiveDatasource(entry.datasource)} - onHover={() => onHoverBar && onHoverBar(barInfo)} - /> - ); - }); - } -} diff --git a/web-console/src/components/segment-timeline/chart-axis.tsx b/web-console/src/components/segment-timeline/chart-axis.tsx index bc333d33b773..b8ee4e9cbb11 100644 --- a/web-console/src/components/segment-timeline/chart-axis.tsx +++ b/web-console/src/components/segment-timeline/chart-axis.tsx @@ -16,22 +16,22 @@ * limitations under the License. */ +import type { Axis } from 'd3-axis'; import { select } from 'd3-selection'; -import React from 'react'; interface ChartAxisProps { - transform?: string; - scale: any; className?: string; + transform?: string; + axis: Axis; } -export const ChartAxis = React.memo(function ChartAxis(props: ChartAxisProps) { - const { transform, scale, className } = props; +export const ChartAxis = function ChartAxis(props: ChartAxisProps) { + const { transform, axis, className } = props; return ( select(node).call(scale)} + ref={node => select(node).call(axis as any)} /> ); -}); +}; diff --git a/web-console/src/components/segment-timeline/common.ts b/web-console/src/components/segment-timeline/common.ts new file mode 100644 index 000000000000..48aa5ce48d46 --- /dev/null +++ b/web-console/src/components/segment-timeline/common.ts @@ -0,0 +1,87 @@ +/* + * 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 { sum } from 'd3-array'; + +import type { Duration } from '../../utils'; +import { formatBytes, formatInteger } from '../../utils'; + +export type IntervalStat = 'segments' | 'size' | 'rows'; + +export const INTERVAL_STATS: IntervalStat[] = ['segments', 'size', 'rows']; + +export function getIntervalStatTitle(intervalStat: IntervalStat): string { + switch (intervalStat) { + case 'segments': + return 'Num. segments'; + + case 'size': + return 'Size'; + + case 'rows': + return 'Rows'; + + default: + return intervalStat; + } +} + +export function aggregateSegmentStats( + xs: readonly Record[], +): Record { + return { + segments: sum(xs, s => s.segments), + size: sum(xs, s => s.size), + rows: sum(xs, s => s.rows), + }; +} + +export function formatIntervalStat(stat: IntervalStat, n: number) { + switch (stat) { + case 'segments': + case 'rows': + return formatInteger(n); + + case 'size': + return formatBytes(n); + + default: + return ''; + } +} + +export interface IntervalRow extends Record { + start: Date; + end: Date; + datasource: string; + realtime: boolean; + originalTimeSpan: Duration; +} + +export interface TrimmedIntervalRow extends IntervalRow { + shownDays: number; + normalized: Record; +} + +export interface IntervalBar extends TrimmedIntervalRow { + offset: Record; +} + +export function formatIsoDateOnly(date: Date): string { + return date.toISOString().slice(0, 10); +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart-render.scss b/web-console/src/components/segment-timeline/segment-bar-chart-render.scss new file mode 100644 index 000000000000..762772a0f5c5 --- /dev/null +++ b/web-console/src/components/segment-timeline/segment-bar-chart-render.scss @@ -0,0 +1,169 @@ +/* + * 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'; + +.segment-bar-chart-render { + position: relative; + overflow: hidden; + + @keyframes pulseOpacity { + 0% { + opacity: 0.8; + } + 100% { + opacity: 0.95; + } + } + + svg { + position: absolute; + + .chart-axis text { + user-select: none; + } + + .hover-highlight { + fill: white; + fill-opacity: 0.1; + } + + .hovered-bar { + fill: none; + stroke: #ffffff; + stroke-width: 1.5px; + } + + .selection { + fill: transparent; + stroke: #ffffff; + stroke-width: 1px; + opacity: 0.8; + + &.done { + opacity: 1; + } + } + + .shifter { + fill: white; + fill-opacity: 0.2; + filter: blur(1px); + } + + .time-shift-indicator { + fill: white; + fill-opacity: 0.001; + cursor: grab; + + &:hover { + fill-opacity: 0.1; + } + + &.shifting { + fill-opacity: 0.2; + cursor: grabbing; + } + } + + .gridline-x { + line { + stroke-dasharray: 5, 5; + opacity: 0.5; + } + } + + .now-line { + stroke: $orange4; + stroke-dasharray: 2, 2; + opacity: 0.7; + } + + .bar-unit { + &.realtime { + animation: pulseOpacity 3s alternate infinite; + } + } + } + + .rule-tape { + position: absolute; + top: 5px; + height: 15px; + font-size: 10px; + + .rule-error { + @include pin-full(); + background-color: $red3; + color: $white; + } + + .load-rule { + position: absolute; + overflow: hidden; + padding-left: 2px; + border-left: 1px solid $dark-gray2; + border-right: 1px solid $dark-gray2; + top: 0; + height: 100%; + text-overflow: ellipsis; + + &.load { + background-color: $green1; + + &:nth-child(even) { + background-color: $green3; + } + } + + &.drop { + background-color: $dark-gray5; + + &:nth-child(even) { + background-color: $gray1; + } + } + + &.broadcast { + background-color: $indigo1; + + &:nth-child(even) { + background-color: $indigo3; + } + } + } + } + + .empty-placeholder { + @include pin-full; + display: flex; + align-items: center; + justify-content: center; + font-size: 20px; + user-select: none; + pointer-events: none; + } +} + +.segment-bar-chart-bubble { + .button-bar { + padding-top: 5px; + display: flex; + gap: 5px; + } +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx b/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx new file mode 100644 index 000000000000..11c7d36834f5 --- /dev/null +++ b/web-console/src/components/segment-timeline/segment-bar-chart-render.tsx @@ -0,0 +1,793 @@ +/* + * 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, Intent } from '@blueprintjs/core'; +import type { NonNullDateRange } from '@blueprintjs/datetime'; +import { IconNames } from '@blueprintjs/icons'; +import IntervalTree from '@flatten-js/interval-tree'; +import classNames from 'classnames'; +import { max, sort, sum } from 'd3-array'; +import { axisBottom, axisLeft } from 'd3-axis'; +import { scaleLinear, scaleUtc } from 'd3-scale'; +import type { MouseEvent as ReactMouseEvent, ReactNode } from 'react'; +import { useMemo, useRef, useState } from 'react'; + +import type { Rule } from '../../druid-models'; +import { getDatasourceColor, RuleUtil } from '../../druid-models'; +import { useClock, useGlobalEventListener } from '../../hooks'; +import { + allSameValue, + arraysEqualByElement, + clamp, + day, + Duration, + formatBytes, + formatNumber, + groupBy, + groupByAsMap, + minute, + month, + pluralIfNeeded, + TZ_UTC, + uniq, +} from '../../utils'; +import type { Margin, Stage } from '../../utils/stage'; +import type { PortalBubbleOpenOn } from '../portal-bubble/portal-bubble'; +import { PortalBubble } from '../portal-bubble/portal-bubble'; + +import { ChartAxis } from './chart-axis'; +import type { IntervalBar, IntervalRow, IntervalStat, TrimmedIntervalRow } from './common'; +import { aggregateSegmentStats, formatIntervalStat, formatIsoDateOnly } from './common'; + +import './segment-bar-chart-render.scss'; + +const CHART_MARGIN: Margin = { top: 20, right: 0, bottom: 25, left: 70 }; +const MIN_BAR_WIDTH = 4; +const POSSIBLE_GRANULARITIES = [ + new Duration('PT15M'), + new Duration('PT1H'), + new Duration('PT6H'), + new Duration('P1D'), + new Duration('P1M'), + new Duration('P1Y'), +]; + +const EXTEND_X_SCALE_DOMAIN_BY = 1; + +function formatStartDuration(start: Date, duration: Duration): string { + let sliceLength; + const { singleSpan } = duration; + switch (singleSpan) { + case 'year': + sliceLength = 4; + break; + + case 'month': + sliceLength = 7; + break; + + case 'day': + sliceLength = 10; + break; + + case 'hour': + sliceLength = 13; + break; + + case 'minute': + sliceLength = 16; + break; + + default: + sliceLength = 19; + break; + } + + return `${start.toISOString().slice(0, sliceLength)}/${duration}`; +} + +// --------------------------------------- +// Load rule stuff + +function loadRuleToBaseType(loadRule: Rule): string { + const m = /^(load|drop|broadcast)/.exec(loadRule.type); + return m ? m[1] : 'load'; +} + +const NEGATIVE_INFINITY_DATE = new Date(Date.UTC(1000, 0, 1)); +const POSITIVE_INFINITY_DATE = new Date(Date.UTC(3000, 0, 1)); + +function loadRuleToDateRange(loadRule: Rule): NonNullDateRange { + switch (loadRule.type) { + case 'loadByInterval': + case 'dropByInterval': + case 'broadcastByInterval': + return String(loadRule.interval) + .split('/') + .map(d => new Date(d)) as NonNullDateRange; + + case 'loadByPeriod': + case 'dropByPeriod': + case 'broadcastByPeriod': + return [ + new Duration(loadRule.period || 'P1D').shift(new Date(), TZ_UTC, -1), + loadRule.includeFuture ? POSITIVE_INFINITY_DATE : new Date(), + ]; + + case 'dropBeforeByPeriod': + return [ + NEGATIVE_INFINITY_DATE, + new Duration(loadRule.period || 'P1D').shift(new Date(), TZ_UTC, -1), + ]; + + default: + return [NEGATIVE_INFINITY_DATE, POSITIVE_INFINITY_DATE]; + } +} + +// --------------------------------------- + +function offsetDateRange(dateRange: NonNullDateRange, offset: number): NonNullDateRange { + return [new Date(dateRange[0].valueOf() + offset), new Date(dateRange[1].valueOf() + offset)]; +} + +function stackIntervalRows(trimmedIntervalRows: TrimmedIntervalRow[]): { + intervalBars: IntervalBar[]; + intervalTree: IntervalTree; +} { + // Total size of the datasource will be used as an ordering tiebreaker + const datasourceToTotalSize = groupByAsMap( + trimmedIntervalRows, + intervalRow => intervalRow.datasource, + intervalRows => sum(intervalRows, intervalRow => intervalRow.size), + ); + + const sortedIntervalRows = sort(trimmedIntervalRows, (a, b) => { + const shownDaysDiff = b.shownDays - a.shownDays; + if (shownDaysDiff) return shownDaysDiff; + + const timeSpanDiff = + b.originalTimeSpan.getCanonicalLength() - a.originalTimeSpan.getCanonicalLength(); + if (timeSpanDiff) return timeSpanDiff; + + const totalSizeDiff = datasourceToTotalSize[b.datasource] - datasourceToTotalSize[a.datasource]; + if (totalSizeDiff) return totalSizeDiff; + + return Number(a.realtime) - Number(b.realtime); + }); + + const intervalTree = new IntervalTree(); + const intervalBars = sortedIntervalRows.map(intervalRow => { + const startMs = intervalRow.start.valueOf(); + const endMs = intervalRow.end.valueOf(); + const intervalRowsBelow = intervalTree.search([startMs + 1, startMs + 2]) as IntervalBar[]; + const intervalBar: IntervalBar = { + ...intervalRow, + offset: aggregateSegmentStats(intervalRowsBelow.map(i => i.normalized)), + }; + intervalTree.insert([startMs, endMs], intervalBar); + return intervalBar; + }); + + return { + intervalBars, + intervalTree, + }; +} + +interface BubbleInfo { + start: Date; + end: Date; + timeLabel: string; + intervalBars: IntervalBar[]; +} + +interface SelectionRange { + start: Date; + end: Date; + done?: boolean; +} + +export interface DatasourceRules { + loadRules: Rule[]; + defaultLoadRules: Rule[]; +} + +export interface SegmentBarChartRenderProps { + intervalRows: IntervalRow[]; + datasourceRules: DatasourceRules | undefined; + datasourceRulesError: string | undefined; + + stage: Stage; + dateRange: NonNullDateRange; + changeDateRange(dateRange: NonNullDateRange): void; + shownIntervalStat: IntervalStat; + shownDatasource: string | undefined; + changeShownDatasource(datasource: string | undefined): void; + getIntervalActionButton?( + start: Date, + end: Date, + datasource?: string, + realtime?: boolean, + ): ReactNode; +} + +export const SegmentBarChartRender = function SegmentBarChartRender( + props: SegmentBarChartRenderProps, +) { + const { + intervalRows, + datasourceRules, + datasourceRulesError, + + stage, + shownIntervalStat, + dateRange, + changeDateRange, + shownDatasource, + changeShownDatasource, + getIntervalActionButton, + } = props; + const [mouseDownAt, setMouseDownAt] = useState< + { time: Date; action: 'select' | 'shift' } | undefined + >(); + const [selection, setSelection] = useState(); + + function setSelectionIfNeeded(newSelection: SelectionRange) { + if ( + selection && + selection.start.valueOf() === newSelection.start.valueOf() && + selection.end.valueOf() === newSelection.end.valueOf() && + selection.done === newSelection.done + ) { + return; + } + setSelection(newSelection); + } + + const [bubbleInfo, setBubbleInfo] = useState(); + + function setBubbleInfoIfNeeded(newBubbleInfo: BubbleInfo) { + if ( + bubbleInfo && + bubbleInfo.start.valueOf() === newBubbleInfo.start.valueOf() && + bubbleInfo.end.valueOf() === newBubbleInfo.end.valueOf() && + bubbleInfo.timeLabel === newBubbleInfo.timeLabel && + arraysEqualByElement(bubbleInfo.intervalBars, newBubbleInfo.intervalBars) + ) { + return; + } + setBubbleInfo(newBubbleInfo); + } + + const [shiftOffset, setShiftOffset] = useState(); + + const now = useClock(minute.canonicalLength); + const svgRef = useRef(null); + + const trimGranularity = useMemo(() => { + return Duration.pickSmallestGranularityThatFits( + POSSIBLE_GRANULARITIES, + dateRange[1].valueOf() - dateRange[0].valueOf(), + Math.floor(stage.width / MIN_BAR_WIDTH), + ).toString(); + }, [dateRange, stage.width]); + + const { intervalBars, intervalTree } = useMemo(() => { + const shownIntervalRows = intervalRows.filter( + ({ start, end, datasource }) => + start <= dateRange[1] && + dateRange[0] < end && + (!shownDatasource || datasource === shownDatasource), + ); + const averageRowSizeByDatasource = groupByAsMap( + shownIntervalRows.filter(intervalRow => intervalRow.size > 0 && intervalRow.rows > 0), + intervalRow => intervalRow.datasource, + intervalRows => sum(intervalRows, d => d.size) / sum(intervalRows, d => d.rows), + ); + + const trimDuration = new Duration(trimGranularity); + const trimmedIntervalRows = shownIntervalRows.map(intervalRow => { + const { start, end, segments, size, rows } = intervalRow; + const startTrimmed = trimDuration.floor(start, TZ_UTC); + let endTrimmed = trimDuration.ceil(end, TZ_UTC); + + // Special handling to catch WEEK intervals when trimming to month. + if (trimGranularity === 'P1M' && intervalRow.originalTimeSpan.toString() === 'P7D') { + endTrimmed = trimDuration.shift(startTrimmed, TZ_UTC); + } + + const shownDays = (endTrimmed.valueOf() - startTrimmed.valueOf()) / day.canonicalLength; + const shownSize = + size === 0 ? rows * averageRowSizeByDatasource[intervalRow.datasource] : size; + return { + ...intervalRow, + start: startTrimmed, + end: endTrimmed, + shownDays, + size: shownSize, + normalized: { + size: shownSize / shownDays, + rows: rows / shownDays, + segments: segments / shownDays, + }, + }; + }); + + const fullyGroupedSegmentRows = groupBy( + trimmedIntervalRows, + trimmedIntervalRow => + [ + trimmedIntervalRow.start.toISOString(), + trimmedIntervalRow.end.toISOString(), + trimmedIntervalRow.originalTimeSpan, + trimmedIntervalRow.datasource, + trimmedIntervalRow.realtime, + ].join('/'), + (trimmedIntervalRows): TrimmedIntervalRow => { + const firstIntervalRow = trimmedIntervalRows[0]; + return { + ...firstIntervalRow, + ...aggregateSegmentStats(trimmedIntervalRows), + normalized: aggregateSegmentStats(trimmedIntervalRows.map(t => t.normalized)), + }; + }, + ); + + return stackIntervalRows(fullyGroupedSegmentRows); + }, [intervalRows, trimGranularity, dateRange, shownDatasource]); + + const innerStage = stage.applyMargin(CHART_MARGIN); + + const baseTimeScale = scaleUtc() + .domain(dateRange) + .range([EXTEND_X_SCALE_DOMAIN_BY, innerStage.width - EXTEND_X_SCALE_DOMAIN_BY]); + const timeScale = shiftOffset + ? baseTimeScale.copy().domain(offsetDateRange(dateRange, shiftOffset)) + : baseTimeScale; + + const maxNormalizedStat = max( + intervalBars, + d => d.normalized[shownIntervalStat] + d.offset[shownIntervalStat], + ); + const statScale = scaleLinear() + .rangeRound([innerStage.height, 0]) + .domain([0, (maxNormalizedStat ?? 1) * 1.05]); + + const formatTickRate = (n: number) => { + switch (shownIntervalStat) { + case 'segments': + return formatNumber(n); // + ' seg/day'; + + case 'rows': + return formatNumber(n); // + ' row/day'; + + case 'size': + return formatBytes(n); + } + }; + + function handleMouseDown(e: ReactMouseEvent) { + const svg = svgRef.current; + if (!svg) return; + e.preventDefault(); + + if (selection) { + setSelection(undefined); + } else { + const rect = svg.getBoundingClientRect(); + const x = e.clientX - rect.x - CHART_MARGIN.left; + const y = e.clientY - rect.y - CHART_MARGIN.top; + const time = baseTimeScale.invert(x); + const action = y > innerStage.height || e.shiftKey ? 'shift' : 'select'; + setBubbleInfo(undefined); + setMouseDownAt({ + time, + action, + }); + } + } + + useGlobalEventListener('mousemove', (e: MouseEvent) => { + const svg = svgRef.current; + if (!svg) return; + const rect = svg.getBoundingClientRect(); + const x = e.clientX - rect.x - CHART_MARGIN.left; + const y = e.clientY - rect.y - CHART_MARGIN.top; + + if (mouseDownAt) { + e.preventDefault(); + + const b = baseTimeScale.invert(x); + if (mouseDownAt.action === 'shift' || e.shiftKey) { + setShiftOffset(mouseDownAt.time.valueOf() - b.valueOf()); + } else { + if (mouseDownAt.time < b) { + setSelectionIfNeeded({ + start: day.floor(mouseDownAt.time, TZ_UTC), + end: day.ceil(b, TZ_UTC), + }); + } else { + setSelectionIfNeeded({ + start: day.floor(b, TZ_UTC), + end: day.ceil(mouseDownAt.time, TZ_UTC), + }); + } + } + } else if (!selection) { + if ( + 0 <= x && + x <= innerStage.width && + 0 <= y && + y <= innerStage.height + CHART_MARGIN.bottom + ) { + const time = baseTimeScale.invert(x); + const shifter = + new Duration(trimGranularity).getCanonicalLength() > day.canonicalLength * 25 + ? month + : day; + const start = shifter.floor(time, TZ_UTC); + const end = shifter.ceil(time, TZ_UTC); + + let intervalBars: IntervalBar[] = []; + if (y <= innerStage.height) { + const bars = intervalTree.search([ + time.valueOf() + 1, + time.valueOf() + 2, + ]) as IntervalBar[]; + + if (bars.length) { + const stat = statScale.invert(y); + const hoverBar = bars.find( + bar => + bar.offset[shownIntervalStat] <= stat && + stat < bar.offset[shownIntervalStat] + bar.normalized[shownIntervalStat], + ); + intervalBars = hoverBar ? [hoverBar] : bars; + } + } + setBubbleInfoIfNeeded({ + start, + end, + timeLabel: start.toISOString().slice(0, shifter === day ? 10 : 7), + intervalBars, + }); + } else { + setBubbleInfo(undefined); + } + } + }); + + useGlobalEventListener('mouseup', (e: MouseEvent) => { + if (!mouseDownAt) return; + e.preventDefault(); + setMouseDownAt(undefined); + + const svg = svgRef.current; + if (!svg) return; + const rect = svg.getBoundingClientRect(); + const x = e.clientX - rect.x - CHART_MARGIN.left; + const y = e.clientY - rect.y - CHART_MARGIN.top; + + if (shiftOffset || selection) { + setShiftOffset(undefined); + if (mouseDownAt.action === 'shift' || e.shiftKey) { + if (shiftOffset) { + changeDateRange(offsetDateRange(dateRange, shiftOffset)); + } + } else { + if (selection) { + setSelection({ ...selection, done: true }); + } + } + } else if (0 <= x && x <= innerStage.width && 0 <= y && y <= innerStage.height) { + const time = baseTimeScale.invert(x); + + const bars = intervalTree.search([time.valueOf() + 1, time.valueOf() + 2]) as IntervalBar[]; + + if (bars.length) { + const stat = statScale.invert(y); + const hoverBar = bars.find( + bar => + bar.offset[shownIntervalStat] <= stat && + stat < bar.offset[shownIntervalStat] + bar.normalized[shownIntervalStat], + ); + if (hoverBar) { + changeShownDatasource(shownDatasource ? undefined : hoverBar.datasource); + } + } + } + }); + + useGlobalEventListener('keydown', (e: KeyboardEvent) => { + if (e.key === 'Escape' && mouseDownAt) { + setMouseDownAt(undefined); + setSelection(undefined); + } + }); + + function startEndToXWidth({ start, end }: { start: Date; end: Date }) { + const xStart = clamp(timeScale(start), 0, innerStage.width); + const xEnd = clamp(timeScale(end), 0, innerStage.width); + + return { + x: xStart, + width: Math.max(xEnd - xStart - 1, 1), + }; + } + + function segmentBarToRect(intervalBar: IntervalBar) { + const y0 = statScale(intervalBar.offset[shownIntervalStat]); + const y = statScale( + intervalBar.normalized[shownIntervalStat] + intervalBar.offset[shownIntervalStat], + ); + + return { + ...startEndToXWidth(intervalBar), + y: y, + height: y0 - y, + }; + } + + let hoveredOpenOn: PortalBubbleOpenOn | undefined; + if (svgRef.current) { + const rect = svgRef.current.getBoundingClientRect(); + + if (bubbleInfo) { + const hoveredIntervalBars = bubbleInfo.intervalBars; + + let title: string | undefined; + let text: ReactNode; + if (hoveredIntervalBars.length === 0) { + title = bubbleInfo.timeLabel; + text = ''; + } else if (hoveredIntervalBars.length === 1) { + const hoveredIntervalBar = hoveredIntervalBars[0]; + title = `${formatStartDuration( + hoveredIntervalBar.start, + hoveredIntervalBar.originalTimeSpan, + )}${hoveredIntervalBar.realtime ? ' (realtime)' : ''}`; + text = ( + <> + {!shownDatasource &&
{`Datasource: ${hoveredIntervalBar.datasource}`}
} +
{`Size: ${ + hoveredIntervalBar.realtime + ? 'estimated for realtime' + : formatIntervalStat('size', hoveredIntervalBar.size) + }`}
+
{`Rows: ${formatIntervalStat('rows', hoveredIntervalBar.rows)}`}
+
{`Segments: ${formatIntervalStat('segments', hoveredIntervalBar.segments)}`}
+ + ); + } else { + const datasources = uniq(hoveredIntervalBars.map(b => b.datasource)); + const agg = aggregateSegmentStats(hoveredIntervalBars); + title = bubbleInfo.timeLabel; + text = ( + <> + {!shownDatasource && ( +
{`Totals for ${pluralIfNeeded(datasources.length, 'datasource')}`}
+ )} +
{`Size: ${formatIntervalStat('size', agg.size)}`}
+
{`Rows: ${formatIntervalStat('rows', agg.rows)}`}
+
{`Segments: ${formatIntervalStat('segments', agg.segments)}`}
+ + ); + } + + hoveredOpenOn = { + x: + rect.x + + CHART_MARGIN.left + + timeScale(new Date((bubbleInfo.start.valueOf() + bubbleInfo.end.valueOf()) / 2)), + y: rect.y + CHART_MARGIN.top, + title, + text, + }; + } else if (selection) { + const selectedBars = intervalTree.search([ + selection.start.valueOf() + 1, + selection.end.valueOf() - 1, + ]) as IntervalBar[]; + const datasources = uniq(selectedBars.map(b => b.datasource)); + const realtime = allSameValue(selectedBars.map(b => b.realtime)); + const agg = aggregateSegmentStats(selectedBars); + hoveredOpenOn = { + x: + rect.x + + CHART_MARGIN.left + + timeScale(new Date((selection.start.valueOf() + selection.end.valueOf()) / 2)), + y: rect.y + CHART_MARGIN.top, + title: `${formatIsoDateOnly(selection.start)} → ${formatIsoDateOnly(selection.end)}`, + text: ( + <> + {selectedBars.length ? ( + <> + {!shownDatasource && ( +
{`Totals for ${pluralIfNeeded(datasources.length, 'datasource')}`}
+ )} +
{`Size: ${formatIntervalStat('size', agg.size)}`}
+
{`Rows: ${formatIntervalStat('rows', agg.rows)}`}
+
{`Segments: ${formatIntervalStat('segments', agg.segments)}`}
+ + ) : ( +
No segments in this interval
+ )} + {selection.done && ( +
+
+ )} + + ), + }; + } + } + + function renderLoadRule(loadRule: Rule, i: number, isDefault: boolean) { + const [start, end] = loadRuleToDateRange(loadRule); + const { x, width } = startEndToXWidth({ start, end }); + const title = RuleUtil.ruleToString(loadRule) + (isDefault ? ' (cluster default)' : ''); + return ( +
+ {title} +
+ ); + } + + const nowX = timeScale(now); + return ( +
+ + + v !== 0)) + .tickSize(-innerStage.width) + .tickFormat(() => '') + .tickSizeOuter(0)} + /> + + + formatTickRate(e.valueOf()))} + /> + + {bubbleInfo && ( + + )} + {0 < nowX && nowX < innerStage.width && ( + + )} + {intervalBars.map((intervalBar, i) => { + return ( + + ); + })} + {bubbleInfo?.intervalBars.length === 1 && + bubbleInfo.intervalBars.map((intervalBar, i) => ( + + ))} + {selection && ( + + )} + {!!shiftOffset && ( + 0 ? timeScale(dateRange[1]) : 0} + y={0} + height={innerStage.height} + width={ + shiftOffset > 0 + ? innerStage.width - timeScale(dateRange[1]) + : timeScale(dateRange[0]) + } + /> + )} + + + + {(datasourceRules || datasourceRulesError) && ( +
+ {datasourceRules?.defaultLoadRules.map((rule, index) => + renderLoadRule(rule, index, true), + )} + {datasourceRules?.loadRules.map((rule, index) => renderLoadRule(rule, index, false))} + {datasourceRulesError && ( +
Rule loading error: {datasourceRulesError}
+ )} +
+ )} + {!intervalRows.length && ( +
+
There are no segments in the selected range
+
+ )} + setSelection(undefined) : undefined} + mute + direction="up" + /> +
+ ); +}; diff --git a/web-console/src/components/segment-timeline/bar-unit.spec.tsx b/web-console/src/components/segment-timeline/segment-bar-chart.scss similarity index 68% rename from web-console/src/components/segment-timeline/bar-unit.spec.tsx rename to web-console/src/components/segment-timeline/segment-bar-chart.scss index d5926dcf69f0..130997119e3e 100644 --- a/web-console/src/components/segment-timeline/bar-unit.spec.tsx +++ b/web-console/src/components/segment-timeline/segment-bar-chart.scss @@ -16,18 +16,6 @@ * limitations under the License. */ -import { render } from '@testing-library/react'; - -import { BarUnit } from './bar-unit'; - -describe('BarUnit', () => { - it('matches snapshot', () => { - const barGroup = ( - - - - ); - const { container } = render(barGroup); - expect(container.firstChild).toMatchSnapshot(); - }); -}); +.segment-bar-chart { + position: relative; +} diff --git a/web-console/src/components/segment-timeline/segment-bar-chart.tsx b/web-console/src/components/segment-timeline/segment-bar-chart.tsx new file mode 100644 index 000000000000..297ea4afe617 --- /dev/null +++ b/web-console/src/components/segment-timeline/segment-bar-chart.tsx @@ -0,0 +1,162 @@ +/* + * 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 { C, F, L, N, sql, SqlExpression, SqlQuery } from 'druid-query-toolkit'; +import { useMemo } from 'react'; + +import { END_OF_TIME_DATE, type Rule, RuleUtil, START_OF_TIME_DATE } from '../../druid-models'; +import type { Capabilities } from '../../helpers'; +import { useQueryManager } from '../../hooks'; +import { Api } from '../../singletons'; +import { Duration, filterMap, getApiArray, queryDruidSql, TZ_UTC } from '../../utils'; +import { Loader } from '../loader/loader'; + +import type { IntervalRow } from './common'; +import type { SegmentBarChartRenderProps } from './segment-bar-chart-render'; +import { SegmentBarChartRender } from './segment-bar-chart-render'; + +import './segment-bar-chart.scss'; + +export interface SegmentBarChartProps + extends Omit< + SegmentBarChartRenderProps, + 'intervalRows' | 'datasourceRules' | 'datasourceRulesError' + > { + capabilities: Capabilities; +} + +export const SegmentBarChart = function SegmentBarChart(props: SegmentBarChartProps) { + const { capabilities, dateRange, shownDatasource, ...otherProps } = props; + + const intervalsQuery = useMemo( + () => ({ capabilities, dateRange, shownDatasource: shownDatasource }), + [capabilities, dateRange, shownDatasource], + ); + + const [intervalRowsState] = useQueryManager({ + query: intervalsQuery, + processQuery: async ({ capabilities, dateRange, shownDatasource }, cancelToken) => { + if (capabilities.hasSql()) { + const query = SqlQuery.from(N('sys').table('segments')) + .changeWhereExpression( + SqlExpression.and( + sql`"start" <= '${dateRange[1].toISOString()}' AND '${dateRange[0].toISOString()}' < "end"`, + C('start').unequal(START_OF_TIME_DATE), + C('end').unequal(END_OF_TIME_DATE), + C('is_overshadowed').equal(0), + shownDatasource ? C('datasource').equal(L(shownDatasource)) : undefined, + ), + ) + .addSelect(C('start'), { addToGroupBy: 'end' }) + .addSelect(C('end'), { addToGroupBy: 'end' }) + .addSelect(C('datasource'), { addToGroupBy: 'end' }) + .addSelect(C('is_realtime').as('realtime'), { addToGroupBy: 'end' }) + .addSelect(F.count().as('segments')) + .addSelect(F.sum(C('size')).as('size')) + .addSelect(F.sum(C('num_rows')).as('rows')) + .toString(); + + return (await queryDruidSql({ query }, cancelToken)).map(sr => { + const start = new Date(sr.start); + const end = new Date(sr.end); + + return { + ...sr, + start, + end, + realtime: Boolean(sr.realtime), + originalTimeSpan: Duration.fromRange(start, end, TZ_UTC), + } as IntervalRow; + }); + } else { + return filterMap( + await getApiArray( + `/druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&includeRealtimeSegments&${ + shownDatasource ? `datasources=${Api.encodePath(shownDatasource)}` : '' + }`, + cancelToken, + ), + (segment: any) => { + if (segment.overshadowed) return; // We have to include overshadowed segments to get the realtime segments in this API + const [startStr, endStr] = segment.interval.split('/'); + if (startStr === START_OF_TIME_DATE && endStr === END_OF_TIME_DATE) return; + const start = new Date(startStr); + const end = new Date(endStr); + if (!(start <= dateRange[1] && dateRange[0] < end)) return; + + return { + start, + end, + datasource: segment.dataSource, + realtime: Boolean(segment.realtime), + originalTimeSpan: Duration.fromRange(start, end, TZ_UTC), + segments: 1, + size: segment.size, + rows: segment.num_rows || 0, // segment.num_rows is really null on this API :-( + } as IntervalRow; + }, + ); + } + }, + }); + + const [allLoadRulesState] = useQueryManager({ + query: shownDatasource ? '' : undefined, + processQuery: async (_, cancelToken) => { + return ( + await Api.instance.get>('/druid/coordinator/v1/rules', { + cancelToken, + }) + ).data; + }, + }); + + const datasourceRules = useMemo(() => { + const allLoadRules = allLoadRulesState.data; + if (!allLoadRules || !shownDatasource) return; + return { + loadRules: (allLoadRules[shownDatasource] || []).toReversed(), + defaultLoadRules: (allLoadRules[RuleUtil.DEFAULT_RULES_KEY] || []).toReversed(), + }; + }, [allLoadRulesState.data, shownDatasource]); + + if (intervalRowsState.error) { + return ( +
+ {`Error when loading data: ${intervalRowsState.getErrorMessage()}`} +
+ ); + } + + const intervalRows = intervalRowsState.getSomeData(); + return ( + <> + {intervalRows && ( + + )} + {intervalRowsState.loading && } + + ); +}; diff --git a/web-console/src/components/segment-timeline/segment-timeline.scss b/web-console/src/components/segment-timeline/segment-timeline.scss index aa437052d617..4224c1cab89d 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.scss +++ b/web-console/src/components/segment-timeline/segment-timeline.scss @@ -16,12 +16,20 @@ * limitations under the License. */ +@import '../../variables'; + .segment-timeline { - display: grid; - grid-template-columns: 1fr 220px; + .control-bar { + @include card-like; + height: 34px; + display: flex; + align-items: start; + padding: 5px; + gap: 10px; - .loader { - width: 85%; + & > .expander { + flex: 1; + } } .loading-error { @@ -31,14 +39,16 @@ transform: translate(-50%, -50%); } - .no-data-text { + .chart-container { position: absolute; - left: 30vw; - top: 15vh; - font-size: 20px; - } + top: 34px; + width: 100%; + bottom: 0; + overflow: hidden; - .side-control { - padding-top: 20px; + .segment-bar-chart, + .segment-bar-chart-render { + @include pin-full; + } } } diff --git a/web-console/src/components/segment-timeline/segment-timeline.spec.tsx b/web-console/src/components/segment-timeline/segment-timeline.spec.tsx index 4f95842801a4..ebc83ae143c7 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.spec.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.spec.tsx @@ -16,40 +16,42 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; import { render } from '@testing-library/react'; import { Capabilities } from '../../helpers'; +import { QueryState } from '../../utils'; import { SegmentTimeline } from './segment-timeline'; -jest.useFakeTimers('modern').setSystemTime(Date.parse('2021-06-08T12:34:56Z')); +jest.useFakeTimers('modern').setSystemTime(Date.parse('2024-11-19T12:34:56Z')); -describe('SegmentTimeline', () => { - it('.getSqlQuery', () => { - expect( - SegmentTimeline.getSqlQuery([ - new Date('2020-01-01T00:00:00Z'), - new Date('2021-02-01T00:00:00Z'), - ]), - ).toEqual(sane` - SELECT - "start", "end", "datasource", - COUNT(*) AS "count", - SUM("size") AS "size" - FROM sys.segments - WHERE - '2020-01-01T00:00:00.000Z' <= "start" AND - "end" <= '2021-02-01T00:00:00.000Z' AND - is_published = 1 AND - is_overshadowed = 0 - GROUP BY 1, 2, 3 - ORDER BY "start" DESC - `); - }); +jest.mock('../../hooks', () => { + return { + useQueryManager: (options: any) => { + if (options.initQuery instanceof Capabilities) { + // This is a query for data sources + return [new QueryState({ data: ['ds1', 'ds2'] })]; + } + + if (options.query === null) { + // This is a query for the data source time range + return [ + new QueryState({ + data: [new Date('2024-11-01 00:00:00Z'), new Date('2024-11-18 00:00:00Z')], + }), + ]; + } + return new QueryState({ error: new Error('not covered') }); + }, + }; +}); + +describe('SegmentTimeline', () => { it('matches snapshot', () => { - const segmentTimeline = ; + const segmentTimeline = ( + + ); const { container } = render(segmentTimeline); expect(container.firstChild).toMatchSnapshot(); }); diff --git a/web-console/src/components/segment-timeline/segment-timeline.tsx b/web-console/src/components/segment-timeline/segment-timeline.tsx index 8aee0c66d477..2774c73276eb 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.tsx @@ -16,628 +16,361 @@ * limitations under the License. */ -import { Button, FormGroup, MenuItem, ResizeSensor, SegmentedControl } from '@blueprintjs/core'; -import type { DateRange, NonNullDateRange } from '@blueprintjs/datetime'; -import { DateRangeInput3 } from '@blueprintjs/datetime2'; +import { + Button, + ButtonGroup, + Intent, + Menu, + MenuItem, + Popover, + Position, + ResizeSensor, +} from '@blueprintjs/core'; +import type { NonNullDateRange } from '@blueprintjs/datetime'; +import { DateRangePicker3 } from '@blueprintjs/datetime2'; import { IconNames } from '@blueprintjs/icons'; -import type { ItemPredicate, ItemRenderer } from '@blueprintjs/select'; import { Select } from '@blueprintjs/select'; -import type { AxisScale } from 'd3-axis'; -import { scaleLinear, scaleUtc } from 'd3-scale'; -import enUS from 'date-fns/locale/en-US'; -import React from 'react'; +import { C, L, N, SqlExpression, SqlQuery } from 'druid-query-toolkit'; +import { useEffect, useMemo, useState } from 'react'; +import { END_OF_TIME_DATE, START_OF_TIME_DATE } from '../../druid-models'; import type { Capabilities } from '../../helpers'; -import { Api } from '../../singletons'; +import { useQueryManager } from '../../hooks'; import { - ceilToUtcDay, - formatBytes, - formatInteger, + checkedCircleIcon, + day, + Duration, + getApiArray, isNonNullRange, localToUtcDateRange, + maxDate, queryDruidSql, - QueryManager, - uniq, + TZ_UTC, utcToLocalDateRange, } from '../../utils'; +import { Stage } from '../../utils/stage'; import { Loader } from '../loader/loader'; -import type { BarUnitData } from './stacked-bar-chart'; -import { StackedBarChart } from './stacked-bar-chart'; +import type { IntervalStat } from './common'; +import { formatIsoDateOnly, getIntervalStatTitle, INTERVAL_STATS } from './common'; +import type { SegmentBarChartProps } from './segment-bar-chart'; +import { SegmentBarChart } from './segment-bar-chart'; import './segment-timeline.scss'; -interface SegmentTimelineProps { - capabilities: Capabilities; -} - -type ActiveDataType = 'sizeData' | 'countData'; - -interface SegmentTimelineState { - chartHeight: number; - chartWidth: number; - data?: Record; - datasources: string[]; - stackedData?: Record; - singleDatasourceData?: Record>; - activeDatasource: string | null; - activeDataType: ActiveDataType; - dataToRender: BarUnitData[]; - loading: boolean; - error?: Error; - xScale: AxisScale | null; - yScale: AxisScale | null; - dateRange: NonNullDateRange; - selectedDateRange?: DateRange; +const DEFAULT_SHOWN_DURATION = new Duration('P1Y'); +const SHOWN_DURATION_OPTIONS: Duration[] = [ + new Duration('P1D'), + new Duration('P1W'), + new Duration('P1M'), + new Duration('P3M'), + new Duration('P1Y'), + new Duration('P5Y'), + new Duration('P10Y'), +]; + +function getDateRange(shownDuration: Duration): NonNullDateRange { + const end = day.ceil(new Date(), TZ_UTC); + return [shownDuration.shift(end, TZ_UTC, -1), end]; } -interface BarChartScales { - xScale: AxisScale; - yScale: AxisScale; +function formatDateRange(dateRange: NonNullDateRange): string { + return `${formatIsoDateOnly(dateRange[0])} → ${formatIsoDateOnly(dateRange[1])}`; } -interface IntervalRow { - start: string; - end: string; - datasource: string; - count: number; - size: number; +function dateRangesEqual(dr1: NonNullDateRange, dr2: NonNullDateRange): boolean { + return dr1[0].valueOf() === dr2[0].valueOf() && dr2[1].valueOf() === dr2[1].valueOf(); } -const DEFAULT_TIME_SPAN_MONTHS = 3; - -function getDefaultDateRange(): NonNullDateRange { - const start = ceilToUtcDay(new Date()); - const end = new Date(start.valueOf()); - start.setUTCMonth(start.getUTCMonth() - DEFAULT_TIME_SPAN_MONTHS); - return [start, end]; +interface SegmentTimelineProps extends Pick { + capabilities: Capabilities; + datasource: string | undefined; } -export class SegmentTimeline extends React.PureComponent< - SegmentTimelineProps, - SegmentTimelineState -> { - static COLORS = [ - '#b33040', - '#d25c4d', - '#f2b447', - '#d9d574', - '#4FAA7E', - '#57ceff', - '#789113', - '#098777', - '#b33040', - '#d2757b', - '#f29063', - '#d9a241', - '#80aa61', - '#c4ff9e', - '#915412', - '#87606c', - ]; - - static getColor(index: number): string { - return SegmentTimeline.COLORS[index % SegmentTimeline.COLORS.length]; - } - - static getSqlQuery(dateRange: NonNullDateRange): string { - return `SELECT - "start", "end", "datasource", - COUNT(*) AS "count", - SUM("size") AS "size" -FROM sys.segments -WHERE - '${dateRange[0].toISOString()}' <= "start" AND - "end" <= '${dateRange[1].toISOString()}' AND - is_published = 1 AND - is_overshadowed = 0 -GROUP BY 1, 2, 3 -ORDER BY "start" DESC`; - } - - static processRawData(data: IntervalRow[]) { - if (data === null) return []; +export const SegmentTimeline = function SegmentTimeline(props: SegmentTimelineProps) { + const { capabilities, datasource, ...otherProps } = props; + const [stage, setStage] = useState(); + const [activeSegmentStat, setActiveSegmentStat] = useState('size'); + const [shownDatasource, setShownDatasource] = useState(datasource); + const [dateRange, setDateRange] = useState(); + const [showCustomDatePicker, setShowCustomDatePicker] = useState(false); + + useEffect(() => { + setShownDatasource(datasource); + }, [datasource]); + + const defaultDateRange = useMemo(() => { + return getDateRange(DEFAULT_SHOWN_DURATION); + }, []); + + const [datasourcesState] = useQueryManager({ + initQuery: capabilities, + processQuery: async (capabilities, cancelToken) => { + if (capabilities.hasSql()) { + const tables = await queryDruidSql<{ TABLE_NAME: string }>( + { + query: `SELECT TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE = 'TABLE'`, + }, + cancelToken, + ); - const countData: Record = {}; - const sizeData: Record = {}; - data.forEach(entry => { - const start = entry.start; - const day = start.split('T')[0]; - const datasource = entry.datasource; - const count = entry.count; - const segmentSize = entry.size; - if (countData[day] === undefined) { - countData[day] = { - day, - [datasource]: count, - total: count, - }; - sizeData[day] = { - day, - [datasource]: segmentSize, - total: segmentSize, - }; + return tables.map(d => d.TABLE_NAME); } else { - const countDataEntry: number | undefined = countData[day][datasource]; - countData[day][datasource] = count + (countDataEntry === undefined ? 0 : countDataEntry); - const sizeDataEntry: number | undefined = sizeData[day][datasource]; - sizeData[day][datasource] = segmentSize + (sizeDataEntry === undefined ? 0 : sizeDataEntry); - countData[day].total += count; - sizeData[day].total += segmentSize; + return await getApiArray(`/druid/coordinator/v1/datasources`, cancelToken); } - }); - - const countDataArray = Object.keys(countData) - .reverse() - .map((time: any) => { - return countData[time]; - }); - - const sizeDataArray = Object.keys(sizeData) - .reverse() - .map((time: any) => { - return sizeData[time]; - }); - - return { countData: countDataArray, sizeData: sizeDataArray }; - } - - static calculateStackedData( - data: Record, - datasources: string[], - ): Record { - const newStackedData: Record = {}; - Object.keys(data).forEach((type: any) => { - const stackedData: any = data[type].map((d: any) => { - let y0 = 0; - return datasources.map((datasource: string, i) => { - const barUnitData = { - x: d.day, - y: d[datasource] === undefined ? 0 : d[datasource], - y0, - datasource, - color: SegmentTimeline.getColor(i), - dailySize: d.total, - }; - y0 += d[datasource] === undefined ? 0 : d[datasource]; - return barUnitData; - }); - }); - newStackedData[type] = stackedData.flat(); - }); + }, + }); + + const [initDatasourceDateRangeState] = useQueryManager({ + query: dateRange ? undefined : shownDatasource ?? null, + processQuery: async (datasource, cancelToken) => { + let queriedStart: Date; + let queriedEnd: Date; + if (capabilities.hasSql()) { + const baseQuery = SqlQuery.from(N('sys').table('segments')) + .changeWhereExpression( + SqlExpression.and( + C('start').unequal(START_OF_TIME_DATE), + C('end').unequal(END_OF_TIME_DATE), + C('is_overshadowed').equal(0), + datasource ? C('datasource').equal(L(datasource)) : undefined, + ), + ) + .changeLimitValue(1); - return newStackedData; - } + const endQuery = baseQuery + .addSelect(C('end'), { addToOrderBy: 'end', direction: 'DESC' }) + .toString(); - static calculateSingleDatasourceData( - data: Record, - datasources: string[], - ): Record> { - const singleDatasourceData: Record> = {}; - Object.keys(data).forEach(dataType => { - singleDatasourceData[dataType] = {}; - datasources.forEach((datasource, i) => { - const currentData = data[dataType]; - if (currentData.length === 0) return; - const dataResult = currentData.map((d: any) => { - let y = 0; - if (d[datasource] !== undefined) { - y = d[datasource]; - } - return { - x: d.day, - y, - datasource, - color: SegmentTimeline.getColor(i), - dailySize: d.total, - }; - }); - if (!dataResult.every((d: any) => d.y === 0)) { - singleDatasourceData[dataType][datasource] = dataResult; + const endRes = await queryDruidSql<{ end: string }>({ query: endQuery }, cancelToken).catch( + () => [], + ); + if (endRes.length !== 1) { + return getDateRange(DEFAULT_SHOWN_DURATION); } - }); - }); - return singleDatasourceData; - } - - private readonly dataQueryManager: QueryManager< - { capabilities: Capabilities; dateRange: NonNullDateRange }, - any - >; - - private readonly chartMargin = { top: 40, right: 15, bottom: 20, left: 60 }; - - constructor(props: SegmentTimelineProps) { - super(props); - const dateRange = getDefaultDateRange(); - - this.state = { - chartWidth: 1, // Dummy init values to be replaced - chartHeight: 1, // after first render - data: {}, - datasources: [], - stackedData: {}, - singleDatasourceData: {}, - dataToRender: [], - activeDatasource: null, - activeDataType: 'sizeData', - loading: true, - xScale: null, - yScale: null, - dateRange, - }; - - this.dataQueryManager = new QueryManager({ - processQuery: async ({ capabilities, dateRange }, cancelToken) => { - let intervals: IntervalRow[]; - let datasources: string[]; - if (capabilities.hasSql()) { - intervals = await queryDruidSql( - { - query: SegmentTimeline.getSqlQuery(dateRange), - }, - cancelToken, - ); - datasources = uniq(intervals.map(r => r.datasource).sort()); - } else if (capabilities.hasCoordinatorAccess()) { - const startIso = dateRange[0].toISOString(); + queriedEnd = day.ceil(new Date(endRes[0].end), TZ_UTC); - datasources = ( - await Api.instance.get(`/druid/coordinator/v1/datasources`, { cancelToken }) - ).data; - intervals = ( - await Promise.all( - datasources.map(async datasource => { - const intervalMap = ( - await Api.instance.get( - `/druid/coordinator/v1/datasources/${Api.encodePath( - datasource, - )}/intervals?simple`, - { cancelToken }, - ) - ).data; + const startQuery = baseQuery + .addSelect(C('start'), { addToOrderBy: 'end', direction: 'ASC' }) + .toString(); - return Object.keys(intervalMap) - .map(interval => { - const [start, end] = interval.split('/'); - const { count, size } = intervalMap[interval]; - return { - start, - end, - datasource, - count, - size, - }; - }) - .filter(a => startIso < a.start); - }), - ) - ) - .flat() - .sort((a, b) => b.start.localeCompare(a.start)); - } else { - throw new Error(`must have SQL or coordinator access`); + const startRes = await queryDruidSql<{ start: string }>( + { query: startQuery }, + cancelToken, + ).catch(() => []); + if (startRes.length !== 1) { + return [DEFAULT_SHOWN_DURATION.shift(queriedEnd, TZ_UTC, -1), queriedEnd]; // Should not really get here } - const data = SegmentTimeline.processRawData(intervals); - const stackedData = SegmentTimeline.calculateStackedData(data, datasources); - const singleDatasourceData = SegmentTimeline.calculateSingleDatasourceData( - data, - datasources, - ); - return { data, datasources, stackedData, singleDatasourceData }; - }, - onStateChange: ({ data, loading, error }) => { - this.setState({ - data: data ? data.data : undefined, - datasources: data ? data.datasources : [], - stackedData: data ? data.stackedData : undefined, - singleDatasourceData: data ? data.singleDatasourceData : undefined, - loading, - error, - }); - }, - }); - } - - componentDidMount(): void { - const { capabilities } = this.props; - const { dateRange } = this.state; - - if (isNonNullRange(dateRange)) { - this.dataQueryManager.runQuery({ capabilities, dateRange }); - } - } - - componentWillUnmount(): void { - this.dataQueryManager.terminate(); - } - - componentDidUpdate(_prevProps: SegmentTimelineProps, prevState: SegmentTimelineState): void { - const { activeDatasource, activeDataType, singleDatasourceData, stackedData } = this.state; - if ( - prevState.data !== this.state.data || - prevState.activeDataType !== this.state.activeDataType || - prevState.activeDatasource !== this.state.activeDatasource || - prevState.chartWidth !== this.state.chartWidth || - prevState.chartHeight !== this.state.chartHeight - ) { - const scales: BarChartScales | undefined = this.calculateScales(); - const dataToRender: BarUnitData[] | undefined = activeDatasource - ? singleDatasourceData - ? singleDatasourceData[activeDataType][activeDatasource] - : undefined - : stackedData - ? stackedData[activeDataType] - : undefined; - - if (scales && dataToRender) { - this.setState({ - dataToRender, - xScale: scales.xScale, - yScale: scales.yScale, - }); + queriedStart = day.floor(new Date(startRes[0].start), TZ_UTC); + } else { + // Don't bother querying if there is no SQL + return getDateRange(DEFAULT_SHOWN_DURATION); } - } - } - - private calculateScales(): BarChartScales | undefined { - const { - chartWidth, - chartHeight, - data, - activeDataType, - activeDatasource, - singleDatasourceData, - dateRange, - } = this.state; - if (!data || !Object.keys(data).length || !isNonNullRange(dateRange)) return; - const activeData = data[activeDataType]; - let yDomain: number[] = [ - 0, - activeData.length === 0 - ? 0 - : activeData.reduce((max: any, d: any) => (max.total > d.total ? max : d)).total, - ]; - - if ( - activeDatasource !== null && - singleDatasourceData![activeDataType][activeDatasource] !== undefined - ) { - yDomain = [ - 0, - singleDatasourceData![activeDataType][activeDatasource].reduce((max: any, d: any) => - max.y > d.y ? max : d, - ).y, + return [ + maxDate(queriedStart, DEFAULT_SHOWN_DURATION.shift(queriedEnd, TZ_UTC, -1)), + queriedEnd, ]; - } - - const xScale: AxisScale = scaleUtc() - .domain(dateRange) - .range([0, chartWidth - this.chartMargin.left - this.chartMargin.right]); - - const yScale: AxisScale = scaleLinear() - .rangeRound([chartHeight - this.chartMargin.top - this.chartMargin.bottom, 0]) - .domain(yDomain); - - return { - xScale, - yScale, - }; + }, + }); + + const effectiveDateRange = + dateRange || + initDatasourceDateRangeState.data || + (initDatasourceDateRangeState.isLoading() ? undefined : defaultDateRange); + + let previousDateRange: NonNullDateRange | undefined; + let zoomedOutDateRange: NonNullDateRange | undefined; + let nextDateRange: NonNullDateRange | undefined; + if (effectiveDateRange) { + const d = Duration.fromRange(effectiveDateRange[0], effectiveDateRange[1], TZ_UTC); + const shiftStartBack = d.shift(effectiveDateRange[0], TZ_UTC, -1); + const shiftEndForward = d.shift(effectiveDateRange[1], TZ_UTC); + const now = day.ceil(new Date(), TZ_UTC); + previousDateRange = [shiftStartBack, effectiveDateRange[0]]; + zoomedOutDateRange = [shiftStartBack, shiftEndForward < now ? shiftEndForward : now]; + nextDateRange = [effectiveDateRange[1], shiftEndForward]; } - private readonly formatTick = (n: number) => { - if (isNaN(n)) return ''; - const { activeDataType } = this.state; - if (activeDataType === 'countData') { - return formatInteger(n); - } else { - return formatBytes(n); - } - }; - - private readonly handleResize = (entries: ResizeObserverEntry[]) => { - const chartRect = entries[0].contentRect; - this.setState({ - chartWidth: chartRect.width, - chartHeight: chartRect.height, - }); - }; - - renderStackedBarChart() { - const { - chartWidth, - chartHeight, - loading, - dataToRender, - activeDataType, - error, - xScale, - yScale, - data, - activeDatasource, - dateRange, - } = this.state; - - if (loading) { - return ( -
- -
- ); - } - - if (error) { - return ( -
- Error when loading data: {error.message} -
- ); - } - - if (xScale === null || yScale === null) { - return ( -
- Error when calculating scales -
- ); - } - - if (data![activeDataType].length === 0) { - return ( -
- There are no segments for the selected interval -
- ); - } - - if ( - activeDatasource !== null && - data![activeDataType].every((d: any) => d[activeDatasource] === undefined) - ) { - return ( -
- - No data available for {activeDatasource} - -
- ); - } - - const millisecondsPerDay = 24 * 60 * 60 * 1000; - const barCounts = (dateRange[1].getTime() - dateRange[0].getTime()) / millisecondsPerDay; - const barWidth = Math.max( - 0, - (chartWidth - this.chartMargin.left - this.chartMargin.right) / barCounts, - ); - return ( - - - this.setState(prevState => ({ - activeDatasource: prevState.activeDatasource ? null : datasource, - })) + return ( +
+
+ + + items={datasourcesState.data || []} + disabled={datasourcesState.isError()} + onItemSelect={setShownDatasource} + itemRenderer={(val, { handleClick, handleFocus, modifiers }) => { + if (!modifiers.matchesPredicate) return null; + return ( + + ); + }} + noResults={} + itemPredicate={(query, val, _index, exactMatch) => { + const normalizedTitle = val.toLowerCase(); + const normalizedQuery = query.toLowerCase(); + + if (exactMatch) { + return normalizedTitle === normalizedQuery; + } else { + return normalizedTitle.includes(normalizedQuery); + } + }} + > +
+ { + const rect = entries[0].contentRect; + setStage(new Stage(rect.width, rect.height)); + }} + > +
+ {stage && effectiveDateRange && ( + - - - - + )} + {initDatasourceDateRangeState.isLoading() && }
-
- ); - } -} +
+
+ ); +}; diff --git a/web-console/src/components/segment-timeline/stacked-bar-chart.tsx b/web-console/src/components/segment-timeline/stacked-bar-chart.tsx deleted file mode 100644 index 8018aaee5f6f..000000000000 --- a/web-console/src/components/segment-timeline/stacked-bar-chart.tsx +++ /dev/null @@ -1,174 +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 { AxisScale } from 'd3-axis'; -import { axisBottom, axisLeft } from 'd3-axis'; -import React, { useState } from 'react'; - -import { BarGroup } from './bar-group'; -import { ChartAxis } from './chart-axis'; - -import './stacked-bar-chart.scss'; - -export interface BarUnitData { - x: number; - y: number; - y0?: number; - width: number; - datasource: string; - color: string; - dailySize?: number; -} - -export interface BarChartMargin { - top: number; - right: number; - bottom: number; - left: number; -} - -export interface HoveredBarInfo { - xCoordinate?: number; - yCoordinate?: number; - height?: number; - width?: number; - datasource?: string; - xValue?: number; - yValue?: number; - dailySize?: number; -} - -interface StackedBarChartProps { - svgWidth: number; - svgHeight: number; - margin: BarChartMargin; - activeDataType?: string; - dataToRender: BarUnitData[]; - changeActiveDatasource: (e: string | null) => void; - formatTick: (e: number) => string; - xScale: AxisScale; - yScale: AxisScale; - barWidth: number; -} - -export const StackedBarChart = React.forwardRef(function StackedBarChart( - props: StackedBarChartProps, - ref, -) { - const { - activeDataType, - svgWidth, - svgHeight, - margin, - formatTick, - xScale, - yScale, - dataToRender, - changeActiveDatasource, - barWidth, - } = props; - const [hoverOn, setHoverOn] = useState(); - - const width = svgWidth - margin.left - margin.right; - const height = svgHeight - margin.top - margin.bottom; - - function renderBarChart() { - return ( - - setHoverOn(undefined)} - > - '') - .tickSizeOuter(0)} - /> - setHoverOn(e)} - hoverOn={hoverOn} - barWidth={barWidth} - /> - - formatTick(e))} - /> - {hoverOn && ( - { - setHoverOn(undefined); - changeActiveDatasource(hoverOn.datasource ?? null); - }} - > - - - )} - - - ); - } - - return ( -
- {hoverOn && ( -
-
Datasource: {hoverOn.datasource}
-
Time: {hoverOn.xValue}
-
- {`${ - activeDataType === 'countData' ? 'Daily total count:' : 'Daily total size:' - } ${formatTick(hoverOn.dailySize!)}`} -
-
- {`${activeDataType === 'countData' ? 'Count:' : 'Size:'} ${formatTick( - hoverOn.yValue!, - )}`} -
-
- )} - {renderBarChart()} -
- ); -}); diff --git a/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx b/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx index 74cb55682f30..a7fa38fd445c 100644 --- a/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx +++ b/web-console/src/components/supervisor-history-panel/supervisor-history-panel.tsx @@ -25,7 +25,7 @@ import type { IngestionSpec } from '../../druid-models'; import { cleanSpec } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { Api } from '../../singletons'; -import { deepSet } from '../../utils'; +import { deepSet, getApiArray } from '../../utils'; import { Loader } from '../loader/loader'; import { ShowValue } from '../show-value/show-value'; @@ -49,11 +49,12 @@ export const SupervisorHistoryPanel = React.memo(function SupervisorHistoryPanel const [historyState] = useQueryManager({ initQuery: supervisorId, processQuery: async (supervisorId, cancelToken) => { - const resp = await Api.instance.get( - `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/history`, - { cancelToken }, - ); - return resp.data.map((vs: SupervisorHistoryEntry) => deepSet(vs, 'spec', cleanSpec(vs.spec))); + return ( + await getApiArray( + `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/history`, + cancelToken, + ) + ).map(vs => deepSet(vs, 'spec', cleanSpec(vs.spec))); }, }); diff --git a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx index aec706a50fc0..38c69e297e0c 100644 --- a/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx +++ b/web-console/src/components/table-clickable-cell/table-clickable-cell.tsx @@ -28,7 +28,7 @@ export interface TableClickableCellProps { className?: string; onClick: MouseEventHandler; hoverIcon?: IconName; - title?: string; + tooltip?: string; disabled?: boolean; children?: ReactNode; } @@ -36,12 +36,13 @@ export interface TableClickableCellProps { export const TableClickableCell = React.memo(function TableClickableCell( props: TableClickableCellProps, ) { - const { className, onClick, hoverIcon, disabled, children, ...rest } = props; + const { className, onClick, hoverIcon, disabled, children, tooltip, ...rest } = props; return (
{children} diff --git a/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx b/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx index 631fa224aaf4..c03f0038cc7f 100644 --- a/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx +++ b/web-console/src/components/table-filterable-cell/table-filterable-cell.tsx @@ -27,7 +27,7 @@ import { Deferred } from '../deferred/deferred'; import './table-filterable-cell.scss'; -const FILTER_MODES: FilterMode[] = ['=', '!=', '<=', '>=']; +const FILTER_MODES: FilterMode[] = ['=', '!=', '<', '>=']; const FILTER_MODES_NO_COMPARISONS: FilterMode[] = ['=', '!=']; export interface TableFilterableCellProps { diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 95336713dbb6..5dfbba77101a 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -182,12 +182,24 @@ export class ConsoleApplication extends React.PureComponent< changeTabWithFilter('datasources', [{ id: 'datasource', value: `=${datasource}` }]); }; - private readonly goToSegments = (datasource: string, onlyUnavailable = false) => { + private readonly goToSegments = ({ + start, + end, + datasource, + realtime, + }: { + start?: Date; + end?: Date; + datasource?: string; + realtime?: boolean; + }) => { changeTabWithFilter( 'segments', compact([ - { id: 'datasource', value: `=${datasource}` }, - onlyUnavailable ? { id: 'is_available', value: '=false' } : undefined, + start && { id: 'start', value: `>=${start.toISOString()}` }, + end && { id: 'end', value: `<${end.toISOString()}` }, + datasource && { id: 'datasource', value: `=${datasource}` }, + typeof realtime === 'boolean' ? { id: 'is_realtime', value: `=${realtime}` } : undefined, ]), ); }; diff --git a/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx b/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx index 9e19e043c713..8ee0ffdbfbc3 100644 --- a/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx +++ b/web-console/src/dialogs/compaction-history-dialog/compaction-history-dialog.tsx @@ -24,7 +24,7 @@ import { Loader, ShowValue } from '../../components'; import type { CompactionConfig } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { Api } from '../../singletons'; -import { formatInteger, formatPercent } from '../../utils'; +import { formatInteger, formatPercent, getApiArray } from '../../utils'; import { DiffDialog } from '../diff-dialog/diff-dialog'; import './compaction-history-dialog.scss'; @@ -65,11 +65,10 @@ export const CompactionHistoryDialog = React.memo(function CompactionHistoryDial initQuery: datasource, processQuery: async (datasource, cancelToken) => { try { - const resp = await Api.instance.get( + return await getApiArray( `/druid/coordinator/v1/config/compaction/${Api.encodePath(datasource)}/history?count=20`, - { cancelToken }, + cancelToken, ); - return resp.data; } catch (e) { if (e.response?.status === 404) return []; throw e; diff --git a/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx b/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx index ec964f5507ee..ab4fed8ae1ad 100644 --- a/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/coordinator-dynamic-config-dialog/coordinator-dynamic-config-dialog.tsx @@ -27,7 +27,7 @@ import { COORDINATOR_DYNAMIC_CONFIG_FIELDS } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { getLink } from '../../links'; import { Api, AppToaster } from '../../singletons'; -import { getDruidErrorMessage } from '../../utils'; +import { getApiArray, getDruidErrorMessage } from '../../utils'; import { SnitchDialog } from '..'; import './coordinator-dynamic-config-dialog.scss'; @@ -47,10 +47,7 @@ export const CoordinatorDynamicConfigDialog = React.memo(function CoordinatorDyn const [historyRecordsState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - const historyResp = await Api.instance.get(`/druid/coordinator/v1/config/history?count=100`, { - cancelToken, - }); - return historyResp.data; + return await getApiArray(`/druid/coordinator/v1/config/history?count=100`, cancelToken); }, }); diff --git a/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx b/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx index 82187c866f44..3dfb2f7879d7 100644 --- a/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx +++ b/web-console/src/dialogs/datasource-table-action-dialog/datasource-columns-table/datasource-columns-table.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { L } from '@druid-toolkit/query'; +import { L } from 'druid-query-toolkit'; import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx b/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx index 1ca92787c1cb..c48cbfa51989 100644 --- a/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx +++ b/web-console/src/dialogs/datasource-table-action-dialog/datasource-preview-pane/datasource-preview-pane.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; -import { QueryRunner, T } from '@druid-toolkit/query'; +import type { QueryResult } from 'druid-query-toolkit'; +import { QueryRunner, T } from 'druid-query-toolkit'; import React from 'react'; import { Loader, RecordTablePane } from '../../../components'; diff --git a/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx b/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx index c36445d17731..122cc4566b85 100644 --- a/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx +++ b/web-console/src/dialogs/lookup-table-action-dialog/lookup-values-table/lookup-values-table.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { N } from '@druid-toolkit/query'; +import { N } from 'druid-query-toolkit'; import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx b/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx index ba30118b0a32..5b1233c6384e 100644 --- a/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/overlord-dynamic-config-dialog/overlord-dynamic-config-dialog.tsx @@ -27,7 +27,7 @@ import { OVERLORD_DYNAMIC_CONFIG_FIELDS } from '../../druid-models'; import { useQueryManager } from '../../hooks'; import { getLink } from '../../links'; import { Api, AppToaster } from '../../singletons'; -import { getDruidErrorMessage } from '../../utils'; +import { getApiArray, getDruidErrorMessage } from '../../utils'; import { SnitchDialog } from '..'; import './overlord-dynamic-config-dialog.scss'; @@ -47,10 +47,7 @@ export const OverlordDynamicConfigDialog = React.memo(function OverlordDynamicCo const [historyRecordsState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - const historyResp = await Api.instance.get(`/druid/indexer/v1/worker/history?count=100`, { - cancelToken, - }); - return historyResp.data; + return await getApiArray(`/druid/indexer/v1/worker/history?count=100`, cancelToken); }, }); diff --git a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap index 19c90c7de335..8bba057fac2d 100644 --- a/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap +++ b/web-console/src/dialogs/retention-dialog/__snapshots__/retention-dialog.spec.tsx.snap @@ -188,7 +188,7 @@ exports[`RetentionDialog matches snapshot 1`] = ` style="transform: translateY(0); transition: none;" >
d.tier); } else if (capabilities.hasCoordinatorAccess()) { - const allServiceResp = await Api.instance.get('/druid/coordinator/v1/servers?simple', { - cancelToken, - }); - return filterMap(allServiceResp.data, (s: any) => - s.type === 'historical' ? s.tier : undefined, + return filterMap( + await getApiArray('/druid/coordinator/v1/servers?simple', cancelToken), + (s: any) => (s.type === 'historical' ? s.tier : undefined), ); } else { throw new Error(`must have sql or coordinator access`); @@ -84,11 +82,10 @@ ORDER BY 1`, const [historyQueryState] = useQueryManager({ initQuery: props.datasource, processQuery: async (datasource, cancelToken) => { - const historyResp = await Api.instance.get( + return await getApiArray( `/druid/coordinator/v1/rules/${Api.encodePath(datasource)}/history?count=200`, - { cancelToken }, + cancelToken, ); - return historyResp.data; }, }); diff --git a/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx b/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx index 4c2487ce0cc6..2925482d9901 100644 --- a/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx +++ b/web-console/src/dialogs/segments-table-action-dialog/segments-preview-pane/segments-preview-pane.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; -import { QueryRunner } from '@druid-toolkit/query'; +import type { QueryResult } from 'druid-query-toolkit'; +import { QueryRunner } from 'druid-query-toolkit'; import React from 'react'; import { Loader, RecordTablePane } from '../../../components'; diff --git a/web-console/src/dialogs/status-dialog/status-dialog.tsx b/web-console/src/dialogs/status-dialog/status-dialog.tsx index 311f3e056646..672fc40910f3 100644 --- a/web-console/src/dialogs/status-dialog/status-dialog.tsx +++ b/web-console/src/dialogs/status-dialog/status-dialog.tsx @@ -50,8 +50,7 @@ export const StatusDialog = React.memo(function StatusDialog(props: StatusDialog const [responseState] = useQueryManager({ initQuery: null, processQuery: async (_, cancelToken) => { - const resp = await Api.instance.get(`/status`, { cancelToken }); - return resp.data; + return (await Api.instance.get(`/status`, { cancelToken })).data; }, }); diff --git a/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx b/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx index 009d83260604..d10fc00eb5ce 100644 --- a/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx +++ b/web-console/src/dialogs/supervisor-reset-offsets-dialog/supervisor-reset-offsets-dialog.tsx @@ -106,11 +106,12 @@ export const SupervisorResetOffsetsDialog = React.memo(function SupervisorResetO const [statusResp] = useQueryManager({ initQuery: supervisorId, processQuery: async (supervisorId, cancelToken) => { - const statusResp = await Api.instance.get( - `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/status`, - { cancelToken }, - ); - return statusResp.data; + return ( + await Api.instance.get( + `/druid/indexer/v1/supervisor/${Api.encodePath(supervisorId)}/status`, + { cancelToken }, + ) + ).data; }, }); diff --git a/web-console/src/components/segment-timeline/stacked-bar-chart.scss b/web-console/src/druid-models/datasource/datasource.ts similarity index 64% rename from web-console/src/components/segment-timeline/stacked-bar-chart.scss rename to web-console/src/druid-models/datasource/datasource.ts index 26e5f5186b5f..313d711b35c2 100644 --- a/web-console/src/components/segment-timeline/stacked-bar-chart.scss +++ b/web-console/src/druid-models/datasource/datasource.ts @@ -16,35 +16,32 @@ * limitations under the License. */ -.stacked-bar-chart { - position: relative; - overflow: hidden; +import { hashJoaat } from '../../utils'; - .bar-chart-tooltip { - position: absolute; - left: 100px; - right: 0; +const COLORS = [ + '#1f77b4', + '#aec7e8', + '#ff7f0e', + '#ffbb78', + '#2ca02c', + '#98df8a', + '#d62728', + '#ff9896', + '#9467bd', + '#c5b0d5', + '#8c564b', + '#c49c94', + '#e377c2', + '#f7b6d2', + '#7f7f7f', + '#c7c7c7', + '#bcbd22', + '#dbdb8d', + '#17becf', + '#9edae5', +]; - div { - display: inline-block; - width: 230px; - } - } - - svg { - position: absolute; - - .hovered-bar { - fill: transparent; - stroke: #ffffff; - stroke-width: 1.5px; - } - - .gridline-x { - line { - stroke-dasharray: 5, 5; - opacity: 0.5; - } - } - } +export function getDatasourceColor(datasource: string) { + const hash = hashJoaat(datasource); + return COLORS[hash % COLORS.length]; } diff --git a/web-console/src/druid-models/execution/execution.ts b/web-console/src/druid-models/execution/execution.ts index f4a73eb97207..1237b49e62da 100644 --- a/web-console/src/druid-models/execution/execution.ts +++ b/web-console/src/druid-models/execution/execution.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { Column, QueryResult, SqlExpression, SqlQuery, SqlWithQuery } from '@druid-toolkit/query'; +import { Column, QueryResult, SqlExpression, SqlQuery, SqlWithQuery } from 'druid-query-toolkit'; import { maybeGetClusterCapacity } from '../../helpers'; import { diff --git a/web-console/src/druid-models/external-config/external-config.ts b/web-console/src/druid-models/external-config/external-config.ts index d6541747ce5a..6fa74eb038bd 100644 --- a/web-console/src/druid-models/external-config/external-config.ts +++ b/web-console/src/druid-models/external-config/external-config.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { SqlQuery } from '@druid-toolkit/query'; +import type { SqlQuery } from 'druid-query-toolkit'; import { C, F, @@ -28,7 +28,7 @@ import { SqlLiteral, SqlStar, SqlType, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import { nonEmptyArray } from '../../utils'; diff --git a/web-console/src/druid-models/index.ts b/web-console/src/druid-models/index.ts index dfeeeeaac837..e31eedeea738 100644 --- a/web-console/src/druid-models/index.ts +++ b/web-console/src/druid-models/index.ts @@ -21,6 +21,7 @@ export * from './compaction-config/compaction-config'; export * from './compaction-status/compaction-status'; export * from './coordinator-dynamic-config/coordinator-dynamic-config'; export * from './dart/dart-query-entry'; +export * from './datasource/datasource'; export * from './dimension-spec/dimension-spec'; export * from './druid-engine/druid-engine'; export * from './execution/execution'; @@ -32,10 +33,12 @@ export * from './ingest-query-pattern/ingest-query-pattern'; export * from './ingestion-spec/ingestion-spec'; export * from './input-format/input-format'; export * from './input-source/input-source'; +export * from './load-rule/load-rule'; export * from './lookup-spec/lookup-spec'; export * from './metric-spec/metric-spec'; export * from './overlord-dynamic-config/overlord-dynamic-config'; export * from './query-context/query-context'; +export * from './segment/segment'; export * from './stages/stages'; export * from './supervisor-status/supervisor-status'; export * from './task/task'; diff --git a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts index 996385fc08ff..cf108c2b3890 100644 --- a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts +++ b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane, SqlQuery } from '@druid-toolkit/query'; +import { sane, SqlQuery } from 'druid-query-toolkit'; import { fitIngestQueryPattern, ingestQueryPatternToQuery } from './ingest-query-pattern'; diff --git a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts index f0ce8a0eb288..d2009ad2e5ee 100644 --- a/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts +++ b/web-console/src/druid-models/ingest-query-pattern/ingest-query-pattern.ts @@ -26,7 +26,7 @@ import { SqlTable, SqlWithPart, T, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import { filterMap, oneOf } from '../../utils'; import type { ExternalConfig } from '../external-config/external-config'; diff --git a/web-console/src/utils/load-rule.ts b/web-console/src/druid-models/load-rule/load-rule.ts similarity index 97% rename from web-console/src/utils/load-rule.ts rename to web-console/src/druid-models/load-rule/load-rule.ts index a32422bbb6a4..31b4d600585a 100644 --- a/web-console/src/utils/load-rule.ts +++ b/web-console/src/druid-models/load-rule/load-rule.ts @@ -18,7 +18,7 @@ import { sum } from 'd3-array'; -import { deepMove, deepSet } from './object-change'; +import { deepMove, deepSet } from '../../utils'; export type RuleType = | 'loadForever' @@ -41,6 +41,7 @@ export interface Rule { } export class RuleUtil { + static DEFAULT_RULES_KEY = '_default'; static TYPES: RuleType[] = [ 'loadForever', 'loadByInterval', diff --git a/web-console/src/druid-models/segment/segment.ts b/web-console/src/druid-models/segment/segment.ts new file mode 100644 index 000000000000..1bff0711fcdf --- /dev/null +++ b/web-console/src/druid-models/segment/segment.ts @@ -0,0 +1,50 @@ +/* + * 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 { Duration } from '../../utils'; + +export const START_OF_TIME_DATE = '-146136543-09-08T08:23:32.096Z'; +export const END_OF_TIME_DATE = '146140482-04-24T15:36:27.903Z'; + +export function computeSegmentTimeSpan(start: string, end: string): string { + if (start === START_OF_TIME_DATE && end === END_OF_TIME_DATE) { + return 'All'; + } + + const startDate = new Date(start); + if (isNaN(startDate.valueOf())) { + return 'Invalid start'; + } + + const endDate = new Date(end); + if (isNaN(endDate.valueOf())) { + return 'Invalid end'; + } + + return Duration.fromRange(startDate, endDate, 'Etc/UTC').getDescription(true); +} + +export interface ShardSpec { + type: string; + partitionNum?: number; + partitions?: number; + dimensions?: string[]; + partitionDimensions?: string[]; + start?: string[]; + end?: string[]; +} diff --git a/web-console/src/druid-models/task/task.ts b/web-console/src/druid-models/task/task.ts index 3445cc5fb45f..e1743c116897 100644 --- a/web-console/src/druid-models/task/task.ts +++ b/web-console/src/druid-models/task/task.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { C } from '@druid-toolkit/query'; +import { C } from 'druid-query-toolkit'; import type { Counters, StageDefinition } from '../stages/stages'; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts index 02bb3e399b02..8456b0d065d5 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.spec.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; +import { sane } from 'druid-query-toolkit'; import { WorkbenchQuery } from './workbench-query'; diff --git a/web-console/src/druid-models/workbench-query/workbench-query.ts b/web-console/src/druid-models/workbench-query/workbench-query.ts index da37127f1d2d..9b1ed42b516b 100644 --- a/web-console/src/druid-models/workbench-query/workbench-query.ts +++ b/web-console/src/druid-models/workbench-query/workbench-query.ts @@ -22,7 +22,7 @@ import type { SqlClusteredByClause, SqlExpression, SqlPartitionedByClause, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import { C, F, @@ -30,7 +30,7 @@ import { SqlOrderByClause, SqlOrderByExpression, SqlQuery, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import Hjson from 'hjson'; import * as JSONBig from 'json-bigint-native'; import { v4 as uuidv4 } from 'uuid'; diff --git a/web-console/src/entry.scss b/web-console/src/entry.scss index 8d368a4146b7..46ea9d298d0f 100644 --- a/web-console/src/entry.scss +++ b/web-console/src/entry.scss @@ -56,6 +56,7 @@ body { position: absolute; height: 100%; width: 100%; + z-index: 0; .console-application { position: absolute; diff --git a/web-console/src/entry.tsx b/web-console/src/entry.tsx index 42cfe08ad13c..55f7abfc59a2 100644 --- a/web-console/src/entry.tsx +++ b/web-console/src/entry.tsx @@ -20,7 +20,7 @@ import 'regenerator-runtime/runtime'; import './bootstrap/ace'; import { OverlaysProvider } from '@blueprintjs/core'; -import { QueryRunner } from '@druid-toolkit/query'; +import { QueryRunner } from 'druid-query-toolkit'; import { createRoot } from 'react-dom/client'; import { bootstrapJsonParse } from './bootstrap/json-parser'; diff --git a/web-console/src/helpers/capabilities.ts b/web-console/src/helpers/capabilities.ts index 013f9368c58c..3255ec93be73 100644 --- a/web-console/src/helpers/capabilities.ts +++ b/web-console/src/helpers/capabilities.ts @@ -211,6 +211,10 @@ export class Capabilities { }; } + public clone(): Capabilities { + return new Capabilities(this.valueOf()); + } + public getMode(): CapabilitiesMode { if (!this.hasSql()) return 'no-sql'; if (!this.hasCoordinatorAccess()) return 'no-proxy'; diff --git a/web-console/src/helpers/execution/general.ts b/web-console/src/helpers/execution/general.ts index a90c02a63061..0e78834172b1 100644 --- a/web-console/src/helpers/execution/general.ts +++ b/web-console/src/helpers/execution/general.ts @@ -16,8 +16,8 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; import type { CancelToken } from 'axios'; +import type { QueryResult } from 'druid-query-toolkit'; import type { Execution } from '../../druid-models'; import { IntermediateQueryState } from '../../utils'; diff --git a/web-console/src/helpers/execution/sql-task-execution.ts b/web-console/src/helpers/execution/sql-task-execution.ts index f4dd45a2cb91..68cf0763b425 100644 --- a/web-console/src/helpers/execution/sql-task-execution.ts +++ b/web-console/src/helpers/execution/sql-task-execution.ts @@ -16,8 +16,8 @@ * limitations under the License. */ -import { QueryResult } from '@druid-toolkit/query'; import type { AxiosResponse, CancelToken } from 'axios'; +import { QueryResult } from 'druid-query-toolkit'; import type { AsyncStatusResponse, MsqTaskPayloadResponse, QueryContext } from '../../druid-models'; import { Execution } from '../../druid-models'; diff --git a/web-console/src/helpers/spec-conversion.ts b/web-console/src/helpers/spec-conversion.ts index 00bc3d5f832c..9fc95d3b60f5 100644 --- a/web-console/src/helpers/spec-conversion.ts +++ b/web-console/src/helpers/spec-conversion.ts @@ -25,7 +25,7 @@ import { SqlExpression, SqlType, T, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import type { diff --git a/web-console/src/hooks/use-clock.ts b/web-console/src/hooks/use-clock.ts index 5d1264421957..a909a2c1f740 100644 --- a/web-console/src/hooks/use-clock.ts +++ b/web-console/src/hooks/use-clock.ts @@ -16,7 +16,9 @@ * limitations under the License. */ -import { useEffect, useState } from 'react'; +import { useState } from 'react'; + +import { useInterval } from './use-interval'; function getNowToSecond(): Date { const now = new Date(); @@ -24,18 +26,12 @@ function getNowToSecond(): Date { return now; } -export function useClock() { +export function useClock(updateInterval = 1000) { const [now, setNow] = useState(getNowToSecond); - useEffect(() => { - const checkInterval = setInterval(() => { - setNow(getNowToSecond()); - }, 1000); - - return () => { - clearInterval(checkInterval); - }; - }, []); + useInterval(() => { + setNow(getNowToSecond()); + }, updateInterval); return now; } diff --git a/web-console/src/react-table/react-table-utils.spec.ts b/web-console/src/react-table/react-table-utils.spec.ts index 0a1bbf3f9ad5..1dd245b49f70 100644 --- a/web-console/src/react-table/react-table-utils.spec.ts +++ b/web-console/src/react-table/react-table-utils.spec.ts @@ -75,5 +75,9 @@ describe('react-table-utils', () => { { id: 'x', value: '~y' }, { id: 'z', value: '=w&' }, ]); + expect(stringToTableFilters('x<3&y<=3')).toEqual([ + { id: 'x', value: '<3' }, + { id: 'y', value: '<=3' }, + ]); }); }); diff --git a/web-console/src/react-table/react-table-utils.ts b/web-console/src/react-table/react-table-utils.ts index ce9497a1209a..7732c90b231c 100644 --- a/web-console/src/react-table/react-table-utils.ts +++ b/web-console/src/react-table/react-table-utils.ts @@ -18,8 +18,7 @@ import type { IconName } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { SqlExpression } from '@druid-toolkit/query'; -import { C, F } from '@druid-toolkit/query'; +import { C, F, SqlExpression } from 'druid-query-toolkit'; import type { Filter } from 'react-table'; import { addOrUpdate, caseInsensitiveContains, filterMap } from '../utils'; @@ -32,9 +31,9 @@ export const STANDARD_TABLE_PAGE_SIZE_OPTIONS = [50, 100, 200]; export const SMALL_TABLE_PAGE_SIZE = 25; export const SMALL_TABLE_PAGE_SIZE_OPTIONS = [25, 50, 100]; -export type FilterMode = '~' | '=' | '!=' | '<=' | '>='; +export type FilterMode = '~' | '=' | '!=' | '<' | '<=' | '>' | '>='; -export const FILTER_MODES: FilterMode[] = ['~', '=', '!=', '<=', '>=']; +export const FILTER_MODES: FilterMode[] = ['~', '=', '!=', '<', '<=', '>', '>=']; export const FILTER_MODES_NO_COMPARISON: FilterMode[] = ['~', '=', '!=']; export function filterModeToIcon(mode: FilterMode): IconName { @@ -45,8 +44,12 @@ export function filterModeToIcon(mode: FilterMode): IconName { return IconNames.EQUALS; case '!=': return IconNames.NOT_EQUAL_TO; + case '<': + return IconNames.LESS_THAN; case '<=': return IconNames.LESS_THAN_OR_EQUAL_TO; + case '>': + return IconNames.GREATER_THAN; case '>=': return IconNames.GREATER_THAN_OR_EQUAL_TO; default: @@ -62,8 +65,12 @@ export function filterModeToTitle(mode: FilterMode): string { return 'Equals'; case '!=': return 'Not equals'; + case '<': + return 'Less than'; case '<=': return 'Less than or equal'; + case '>': + return 'Greater than'; case '>=': return 'Greater than or equal'; default: @@ -89,7 +96,7 @@ export function parseFilterModeAndNeedle( filter: Filter, loose = false, ): FilterModeAndNeedle | undefined { - const m = /^(~|=|!=|<=|>=)?(.*)$/.exec(String(filter.value)); + const m = /^(~|=|!=|<(?!=)|<=|>(?!=)|>=)?(.*)$/.exec(String(filter.value)); if (!m) return; if (!loose && !m[2]) return; const mode = (m[1] as FilterMode) || '~'; @@ -112,21 +119,28 @@ export function booleanCustomTableFilter(filter: Filter, value: unknown): boolea const modeAndNeedle = parseFilterModeAndNeedle(filter); if (!modeAndNeedle) return true; const { mode, needle } = modeAndNeedle; + const strValue = String(value); switch (mode) { case '=': - return String(value) === needle; + return strValue === needle; case '!=': - return String(value) !== needle; + return strValue !== needle; + + case '<': + return strValue < needle; case '<=': - return String(value) <= needle; + return strValue <= needle; + + case '>': + return strValue > needle; case '>=': - return String(value) >= needle; + return strValue >= needle; default: - return caseInsensitiveContains(String(value), needle); + return caseInsensitiveContains(strValue, needle); } } @@ -142,9 +156,15 @@ export function sqlQueryCustomTableFilter(filter: Filter): SqlExpression | undef case '!=': return column.unequal(needle); + case '<': + return column.lessThan(needle); + case '<=': return column.lessThanOrEqual(needle); + case '>': + return column.greaterThan(needle); + case '>=': return column.greaterThanOrEqual(needle); @@ -153,6 +173,10 @@ export function sqlQueryCustomTableFilter(filter: Filter): SqlExpression | undef } } +export function sqlQueryCustomTableFilters(filters: Filter[]): SqlExpression { + return SqlExpression.and(...filterMap(filters, sqlQueryCustomTableFilter)); +} + export function tableFiltersToString(tableFilters: Filter[]): string { return tableFilters .map(({ id, value }) => `${id}${value.replace(/[&%]/g, encodeURIComponent)}`) @@ -161,9 +185,11 @@ export function tableFiltersToString(tableFilters: Filter[]): string { export function stringToTableFilters(str: string | undefined): Filter[] { if (!str) return []; - // '~' | '=' | '!=' | '<=' | '>='; + // '~' | '=' | '!=' | '<' | '<=' | '>' | '>='; return filterMap(str.split('&'), clause => { - const m = /^(\w+)((?:~|=|!=|<=|>=).*)$/.exec(clause.replace(/%2[56]/g, decodeURIComponent)); + const m = /^(\w+)((?:~|=|!=|<(?!=)|<=|>(?!=)|>=).*)$/.exec( + clause.replace(/%2[56]/g, decodeURIComponent), + ); if (!m) return; return { id: m[1], value: m[2] }; }); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts new file mode 100755 index 000000000000..5ba63b04468f --- /dev/null +++ b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil-utc.spec.ts @@ -0,0 +1,169 @@ +/* + * 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 { shifters } from './date-floor-shift-ceil'; + +function pairwise(array: T[], callback: (t1: T, t2: T) => void) { + for (let i = 0; i < array.length - 1; i++) { + callback(array[i], array[i + 1]); + } +} + +describe('floor, shift, ceil (UTC)', () => { + const tz = 'Etc/UTC'; + + it('moves seconds', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-04T00:00:03Z'), + new Date('2012-11-04T00:00:06Z'), + new Date('2012-11-04T00:00:09Z'), + new Date('2012-11-04T00:00:12Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.second.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('rounds minutes', () => { + expect(shifters.minute.round(new Date('2012-11-04T00:29:00Z'), 15, tz)).toEqual( + new Date('2012-11-04T00:15:00Z'), + ); + + expect(shifters.minute.round(new Date('2012-11-04T00:29:00Z'), 4, tz)).toEqual( + new Date('2012-11-04T00:28:00Z'), + ); + }); + + it('moves minutes', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-04T00:03:00Z'), + new Date('2012-11-04T00:06:00Z'), + new Date('2012-11-04T00:09:00Z'), + new Date('2012-11-04T00:12:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.minute.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('floors hour correctly', () => { + expect(shifters.hour.floor(new Date('2012-11-04T00:30:00Z'), tz)).toEqual( + new Date('2012-11-04T00:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00Z'), tz)).toEqual( + new Date('2012-11-04T01:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00Z'), tz)).toEqual( + new Date('2012-11-04T01:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T02:30:00Z'), tz)).toEqual( + new Date('2012-11-04T02:00:00Z'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T03:30:00Z'), tz)).toEqual( + new Date('2012-11-04T03:00:00Z'), + ); + }); + + it('moves hour', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-04T01:00:00Z'), + new Date('2012-11-04T02:00:00Z'), + new Date('2012-11-04T03:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('moves day', () => { + const dates: Date[] = [ + new Date('2012-11-03T00:00:00Z'), + new Date('2012-11-04T00:00:00Z'), + new Date('2012-11-05T00:00:00Z'), + new Date('2012-11-06T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.day.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('ceils day', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-12T00:00:00.000Z'); + expect(shifters.day.ceil(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-08T00:00:00.000Z'); + d2 = new Date('2014-12-08T00:00:00.000Z'); + expect(shifters.day.ceil(d1, tz)).toEqual(d2); + }); + + it('moves week', () => { + const dates: Date[] = [ + new Date('2012-10-29T00:00:00Z'), + new Date('2012-11-05T00:00:00Z'), + new Date('2012-11-12T00:00:00Z'), + new Date('2012-11-19T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.week.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('floors week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-08T00:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-01T00:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + }); + + it('ceils week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-15T00:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-08T00:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + }); + + it('moves month', () => { + const dates: Date[] = [ + new Date('2012-11-01T00:00:00Z'), + new Date('2012-12-01T00:00:00Z'), + new Date('2013-01-01T00:00:00Z'), + new Date('2013-02-01T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.month.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts month on the 31st', () => { + const d1 = new Date('2016-03-31T00:00:00.000Z'); + const d2 = new Date('2016-05-01T00:00:00.000Z'); + expect(shifters.month.shift(d1, tz, 1)).toEqual(d2); + }); + + it('moves year', () => { + const dates: Date[] = [ + new Date('2010-01-01T00:00:00Z'), + new Date('2011-01-01T00:00:00Z'), + new Date('2012-01-01T00:00:00Z'), + new Date('2013-01-01T00:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.year.shift(d1, tz, 1)).toEqual(d2)); + }); +}); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts new file mode 100755 index 000000000000..1612c9ce5792 --- /dev/null +++ b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.spec.ts @@ -0,0 +1,181 @@ +/* + * 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 { shifters } from './date-floor-shift-ceil'; + +function pairwise(array: T[], callback: (t1: T, t2: T) => void) { + for (let i = 0; i < array.length - 1; i++) { + callback(array[i], array[i + 1]); + } +} + +describe('floor/shift/ceil', () => { + const tz = 'America/Los_Angeles'; + + it('shifts seconds', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-04T00:00:03-07:00'), + new Date('2012-11-04T00:00:06-07:00'), + new Date('2012-11-04T00:00:09-07:00'), + new Date('2012-11-04T00:00:12-07:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.second.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('shifts minutes', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-04T00:03:00-07:00'), + new Date('2012-11-04T00:06:00-07:00'), + new Date('2012-11-04T00:09:00-07:00'), + new Date('2012-11-04T00:12:00-07:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.minute.shift(d1, tz, 3)).toEqual(d2)); + }); + + it('floors hour correctly', () => { + expect(shifters.hour.floor(new Date('2012-11-04T00:30:00-07:00'), tz)).toEqual( + new Date('2012-11-04T00:00:00-07:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00-07:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T01:30:00-08:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T02:30:00-08:00'), tz)).toEqual( + new Date('2012-11-04T02:00:00-08:00'), + ); + + expect(shifters.hour.floor(new Date('2012-11-04T03:30:00-08:00'), tz)).toEqual( + new Date('2012-11-04T03:00:00-08:00'), + ); + }); + + it('shifting 24 hours over DST is not the same as shifting a day', () => { + const start = new Date('2012-11-04T07:00:00Z'); + + const shift1Day = shifters.day.shift(start, tz, 1); + const shift24Hours = shifters.hour.shift(start, tz, 24); + + expect(shift1Day).toEqual(new Date('2012-11-05T08:00:00Z')); + expect(shift24Hours).toEqual(new Date('2012-11-05T07:00:00Z')); + }); + + it('shifts hour over DST 1', () => { + const dates: Date[] = [ + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-04T08:00:00Z'), + new Date('2012-11-04T09:00:00Z'), + new Date('2012-11-04T10:00:00Z'), + new Date('2012-11-04T11:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('floors hour over DST 1', () => { + expect(shifters.hour.floor(new Date('2012-11-04T00:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T00:00:00-07:00'), + ); + expect(shifters.hour.floor(new Date('2012-11-04T01:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + expect(shifters.hour.floor(new Date('2012-11-04T02:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T01:00:00-07:00'), + ); + expect(shifters.hour.floor(new Date('2012-11-04T03:05:00-07:00'), tz)).toEqual( + new Date('2012-11-04T03:00:00-07:00'), + ); + }); + + it('shifts hour over DST 2', () => { + // "2018-03-11T09:00:00Z" + const dates: Date[] = [ + new Date('2018-03-11T01:00:00-07:00'), + new Date('2018-03-11T09:00:00Z'), + new Date('2018-03-11T10:00:00Z'), + new Date('2018-03-11T11:00:00Z'), + new Date('2018-03-11T12:00:00Z'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.hour.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts day over DST', () => { + const dates: Date[] = [ + new Date('2012-11-03T00:00:00-07:00'), + new Date('2012-11-04T00:00:00-07:00'), + new Date('2012-11-05T00:00:00-08:00'), + new Date('2012-11-06T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.day.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts week over DST', () => { + const dates: Date[] = [ + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-11-05T00:00:00-08:00'), + new Date('2012-11-12T00:00:00-08:00'), + new Date('2012-11-19T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.week.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('floors week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-08T08:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-01T08:00:00.000Z'); + expect(shifters.week.floor(d1, tz)).toEqual(d2); + }); + + it('ceils week correctly', () => { + let d1 = new Date('2014-12-11T22:11:57.469Z'); + let d2 = new Date('2014-12-15T08:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + + d1 = new Date('2014-12-07T12:11:57.469Z'); + d2 = new Date('2014-12-08T08:00:00.000Z'); + expect(shifters.week.ceil(d1, tz)).toEqual(d2); + }); + + it('shifts month over DST', () => { + const dates: Date[] = [ + new Date('2012-11-01T00:00:00-07:00'), + new Date('2012-12-01T00:00:00-08:00'), + new Date('2013-01-01T00:00:00-08:00'), + new Date('2013-02-01T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.month.shift(d1, tz, 1)).toEqual(d2)); + }); + + it('shifts year', () => { + const dates: Date[] = [ + new Date('2010-01-01T00:00:00-08:00'), + new Date('2011-01-01T00:00:00-08:00'), + new Date('2012-01-01T00:00:00-08:00'), + new Date('2013-01-01T00:00:00-08:00'), + ]; + pairwise(dates, (d1, d2) => expect(shifters.year.shift(d1, tz, 1)).toEqual(d2)); + }); +}); diff --git a/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts new file mode 100755 index 000000000000..3306b05267d9 --- /dev/null +++ b/web-console/src/utils/date-floor-shift-ceil/date-floor-shift-ceil.ts @@ -0,0 +1,296 @@ +/* + * 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 { fromDate, startOfWeek } from '@internationalized/date'; + +export type AlignFn = (dt: Date, tz: string) => Date; + +export type ShiftFn = (dt: Date, tz: string, step: number) => Date; + +export type RoundFn = (dt: Date, roundTo: number, tz: string) => Date; + +export interface TimeShifterNoCeil { + canonicalLength: number; + siblings?: number; + floor: AlignFn; + round: RoundFn; + shift: ShiftFn; +} + +export interface TimeShifter extends TimeShifterNoCeil { + ceil: AlignFn; +} + +function isUTC(tz: string): boolean { + return tz === 'Etc/UTC'; +} + +function adjustDay(day: number): number { + return (day + 6) % 7; +} + +function floorTo(n: number, roundTo: number): number { + return Math.floor(n / roundTo) * roundTo; +} + +function timeShifterFiller(tm: TimeShifterNoCeil): TimeShifter { + const { floor, shift } = tm; + return { + ...tm, + ceil: (dt: Date, tz: string) => { + const floored = floor(dt, tz); + if (floored.valueOf() === dt.valueOf()) return dt; // Just like ceil(3) is 3 and not 4 + return shift(floored, tz, 1); + }, + }; +} + +export const second = timeShifterFiller({ + canonicalLength: 1000, + siblings: 60, + floor: (dt, _tz) => { + // Seconds do not actually need a timezone because all timezones align on seconds... for now... + dt = new Date(dt.valueOf()); + dt.setUTCMilliseconds(0); + return dt; + }, + round: (dt, roundTo, _tz) => { + const cur = dt.getUTCSeconds(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCSeconds(adj); + return dt; + }, + shift: (dt, _tz, step) => { + dt = new Date(dt.valueOf()); + dt.setUTCSeconds(dt.getUTCSeconds() + step); + return dt; + }, +}); + +export const minute = timeShifterFiller({ + canonicalLength: 60000, + siblings: 60, + floor: (dt, _tz) => { + // Minutes do not actually need a timezone because all timezones align on minutes... for now... + dt = new Date(dt.valueOf()); + dt.setUTCSeconds(0, 0); + return dt; + }, + round: (dt, roundTo, _tz) => { + const cur = dt.getUTCMinutes(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCMinutes(adj); + return dt; + }, + shift: (dt, _tz, step) => { + dt = new Date(dt.valueOf()); + dt.setUTCMinutes(dt.getUTCMinutes() + step); + return dt; + }, +}); + +// Movement by hour is tz independent because in every timezone an hour is 60 min +function hourMove(dt: Date, _tz: string, step: number) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(dt.getUTCHours() + step); + return dt; +} + +export const hour = timeShifterFiller({ + canonicalLength: 3600000, + siblings: 24, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCMinutes(0, 0, 0); + return dt; + } else { + return fromDate(dt, tz).set({ second: 0, minute: 0, millisecond: 0 }).toDate(); + } + }, + round: (dt, roundTo, tz) => { + if (isUTC(tz)) { + const cur = dt.getUTCHours(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCHours(adj); + } else { + const cur = fromDate(dt, tz).hour; + const adj = floorTo(cur, roundTo); + if (cur !== adj) return hourMove(dt, tz, adj - cur); + } + return dt; + }, + shift: hourMove, +}); + +export const day = timeShifterFiller({ + canonicalLength: 24 * 3600000, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + return dt; + } else { + return fromDate(dt, tz).set({ hour: 0, second: 0, minute: 0, millisecond: 0 }).toDate(); + } + }, + shift: (dt, tz, step) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCDate(dt.getUTCDate() + step); + return dt; + } else { + return fromDate(dt, tz).add({ days: step }).toDate(); + } + }, + round: () => { + throw new Error('missing day round'); + }, +}); + +export const week = timeShifterFiller({ + canonicalLength: 7 * 24 * 3600000, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + dt.setUTCDate(dt.getUTCDate() - adjustDay(dt.getUTCDay())); + } else { + const zd = fromDate(dt, tz); + return startOfWeek( + zd.set({ hour: 0, second: 0, minute: 0, millisecond: 0 }), + 'fr-FR', // We want the week to start on Monday + ).toDate(); + } + return dt; + }, + shift: (dt, tz, step) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCDate(dt.getUTCDate() + step * 7); + return dt; + } else { + return fromDate(dt, tz).add({ weeks: step }).toDate(); + } + }, + round: () => { + throw new Error('missing week round'); + }, +}); + +function monthShift(dt: Date, tz: string, step: number) { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCMonth(dt.getUTCMonth() + step); + return dt; + } else { + return fromDate(dt, tz).add({ months: step }).toDate(); + } +} + +export const month = timeShifterFiller({ + canonicalLength: 30 * 24 * 3600000, + siblings: 12, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + dt.setUTCDate(1); + return dt; + } else { + return fromDate(dt, tz) + .set({ day: 1, hour: 0, second: 0, minute: 0, millisecond: 0 }) + .toDate(); + } + }, + round: (dt, roundTo, tz) => { + if (isUTC(tz)) { + const cur = dt.getUTCMonth(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCMonth(adj); + } else { + const cur = fromDate(dt, tz).month - 1; // Needs to be zero indexed + const adj = floorTo(cur, roundTo); + if (cur !== adj) return monthShift(dt, tz, adj - cur); + } + return dt; + }, + shift: monthShift, +}); + +function yearShift(dt: Date, tz: string, step: number) { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCFullYear(dt.getUTCFullYear() + step); + return dt; + } else { + return fromDate(dt, tz).add({ years: step }).toDate(); + } +} + +export const year = timeShifterFiller({ + canonicalLength: 365 * 24 * 3600000, + siblings: 1000, + floor: (dt, tz) => { + if (isUTC(tz)) { + dt = new Date(dt.valueOf()); + dt.setUTCHours(0, 0, 0, 0); + dt.setUTCMonth(0, 1); + return dt; + } else { + return fromDate(dt, tz) + .set({ month: 1, day: 1, hour: 0, second: 0, minute: 0, millisecond: 0 }) + .toDate(); + } + }, + round: (dt, roundTo, tz) => { + if (isUTC(tz)) { + const cur = dt.getUTCFullYear(); + const adj = floorTo(cur, roundTo); + if (cur !== adj) dt.setUTCFullYear(adj); + } else { + const cur = fromDate(dt, tz).year; + const adj = floorTo(cur, roundTo); + if (cur !== adj) return yearShift(dt, tz, adj - cur); + } + return dt; + }, + shift: yearShift, +}); + +export interface Shifters { + second: TimeShifter; + minute: TimeShifter; + hour: TimeShifter; + day: TimeShifter; + week: TimeShifter; + month: TimeShifter; + year: TimeShifter; + + [key: string]: TimeShifter; +} + +export const shifters: Shifters = { + second, + minute, + hour, + day, + week, + month, + year, +}; diff --git a/web-console/src/utils/date.spec.ts b/web-console/src/utils/date.spec.ts index 843c144244ef..b219ee17af06 100644 --- a/web-console/src/utils/date.spec.ts +++ b/web-console/src/utils/date.spec.ts @@ -17,7 +17,6 @@ */ import { - ceilToUtcDay, dateToIsoDateString, intervalToLocalDateRange, localDateRangeToInterval, @@ -60,12 +59,4 @@ describe('date', () => { expect(localDateRangeToInterval(intervalToLocalDateRange(interval))).toEqual(interval); }); }); - - describe('ceilToUtcDay', () => { - it('works', () => { - expect(ceilToUtcDay(new Date('2021-02-03T12:03:02.001Z'))).toEqual( - new Date('2021-02-04T00:00:00Z'), - ); - }); - }); }); diff --git a/web-console/src/utils/date.ts b/web-console/src/utils/date.ts index 1d5aa4f61e95..08f1d1df43ba 100644 --- a/web-console/src/utils/date.ts +++ b/web-console/src/utils/date.ts @@ -99,9 +99,10 @@ export function localDateRangeToInterval(localRange: DateRange): string { }`; } -export function ceilToUtcDay(date: Date): Date { - date = new Date(date.valueOf()); - date.setUTCHours(0, 0, 0, 0); - date.setUTCDate(date.getUTCDate() + 1); - return date; +export function maxDate(a: Date, b: Date): Date { + return a > b ? a : b; +} + +export function minDate(a: Date, b: Date): Date { + return a < b ? a : b; } diff --git a/web-console/src/utils/download.ts b/web-console/src/utils/download.ts index 4fb3342e523c..40b0d95e8b91 100644 --- a/web-console/src/utils/download.ts +++ b/web-console/src/utils/download.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { QueryResult } from '@druid-toolkit/query'; +import type { QueryResult } from 'druid-query-toolkit'; import FileSaver from 'file-saver'; import * as JSONBig from 'json-bigint-native'; diff --git a/web-console/src/utils/druid-query.spec.ts b/web-console/src/utils/druid-query.spec.ts index ee867ff47ea5..a940b8ac7bb6 100644 --- a/web-console/src/utils/druid-query.spec.ts +++ b/web-console/src/utils/druid-query.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; +import { sane } from 'druid-query-toolkit'; import { DruidError, getDruidErrorMessage } from './druid-query'; diff --git a/web-console/src/utils/druid-query.ts b/web-console/src/utils/druid-query.ts index 8102db89ca3a..ee1ba1ef4a8c 100644 --- a/web-console/src/utils/druid-query.ts +++ b/web-console/src/utils/druid-query.ts @@ -16,9 +16,9 @@ * limitations under the License. */ -import { C } from '@druid-toolkit/query'; import type { AxiosResponse, CancelToken } from 'axios'; import axios from 'axios'; +import { C } from 'druid-query-toolkit'; import { Api } from '../singletons'; @@ -358,6 +358,12 @@ export async function queryDruidSqlDart( return sqlResultResp.data; } +export async function getApiArray(url: string, cancelToken?: CancelToken): Promise { + const result = (await Api.instance.get(url, { cancelToken })).data; + if (!Array.isArray(result)) throw new Error('unexpected result'); + return result; +} + export interface QueryExplanation { query: any; signature: { name: string; type: string }[]; diff --git a/web-console/src/utils/duration/duration.spec.ts b/web-console/src/utils/duration/duration.spec.ts new file mode 100755 index 000000000000..8b20ac0a6b2a --- /dev/null +++ b/web-console/src/utils/duration/duration.spec.ts @@ -0,0 +1,505 @@ +/* + * 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 { Duration } from './duration'; + +describe('Duration', () => { + const TZ_LA = 'America/Los_Angeles'; + const TZ_JUNEAU = 'America/Juneau'; + + describe('errors', () => { + it('throws error if invalid duration', () => { + expect(() => new Duration('')).toThrow("Can not parse duration ''"); + + expect(() => new Duration('P00')).toThrow("Can not parse duration 'P00'"); + + expect(() => new Duration('P')).toThrow('Duration can not be empty'); + + expect(() => new Duration('P0YT0H')).toThrow('Duration can not be empty'); + + expect(() => new Duration('P0W').shift(new Date(), TZ_LA)).toThrow( + 'Duration can not have empty weeks', + ); + + expect(() => new Duration('P0Y0MT0H0M0S').shift(new Date(), TZ_LA)).toThrow( + 'Duration can not be empty', + ); + }); + + it('throws error if fromJS is not given a string', () => { + expect(() => new Duration(new Date() as any)).toThrow('Duration can not be empty'); + }); + }); + + describe('#toString', () => { + it('gives back the correct string', () => { + let durationStr: string; + + durationStr = 'P3Y'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + + durationStr = 'P2W'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).toString()).toEqual(durationStr); + }); + + it('eliminates 0', () => { + expect(new Duration('P0DT15H').toString()).toEqual('PT15H'); + }); + }); + + describe('fromCanonicalLength', () => { + it('handles zero', () => { + expect(() => { + Duration.fromCanonicalLength(0); + }).toThrow('length must be positive'); + }); + + it('works 1', () => { + expect(Duration.fromCanonicalLength(86400000).toString()).toEqual('P1D'); + }); + + it('works 2', () => { + const len = + new Date('2018-03-01T00:00:00Z').valueOf() - new Date('2016-02-22T00:00:00Z').valueOf(); + expect(Duration.fromCanonicalLength(len).toString()).toEqual('P2Y8D'); + }); + + it('works 3', () => { + const len = + new Date('2018-09-15T00:00:00Z').valueOf() - new Date('2018-09-04T00:00:00Z').valueOf(); + expect(Duration.fromCanonicalLength(len).toString()).toEqual('P11D'); + }); + + it('works with months', () => { + expect(Duration.fromCanonicalLength(2592000000).toString()).toEqual('P1M'); + expect(Duration.fromCanonicalLength(2678400000).toString()).toEqual('P1M1D'); + }); + + it('works without months', () => { + expect(Duration.fromCanonicalLength(2592000000, true).toString()).toEqual('P30D'); + expect(Duration.fromCanonicalLength(2678400000, true).toString()).toEqual('P31D'); + }); + }); + + describe('construct from span', () => { + it('parses days over DST', () => { + expect( + Duration.fromRange( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-11-05T00:00:00-08:00'), + TZ_LA, + ).toString(), + ).toEqual('P7D'); + + expect( + Duration.fromRange( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-11-12T00:00:00-08:00'), + TZ_LA, + ).toString(), + ).toEqual('P14D'); + }); + + it('parses complex case', () => { + expect( + Duration.fromRange( + new Date('2012-10-29T00:00:00-07:00'), + new Date(new Date('2012-11-05T00:00:00-08:00').valueOf() - 1000), + TZ_LA, + ).toString(), + ).toEqual('P6DT24H59M59S'); + + expect( + Duration.fromRange( + new Date('2012-01-01T00:00:00-08:00'), + new Date('2013-03-04T04:05:06-08:00'), + TZ_LA, + ).toString(), + ).toEqual('P1Y2M3DT4H5M6S'); + }); + }); + + describe('#isFloorable', () => { + const floorable = 'P1Y P5Y P10Y P100Y P1M P2M P3M P4M P1D'.split(' '); + for (const v of floorable) { + it(`works on floorable ${v}`, () => { + expect(new Duration(v).isFloorable()).toEqual(true); + }); + } + + const unfloorable = 'P1Y1M P5M P2D P3D'.split(' '); + for (const v of unfloorable) { + it(`works on not floorable ${v}`, () => { + expect(new Duration(v).isFloorable()).toEqual(false); + }); + } + }); + + describe('#floor', () => { + it('throws error if complex duration', () => { + expect(() => new Duration('P1Y2D').floor(new Date(), TZ_LA)).toThrow( + 'Can not floor on a complex duration', + ); + + expect(() => new Duration('P3DT15H').floor(new Date(), TZ_LA)).toThrow( + 'Can not floor on a complex duration', + ); + + expect(() => new Duration('PT5H').floor(new Date(), TZ_LA)).toThrow( + 'Can not floor on a hour duration that does not divide into 24', + ); + }); + + it('works for year', () => { + const p1y = new Duration('P1Y'); + expect(p1y.floor(new Date('2013-09-29T01:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-01-01T00:00:00.000-08:00'), + ); + }); + + it('works for PT2M', () => { + const pt2h = new Duration('PT2M'); + expect(pt2h.floor(new Date('2013-09-29T03:03:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-29T03:02:00.000-07:00'), + ); + }); + + it('works for P2H', () => { + const pt2h = new Duration('PT2H'); + expect(pt2h.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-29T02:00:00.000-07:00'), + ); + }); + + it('works for PT12H', () => { + const pt12h = new Duration('PT12H'); + expect(pt12h.floor(new Date('2015-09-12T13:05:00-08:00'), TZ_JUNEAU)).toEqual( + new Date('2015-09-12T12:00:00-08:00'), + ); + }); + + it('works for P1W', () => { + const p1w = new Duration('P1W'); + + expect(p1w.floor(new Date('2013-09-29T01:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-23T07:00:00.000Z'), + ); + + expect(p1w.floor(new Date('2013-10-03T01:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-09-30T00:00:00.000-07:00'), + ); + }); + + it('works for P3M', () => { + const p3m = new Duration('P3M'); + expect(p3m.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-07-01T00:00:00.000-07:00'), + ); + + expect(p3m.floor(new Date('2013-02-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2013-01-01T00:00:00.000-08:00'), + ); + }); + + it('works for P4Y', () => { + const p4y = new Duration('P4Y'); + expect(p4y.floor(new Date('2013-09-29T03:02:03.456-07:00'), TZ_LA)).toEqual( + new Date('2012-01-01T00:00:00.000-08:00'), + ); + }); + }); + + describe('#shift', () => { + it('works for weeks', () => { + let p1w = new Duration('P1W'); + expect(p1w.shift(new Date('2012-10-29T00:00:00-07:00'), TZ_LA)).toEqual( + new Date('2012-11-05T00:00:00-08:00'), + ); + + p1w = new Duration('P1W'); + expect(p1w.shift(new Date('2012-10-29T00:00:00-07:00'), TZ_LA, 2)).toEqual( + new Date('2012-11-12T00:00:00-08:00'), + ); + + const p2w = new Duration('P2W'); + expect(p2w.shift(new Date('2012-10-29T05:16:17-07:00'), TZ_LA)).toEqual( + new Date('2012-11-12T05:16:17-08:00'), + ); + }); + + it('works for general complex case', () => { + const pComplex = new Duration('P1Y2M3DT4H5M6S'); + expect(pComplex.shift(new Date('2012-01-01T00:00:00-08:00'), TZ_LA)).toEqual( + new Date('2013-03-04T04:05:06-08:00'), + ); + }); + }); + + describe('#materialize', () => { + it('works for weeks', () => { + const p1w = new Duration('P1W'); + + expect( + p1w.materialize( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-12-01T00:00:00-08:00'), + TZ_LA, + ), + ).toEqual([ + new Date('2012-10-29T07:00:00.000Z'), + new Date('2012-11-05T08:00:00.000Z'), + new Date('2012-11-12T08:00:00.000Z'), + new Date('2012-11-19T08:00:00.000Z'), + new Date('2012-11-26T08:00:00.000Z'), + ]); + + expect( + p1w.materialize( + new Date('2012-10-29T00:00:00-07:00'), + new Date('2012-12-01T00:00:00-08:00'), + TZ_LA, + 2, + ), + ).toEqual([ + new Date('2012-10-29T07:00:00.000Z'), + new Date('2012-11-12T08:00:00.000Z'), + new Date('2012-11-26T08:00:00.000Z'), + ]); + }); + }); + + describe('#isAligned', () => { + it('works for weeks', () => { + const p1w = new Duration('P1W'); + expect(p1w.isAligned(new Date('2012-10-29T00:00:00-07:00'), TZ_LA)).toEqual(true); + expect(p1w.isAligned(new Date('2012-10-29T00:00:00-07:00'), 'Etc/UTC')).toEqual(false); + }); + }); + + describe('#dividesBy', () => { + const divisible = 'P5Y/P1Y P1D/P1D P1M/P1D P1W/P1D P1D/PT6H PT3H/PT1H'.split(' '); + for (const v of divisible) { + it(`works for ${v} (true)`, () => { + const p = v.split('/'); + expect(new Duration(p[0]).dividesBy(new Duration(p[1]))).toEqual(true); + }); + } + + const undivisible = 'P1D/P1M PT5H/PT1H'.split(' '); + for (const v of undivisible) { + it(`works for ${v} (false)`, () => { + const p = v.split('/'); + expect(new Duration(p[0]).dividesBy(new Duration(p[1]))).toEqual(false); + }); + } + }); + + describe('#getCanonicalLength', () => { + it('gives back the correct canonical length', () => { + let durationStr: string; + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(94608000000); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(1209600000); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(18000000); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getCanonicalLength()).toEqual(313200000); + }); + }); + + describe('#add()', () => { + it('works with a simple duration', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('P1D'); + + expect(d1.add(d2).toString()).toEqual('P2D'); + }); + + it('works with heterogeneous spans', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('P1Y'); + + expect(d1.add(d2).toString()).toEqual('P1Y1D'); + }); + + it('works with weeks', () => { + let d1 = new Duration('P1W'); + let d2 = new Duration('P2W'); + expect(d1.add(d2).toString()).toEqual('P3W'); + + d1 = new Duration('P6D'); + d2 = new Duration('P1D'); + expect(d1.add(d2).toString()).toEqual('P1W'); + }); + }); + + describe('#subtract()', () => { + it('works with a simple duration', () => { + const d1 = new Duration('P1DT2H'); + const d2 = new Duration('PT1H'); + + expect(d1.subtract(d2).toString()).toEqual('P1DT1H'); + }); + + it('works with a less simple duration', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('PT1H'); + + expect(d1.subtract(d2).toString()).toEqual('PT23H'); + }); + + it('works with weeks', () => { + const d1 = new Duration('P1W'); + const d2 = new Duration('P1D'); + + expect(d1.subtract(d2).toString()).toEqual('P6D'); + }); + + it('throws an error if result is going to be negative', () => { + const d1 = new Duration('P1D'); + const d2 = new Duration('P2D'); + + expect(() => d1.subtract(d2)).toThrow(); + }); + }); + + describe('#multiply()', () => { + it('works with a simple duration', () => { + const d = new Duration('P1D'); + expect(d.multiply(5).toString()).toEqual('P5D'); + }); + + it('works with a less simple duration', () => { + const d = new Duration('P1DT2H'); + expect(d.multiply(2).toString()).toEqual('P2DT4H'); + }); + + it('works with weeks', () => { + const d = new Duration('P1W'); + expect(d.multiply(5).toString()).toEqual('P5W'); + }); + + it('throws an error if result is going to be negative', () => { + const d = new Duration('P1D'); + expect(() => d.multiply(-1)).toThrow('Multiplier must be positive non-zero'); + }); + + it('gets description properly', () => { + const d = new Duration('P2D'); + expect(d.multiply(2).getDescription(true)).toEqual('4 Days'); + }); + }); + + describe('#getDescription()', () => { + it('gives back the correct description', () => { + let durationStr: string; + + durationStr = 'P1D'; + expect(new Duration(durationStr).getDescription()).toEqual('day'); + + durationStr = 'P1DT2H'; + expect(new Duration(durationStr).getDescription()).toEqual('1 day, 2 hours'); + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getDescription()).toEqual('3 years'); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getDescription()).toEqual('2 weeks'); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getDescription()).toEqual('5 hours'); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getDescription()).toEqual('3 days, 15 hours'); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getDescription(true)).toEqual('3 Days, 15 Hours'); + }); + }); + + describe('#getSingleSpan()', () => { + it('gives back the correct span', () => { + let durationStr: string; + + durationStr = 'P1D'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('day'); + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('year'); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('week'); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getSingleSpan()).toEqual('hour'); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpan()).toBeUndefined(); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpan()).toBeUndefined(); + }); + }); + + describe('#getSingleSpanValue()', () => { + it('gives back the correct span value', () => { + let durationStr: string; + + durationStr = 'P1D'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(1); + + durationStr = 'P3Y'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(3); + + durationStr = 'P2W'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(2); + + durationStr = 'PT5H'; + expect(new Duration(durationStr).getSingleSpanValue()).toEqual(5); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpanValue()).toBeUndefined(); + + durationStr = 'P3DT15H'; + expect(new Duration(durationStr).getSingleSpanValue()).toBeUndefined(); + }); + }); + + describe('#limitToDays', () => { + it('works', () => { + expect(new Duration('P6D').limitToDays().toString()).toEqual('P6D'); + + expect(new Duration('P1M').limitToDays().toString()).toEqual('P30D'); + + expect(new Duration('P1Y').limitToDays().toString()).toEqual('P365D'); + + expect(new Duration('P1Y2M').limitToDays().toString()).toEqual('P425D'); + }); + }); +}); diff --git a/web-console/src/utils/duration/duration.ts b/web-console/src/utils/duration/duration.ts new file mode 100755 index 000000000000..bc7236821f12 --- /dev/null +++ b/web-console/src/utils/duration/duration.ts @@ -0,0 +1,388 @@ +/* + * 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 { second, shifters } from '../date-floor-shift-ceil/date-floor-shift-ceil'; +import { capitalizeFirst, pluralIfNeeded } from '../general'; + +export const TZ_UTC = 'Etc/UTC'; + +export type DurationSpan = 'year' | 'month' | 'week' | 'day' | 'hour' | 'minute' | 'second'; + +const SPANS_WITH_WEEK: DurationSpan[] = [ + 'year', + 'month', + 'week', + 'day', + 'hour', + 'minute', + 'second', +]; +const SPANS_WITHOUT_WEEK: DurationSpan[] = ['year', 'month', 'day', 'hour', 'minute', 'second']; +const SPANS_WITHOUT_WEEK_OR_MONTH: DurationSpan[] = ['year', 'day', 'hour', 'minute', 'second']; +const SPANS_UP_TO_DAY: DurationSpan[] = ['day', 'hour', 'minute', 'second']; + +export type DurationValue = Partial>; + +const periodWeekRegExp = /^P(\d+)W$/; +const periodRegExp = /^P(?:(\d+)Y)?(?:(\d+)M)?(?:(\d+)D)?(?:T(?:(\d+)H)?(?:(\d+)M)?(?:(\d+)S)?)?$/; +// P (year ) (month ) (day ) T(hour ) (minute ) (second ) + +function getSpansFromString(durationStr: string): DurationValue { + const spans: DurationValue = {}; + let matches: RegExpExecArray | null; + if ((matches = periodWeekRegExp.exec(durationStr))) { + spans.week = Number(matches[1]); + if (!spans.week) throw new Error('Duration can not have empty weeks'); + } else if ((matches = periodRegExp.exec(durationStr))) { + const nums = matches.map(Number); + for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { + const span = SPANS_WITHOUT_WEEK[i]; + const value = nums[i + 1]; + if (value) spans[span] = value; + } + } else { + throw new Error("Can not parse duration '" + durationStr + "'"); + } + return spans; +} + +function getSpansFromStartEnd(start: Date, end: Date, timezone: string): DurationValue { + start = second.floor(start, timezone); + end = second.floor(end, timezone); + if (end <= start) throw new Error('start must come before end'); + + const spans: DurationValue = {}; + let iterator: Date = start; + for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { + const span = SPANS_WITHOUT_WEEK[i]; + let spanCount = 0; + + // Shortcut + const length = end.valueOf() - iterator.valueOf(); + const canonicalLength: number = shifters[span].canonicalLength; + if (length < canonicalLength / 4) continue; + const numberToFit = Math.min(0, Math.floor(length / canonicalLength) - 1); + let iteratorMove: Date; + if (numberToFit > 0) { + // try to skip by numberToFit + iteratorMove = shifters[span].shift(iterator, timezone, numberToFit); + if (iteratorMove <= end) { + spanCount += numberToFit; + iterator = iteratorMove; + } + } + + while (true) { + iteratorMove = shifters[span].shift(iterator, timezone, 1); + if (iteratorMove <= end) { + iterator = iteratorMove; + spanCount++; + } else { + break; + } + } + + if (spanCount) { + spans[span] = spanCount; + } + } + return spans; +} + +function removeZeros(spans: DurationValue): DurationValue { + const newSpans: DurationValue = {}; + for (let i = 0; i < SPANS_WITH_WEEK.length; i++) { + const span = SPANS_WITH_WEEK[i]; + if (Number(spans[span]) > 0) { + newSpans[span] = spans[span]; + } + } + return newSpans; +} + +function fitIntoSpans(length: number, spansToCheck: DurationSpan[]): DurationValue { + const spans: DurationValue = {}; + + let lengthLeft = length; + for (let i = 0; i < spansToCheck.length; i++) { + const span = spansToCheck[i]; + const spanLength = shifters[span].canonicalLength; + const count = Math.floor(lengthLeft / spanLength); + + if (count) { + lengthLeft -= spanLength * count; + spans[span] = count; + } + } + + return spans; +} + +/** + * Represents an ISO duration like P1DT3H + */ +export class Duration { + public readonly singleSpan?: DurationSpan; + public readonly spans: Readonly; + + static fromCanonicalLength(length: number, skipMonths = false): Duration { + if (length <= 0) throw new Error('length must be positive'); + let spans = fitIntoSpans(length, skipMonths ? SPANS_WITHOUT_WEEK_OR_MONTH : SPANS_WITHOUT_WEEK); + + if ( + length % shifters['week'].canonicalLength === 0 && // Weeks fits + (Object.keys(spans).length > 1 || // We already have a more complex span + spans['day']) // or... we only have days and it might be simpler to express as weeks + ) { + spans = { week: length / shifters['week'].canonicalLength }; + } + + return new Duration(spans); + } + + static fromCanonicalLengthUpToDays(length: number): Duration { + if (length <= 0) throw new Error('length must be positive'); + return new Duration(fitIntoSpans(length, SPANS_UP_TO_DAY)); + } + + static fromRange(start: Date, end: Date, timezone: string): Duration { + return new Duration(getSpansFromStartEnd(start, end, timezone)); + } + + static pickSmallestGranularityThatFits( + granularities: Duration[], + span: number, + maxEntities: number, + ): Duration { + for (const granularity of granularities) { + if (span / granularity.getCanonicalLength() < maxEntities) return granularity; + } + return granularities[granularities.length - 1]; + } + + constructor(spans: DurationValue | string) { + const effectiveSpans: DurationValue = + typeof spans === 'string' ? getSpansFromString(spans) : removeZeros(spans); + + const usedSpans = Object.keys(effectiveSpans) as DurationSpan[]; + if (!usedSpans.length) throw new Error('Duration can not be empty'); + if (usedSpans.length === 1) { + this.singleSpan = usedSpans[0]; + } else if (effectiveSpans.week) { + throw new Error("Can not mix 'week' and other spans"); + } + this.spans = effectiveSpans; + } + + public toString() { + const strArr: string[] = ['P']; + const spans = this.spans; + if (spans.week) { + strArr.push(String(spans.week), 'W'); + } else { + let addedT = false; + for (let i = 0; i < SPANS_WITHOUT_WEEK.length; i++) { + const span = SPANS_WITHOUT_WEEK[i]; + const value = spans[span]; + if (!value) continue; + if (!addedT && i >= 3) { + strArr.push('T'); + addedT = true; + } + strArr.push(String(value), span[0].toUpperCase()); + } + } + return strArr.join(''); + } + + public add(duration: Duration): Duration { + return Duration.fromCanonicalLength(this.getCanonicalLength() + duration.getCanonicalLength()); + } + + public subtract(duration: Duration): Duration { + const newCanonicalDuration = this.getCanonicalLength() - duration.getCanonicalLength(); + if (newCanonicalDuration < 0) throw new Error('A duration can not be negative.'); + return Duration.fromCanonicalLength(newCanonicalDuration); + } + + public multiply(multiplier: number): Duration { + if (multiplier <= 0) throw new Error('Multiplier must be positive non-zero'); + if (multiplier === 1) return this; + const newCanonicalDuration = this.getCanonicalLength() * multiplier; + return Duration.fromCanonicalLength(newCanonicalDuration); + } + + public valueOf() { + return this.spans; + } + + public equals(other: Duration | undefined): boolean { + return other instanceof Duration && this.toString() === other.toString(); + } + + public isSimple(): boolean { + const { singleSpan } = this; + if (!singleSpan) return false; + return this.spans[singleSpan] === 1; + } + + public isFloorable(): boolean { + const { singleSpan } = this; + if (!singleSpan) return false; + const span = Number(this.spans[singleSpan]); + if (span === 1) return true; + const { siblings } = shifters[singleSpan]; + if (!siblings) return false; + return siblings % span === 0; + } + + /** + * Floors the date according to this duration. + * @param date The date to floor + * @param timezone The timezone within which to floor + */ + public floor(date: Date, timezone: string): Date { + const { singleSpan } = this; + if (!singleSpan) throw new Error('Can not floor on a complex duration'); + const span = this.spans[singleSpan]!; + const mover = shifters[singleSpan]; + let dt = mover.floor(date, timezone); + if (span !== 1) { + if (!mover.siblings) { + throw new Error(`Can not floor on a ${singleSpan} duration that is not 1`); + } + if (mover.siblings % span !== 0) { + throw new Error( + `Can not floor on a ${singleSpan} duration that does not divide into ${mover.siblings}`, + ); + } + dt = mover.round(dt, span, timezone); + } + return dt; + } + + /** + * Moves the given date by 'step' times of the duration + * Negative step value will move back in time. + * @param date The date to move + * @param timezone The timezone within which to make the move + * @param step The number of times to step by the duration + */ + public shift(date: Date, timezone: string, step = 1): Date { + const spans = this.spans; + for (const span of SPANS_WITH_WEEK) { + const value = spans[span]; + if (value) date = shifters[span].shift(date, timezone, step * value); + } + return date; + } + + public ceil(date: Date, timezone: string): Date { + const floored = this.floor(date, timezone); + if (floored.valueOf() === date.valueOf()) return date; // Just like ceil(3) is 3 and not 4 + return this.shift(floored, timezone, 1); + } + + public round(date: Date, timezone: string): Date { + const floorDate = this.floor(date, timezone); + const ceilDate = this.ceil(date, timezone); + const distanceToFloor = Math.abs(date.valueOf() - floorDate.valueOf()); + const distanceToCeil = Math.abs(date.valueOf() - ceilDate.valueOf()); + return distanceToFloor < distanceToCeil ? floorDate : ceilDate; + } + + /** + * Materializes all the values of this duration form start to end + * @param start The date to start on + * @param end The date to start on + * @param timezone The timezone within which to materialize + * @param step The number of times to step by the duration + */ + public materialize(start: Date, end: Date, timezone: string, step = 1): Date[] { + const values: Date[] = []; + let iter = this.floor(start, timezone); + while (iter <= end) { + values.push(iter); + iter = this.shift(iter, timezone, step); + } + return values; + } + + /** + * Checks to see if date is aligned to this duration within the timezone (floors to itself) + * @param date The date to check + * @param timezone The timezone within which to make the check + */ + public isAligned(date: Date, timezone: string): boolean { + return this.floor(date, timezone).valueOf() === date.valueOf(); + } + + /** + * Check to see if this duration can be divided by the given duration + * @param smaller The smaller duration to divide by + */ + public dividesBy(smaller: Duration): boolean { + const myCanonicalLength = this.getCanonicalLength(); + const smallerCanonicalLength = smaller.getCanonicalLength(); + return ( + myCanonicalLength % smallerCanonicalLength === 0 && + this.isFloorable() && + smaller.isFloorable() + ); + } + + public getCanonicalLength(): number { + const spans = this.spans; + let length = 0; + for (const span of SPANS_WITH_WEEK) { + const value = spans[span]; + if (value) length += value * shifters[span].canonicalLength; + } + return length; + } + + public getDescription(capitalize?: boolean): string { + const spans = this.spans; + const description: string[] = []; + for (const span of SPANS_WITH_WEEK) { + const value = spans[span]; + const spanTitle = capitalize ? capitalizeFirst(span) : span; + if (value) { + if (value === 1 && this.singleSpan) { + description.push(spanTitle); + } else { + description.push(pluralIfNeeded(value, spanTitle)); + } + } + } + return description.join(', '); + } + + public getSingleSpan(): string | undefined { + return this.singleSpan; + } + + public getSingleSpanValue(): number | undefined { + if (!this.singleSpan) return; + return this.spans[this.singleSpan]; + } + + public limitToDays(): Duration { + return Duration.fromCanonicalLengthUpToDays(this.getCanonicalLength()); + } +} diff --git a/web-console/src/utils/general.spec.ts b/web-console/src/utils/general.spec.ts index 4b97cb19acb0..39deab9420aa 100644 --- a/web-console/src/utils/general.spec.ts +++ b/web-console/src/utils/general.spec.ts @@ -29,8 +29,8 @@ import { hashJoaat, moveElement, moveToIndex, - objectHash, offsetToRowColumn, + OVERLAY_OPEN_SELECTOR, parseCsvLine, swapElements, } from './general'; @@ -178,12 +178,6 @@ describe('general', () => { }); }); - describe('objectHash', () => { - it('works', () => { - expect(objectHash({ hello: 'world1' })).toEqual('cc14ad13'); - }); - }); - describe('offsetToRowColumn', () => { it('works', () => { const str = 'Hello\nThis is a test\nstring.'; @@ -217,4 +211,10 @@ describe('general', () => { expect(caseInsensitiveEquals(undefined, '')).toEqual(false); }); }); + + describe('OVERLAY_OPEN_SELECTOR', () => { + it('is what it is', () => { + expect(OVERLAY_OPEN_SELECTOR).toEqual('.bp5-portal .bp5-overlay-open'); + }); + }); }); diff --git a/web-console/src/utils/general.tsx b/web-console/src/utils/general.tsx index b742013b2e83..d128d47f3462 100644 --- a/web-console/src/utils/general.tsx +++ b/web-console/src/utils/general.tsx @@ -59,6 +59,10 @@ export function isSimpleArray(a: any): a is (string | number | boolean)[] { ); } +export function arraysEqualByElement(xs: T[], ys: T[]): boolean { + return xs.length === ys.length && xs.every((x, i) => x === ys[i]); +} + export function wait(ms: number): Promise { return new Promise(resolve => { setTimeout(resolve, ms); @@ -251,6 +255,14 @@ export function uniq(array: readonly string[]): string[] { }); } +export function allSameValue(xs: readonly T[]): T | undefined { + const sameValue: T | undefined = xs[0]; + for (let i = 1; i < xs.length; i++) { + if (sameValue !== xs[i]) return; + } + return sameValue; +} + // ---------------------------- export function formatEmpty(str: string): string { @@ -373,6 +385,14 @@ export function formatDurationHybrid(ms: NumberLike): string { } } +export function timezoneOffsetInMinutesToString(offsetInMinutes: number, padHour: boolean): string { + const sign = offsetInMinutes < 0 ? '-' : '+'; + const absOffset = Math.abs(offsetInMinutes); + const h = Math.floor(absOffset / 60); + const m = absOffset % 60; + return `${sign}${padHour ? pad2(h) : h}:${pad2(m)}`; +} + function pluralize(word: string): string { // Ignoring irregular plurals. if (/(s|x|z|ch|sh)$/.test(word)) { @@ -612,12 +632,10 @@ export function hashJoaat(str: string): number { return (hash & 4294967295) >>> 0; } -export function objectHash(obj: any): string { - return hashJoaat(JSONBig.stringify(obj)).toString(16).padStart(8); -} +export const OVERLAY_OPEN_SELECTOR = `.${Classes.PORTAL} .${Classes.OVERLAY_OPEN}`; -export function hasPopoverOpen(): boolean { - return Boolean(document.querySelector(`${Classes.PORTAL} ${Classes.OVERLAY} ${Classes.POPOVER}`)); +export function hasOverlayOpen(): boolean { + return Boolean(document.querySelector(OVERLAY_OPEN_SELECTOR)); } export function checkedCircleIcon(checked: boolean): IconName { diff --git a/web-console/src/utils/index.tsx b/web-console/src/utils/index.tsx index edea5ad0a520..096f0dfe0634 100644 --- a/web-console/src/utils/index.tsx +++ b/web-console/src/utils/index.tsx @@ -19,10 +19,12 @@ export * from './base64-url'; export * from './column-metadata'; export * from './date'; +export * from './date-floor-shift-ceil/date-floor-shift-ceil'; export * from './download'; export * from './download-query-detail-archive'; export * from './druid-lookup'; export * from './druid-query'; +export * from './duration/duration'; export * from './formatter'; export * from './general'; export * from './local-storage-backed-visibility'; diff --git a/web-console/src/utils/query-action.ts b/web-console/src/utils/query-action.ts index 45ade6961388..3f17f22b67d2 100644 --- a/web-console/src/utils/query-action.ts +++ b/web-console/src/utils/query-action.ts @@ -16,6 +16,6 @@ * limitations under the License. */ -import type { SqlQuery } from '@druid-toolkit/query'; +import type { SqlQuery } from 'druid-query-toolkit'; export type QueryAction = (query: SqlQuery) => SqlQuery; diff --git a/web-console/src/utils/query-manager/query-manager.ts b/web-console/src/utils/query-manager/query-manager.ts index 5b141c7c89db..1023b23412ad 100644 --- a/web-console/src/utils/query-manager/query-manager.ts +++ b/web-console/src/utils/query-manager/query-manager.ts @@ -288,7 +288,7 @@ export class QueryManager { return this.lastQuery; } - public getLastIntermediateQuery(): any { + public getLastIntermediateQuery(): unknown { return this.lastIntermediateQuery; } diff --git a/web-console/src/utils/sampler.ts b/web-console/src/utils/sampler.ts index 7fd3d67364cc..5333013c12b4 100644 --- a/web-console/src/utils/sampler.ts +++ b/web-console/src/utils/sampler.ts @@ -16,8 +16,8 @@ * limitations under the License. */ -import { dedupe, F, SqlExpression, SqlFunction } from '@druid-toolkit/query'; import type { CancelToken } from 'axios'; +import { dedupe, F, SqlExpression, SqlFunction } from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import type { diff --git a/web-console/src/utils/sql.spec.ts b/web-console/src/utils/sql.spec.ts index 9e5533ee4e62..57e7bce95fe8 100644 --- a/web-console/src/utils/sql.spec.ts +++ b/web-console/src/utils/sql.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import { sane } from '@druid-toolkit/query'; +import { sane } from 'druid-query-toolkit'; import { findAllSqlQueriesInText, findSqlQueryPrefix } from './sql'; diff --git a/web-console/src/utils/sql.ts b/web-console/src/utils/sql.ts index 1cdf243819d2..6c168002078a 100644 --- a/web-console/src/utils/sql.ts +++ b/web-console/src/utils/sql.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { SqlBase } from '@druid-toolkit/query'; +import type { SqlBase } from 'druid-query-toolkit'; import { SqlColumn, SqlExpression, @@ -24,7 +24,7 @@ import { SqlLiteral, SqlQuery, SqlStar, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import type { RowColumn } from './general'; import { offsetToRowColumn } from './general'; diff --git a/web-console/src/views/explore-view/models/stage.ts b/web-console/src/utils/stage.ts similarity index 80% rename from web-console/src/views/explore-view/models/stage.ts rename to web-console/src/utils/stage.ts index a1d01258f0a2..7b65cfebd9cb 100644 --- a/web-console/src/views/explore-view/models/stage.ts +++ b/web-console/src/utils/stage.ts @@ -16,6 +16,13 @@ * limitations under the License. */ +export interface Margin { + top: number; + right: number; + bottom: number; + left: number; +} + export class Stage { public readonly width: number; public readonly height: number; @@ -28,4 +35,11 @@ export class Stage { public equals(other: Stage | undefined): boolean { return Boolean(other && this.width === other.width && this.height === other.height); } + + public applyMargin(margin: Margin): Stage { + return new Stage( + this.width - margin.left - margin.right, + this.height - margin.top - margin.bottom, + ); + } } diff --git a/web-console/src/utils/table-helpers.ts b/web-console/src/utils/table-helpers.ts index 45e8758bf6f8..a4ba4fce7ddf 100644 --- a/web-console/src/utils/table-helpers.ts +++ b/web-console/src/utils/table-helpers.ts @@ -16,9 +16,10 @@ * limitations under the License. */ -import type { QueryResult, SqlExpression } from '@druid-toolkit/query'; -import { C } from '@druid-toolkit/query'; -import type { Filter } from 'react-table'; +import { ascending, descending, sort } from 'd3-array'; +import type { QueryResult, SqlExpression } from 'druid-query-toolkit'; +import { C } from 'druid-query-toolkit'; +import type { Filter, SortingRule } from 'react-table'; import { filterMap, formatNumber, isNumberLike, oneOf } from './general'; import { deepSet } from './object-change'; @@ -67,19 +68,24 @@ export function getNumericColumnBraces( return numericColumnBraces; } -export interface Sorted { - id: string; - desc: boolean; -} - export interface TableState { page: number; pageSize: number; filtered: Filter[]; - sorted: Sorted[]; + sorted: SortingRule[]; } -export function sortedToOrderByClause(sorted: Sorted[]): string | undefined { +export function sortedToOrderByClause(sorted: SortingRule[]): string | undefined { if (!sorted.length) return; return 'ORDER BY ' + sorted.map(sort => `${C(sort.id)} ${sort.desc ? 'DESC' : 'ASC'}`).join(', '); } + +export function applySorting(xs: any[], sorted: SortingRule[]): any[] { + const firstSortingRule = sorted[0]; + if (!firstSortingRule) return xs; + const { id, desc } = firstSortingRule; + return sort( + xs, + desc ? (d1, d2) => descending(d1[id], d2[id]) : (d1, d2) => ascending(d1[id], d2[id]), + ); +} diff --git a/web-console/src/utils/types.ts b/web-console/src/utils/types.ts index 7f48655cb1c4..a3e38a2c8218 100644 --- a/web-console/src/utils/types.ts +++ b/web-console/src/utils/types.ts @@ -18,7 +18,7 @@ import type { IconName } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { Column } from '@druid-toolkit/query'; +import type { Column } from 'druid-query-toolkit'; export function columnToSummary(column: Column): string { const lines: string[] = [column.name]; diff --git a/web-console/src/utils/values-query.spec.tsx b/web-console/src/utils/values-query.spec.tsx index 7bc093bc3e82..89f4a4d2f14f 100644 --- a/web-console/src/utils/values-query.spec.tsx +++ b/web-console/src/utils/values-query.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import { QueryResult, sane } from '@druid-toolkit/query'; +import { QueryResult, sane } from 'druid-query-toolkit'; import { queryResultToValuesQuery } from './values-query'; diff --git a/web-console/src/utils/values-query.tsx b/web-console/src/utils/values-query.tsx index 1b5e62b44c23..69130d380e75 100644 --- a/web-console/src/utils/values-query.tsx +++ b/web-console/src/utils/values-query.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import type { Column, QueryResult, SqlExpression } from '@druid-toolkit/query'; +import type { Column, QueryResult, SqlExpression } from 'druid-query-toolkit'; import { C, F, @@ -28,7 +28,7 @@ import { SqlRecord, SqlType, SqlValues, -} from '@druid-toolkit/query'; +} from 'druid-query-toolkit'; import * as JSONBig from 'json-bigint-native'; import { oneOf } from './general'; diff --git a/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap b/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap index 6da41f6eb763..5f471fa1e390 100644 --- a/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap +++ b/web-console/src/views/datasources-view/__snapshots__/datasources-view.spec.tsx.snap @@ -109,6 +109,7 @@ exports[`DatasourcesView matches snapshot 1`] = ` /> { + datasources = datasourcesResp.map((d: any): DatasourceQueryResultRow => { const totalDataSize = deepGet(d, 'properties.segments.size') || -1; const segmentsToLoad = Number(loadstatus[d.name] || 0); const availableSegments = Number(deepGet(d, 'properties.segments.count')); @@ -521,9 +531,10 @@ GROUP BY 1, 2`; if (capabilities.hasOverlordAccess()) { auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => { try { - const taskList = ( - await Api.instance.get(`/druid/indexer/v1/tasks?state=running`, { cancelToken }) - ).data; + const taskList = await getApiArray( + `/druid/indexer/v1/tasks?state=running`, + cancelToken, + ); const runningTasksByDatasource = groupByAsMap( taskList, @@ -562,10 +573,10 @@ GROUP BY 1, 2`; if (showUnused) { try { unused = ( - await Api.instance.get( + await getApiArray( '/druid/coordinator/v1/metadata/datasources?includeUnused', ) - ).data.filter(d => !seen[d]); + ).filter(d => !seen[d]); } catch { AppToaster.show({ icon: IconNames.ERROR, @@ -578,7 +589,7 @@ GROUP BY 1, 2`; // Rules auxiliaryQueries.push(async (datasourcesAndDefaultRules, cancelToken) => { try { - const rules: Record = ( + const rules = ( await Api.instance.get>('/druid/coordinator/v1/rules', { cancelToken, }) @@ -589,7 +600,7 @@ GROUP BY 1, 2`; ...ds, rules: rules[ds.datasource] || [], })), - defaultRules: rules[DEFAULT_RULES_KEY], + defaultRules: rules[RuleUtil.DEFAULT_RULES_KEY], }; } catch { AppToaster.show({ @@ -657,8 +668,19 @@ GROUP BY 1, 2`; } private readonly refresh = (auto: boolean): void => { - if (auto && hasPopoverOpen()) return; + if (auto && hasOverlayOpen()) return; this.datasourceQueryManager.rerunLastQuery(auto); + + const { showSegmentTimeline } = this.state; + if (showSegmentTimeline) { + // Create a new capabilities object to force the segment timeline to re-render + this.setState(({ showSegmentTimeline }) => ({ + showSegmentTimeline: { + ...showSegmentTimeline, + capabilities: this.props.capabilities.clone(), + }, + })); + } }; private fetchDatasourceData() { @@ -846,9 +868,9 @@ GROUP BY 1, 2`; { - if (!lastDatasourcesQuery) return; + if (typeof lastDatasourcesQuery !== 'string') return; goToQuery({ queryString: lastDatasourcesQuery }); }} /> @@ -921,6 +943,7 @@ GROUP BY 1, 2`; retentionDialogOpenOn: { datasource: '_default', rules: defaultRules, + defaultRules, }, }; }); @@ -1030,10 +1053,13 @@ GROUP BY 1, 2`; icon: IconNames.AUTOMATIC_UPDATES, title: 'Edit retention rules', onAction: () => { + const defaultRules = this.state.datasourcesAndDefaultRulesState.data?.defaultRules; + if (!defaultRules) return; this.setState({ retentionDialogOpenOn: { datasource, rules: rules || [], + defaultRules, }, }); }, @@ -1097,9 +1123,8 @@ GROUP BY 1, 2`; private renderRetentionDialog() { const { capabilities } = this.props; - const { retentionDialogOpenOn, datasourcesAndDefaultRulesState } = this.state; - const defaultRules = datasourcesAndDefaultRulesState.data?.defaultRules; - if (!retentionDialogOpenOn || !defaultRules) return; + const { retentionDialogOpenOn } = this.state; + if (!retentionDialogOpenOn) return; return ( this.setState({ retentionDialogOpenOn: undefined })} onSave={this.saveRules} /> @@ -1139,8 +1164,9 @@ GROUP BY 1, 2`; } private renderDatasourcesTable() { - const { goToSegments, goToTasks, capabilities, filters, onFiltersChange } = this.props; - const { datasourcesAndDefaultRulesState, showUnused, visibleColumns } = this.state; + const { goToTasks, capabilities, filters, onFiltersChange } = this.props; + const { datasourcesAndDefaultRulesState, showUnused, visibleColumns, showSegmentTimeline } = + this.state; let { datasources, defaultRules } = datasourcesAndDefaultRulesState.data || { datasources: [] }; @@ -1194,12 +1220,19 @@ GROUP BY 1, 2`; show: visibleColumns.shown('Datasource name'), accessor: 'datasource', width: 150, - Cell: row => ( + Cell: ({ value, original }) => ( this.onDetail(row.original)} + onClick={() => this.onDetail(original)} hoverIcon={IconNames.SEARCH_TEMPLATE} + tooltip="Show detail" > - {row.value} + {showSegmentTimeline ? ( + <> + {value} + + ) : ( + value + )} ), }, @@ -1225,7 +1258,12 @@ GROUP BY 1, 2`; const hasZeroReplicationRule = RuleUtil.hasZeroReplicaRule(rules, defaultRules); const descriptor = hasZeroReplicationRule ? 'pre-cached' : 'available'; const segmentsEl = ( - goToSegments(datasource)}> + + this.setState({ showSegmentTimeline: { capabilities, datasource } }) + } + data-tooltip="Show in segment timeline" + > {pluralIfNeeded(num_segments, 'segment')} ); @@ -1318,7 +1356,7 @@ GROUP BY 1, 2`; goToTasks(original.datasource)} hoverIcon={IconNames.ARROW_TOP_RIGHT} - title="Go to tasks" + tooltip="Go to tasks" > {formatRunningTasks(runningTasks)} @@ -1498,6 +1536,7 @@ GROUP BY 1, 2`; if (!compaction) return; return ( { if (!compaction) return; @@ -1614,6 +1653,7 @@ GROUP BY 1, 2`; return ( { if (!defaultRules) return; @@ -1621,6 +1661,7 @@ GROUP BY 1, 2`; retentionDialogOpenOn: { datasource, rules, + defaultRules, }, }); }} @@ -1668,7 +1709,7 @@ GROUP BY 1, 2`; } render() { - const { capabilities } = this.props; + const { capabilities, goToSegments } = this.props; const { showUnused, visibleColumns, @@ -1681,9 +1722,7 @@ GROUP BY 1, 2`;
{ - this.refresh(auto); - }} + onRefresh={this.refresh} localStorageKey={LocalStorageKeys.DATASOURCES_REFRESH_RATE} /> {this.renderBulkDatasourceActions()} @@ -1694,9 +1733,13 @@ GROUP BY 1, 2`; disabled={!capabilities.hasCoordinatorAccess()} /> this.setState({ showSegmentTimeline: !showSegmentTimeline })} + onChange={() => + this.setState({ + showSegmentTimeline: showSegmentTimeline ? undefined : { capabilities }, + }) + } disabled={!capabilities.hasSqlOrCoordinatorAccess()} /> - {showSegmentTimeline && } + {showSegmentTimeline && ( + { + return ( +