From 814b4eccda48d456a6b5346ebb2e72123b74231a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 26 Feb 2026 10:04:59 -0800 Subject: [PATCH 01/12] Add support for MSQ CLUSTERED BY expressions to be preserved in the segment shard spec as virtual columns changes: * `ShardSpec` interface has a new method, `getDomainVirtualColumns` to provide the virtual column information for pruning * `DimensionRangeShardSpec` stores `VirtualColumns` in segment metadata so they can be compared to query expressions and be used for pruning * `FilterSegmentPruner` is virtual column aware for segment pruning using the new methods * `ClusterBy` now contains a map of key column to `VirtualColumn` alongside key columns, to support key columns being virtual columns * `ControllerImpl` persists clustering virtual columns in compaction state in the transform spec * `MSQCompactionRunner` handles virtual columns in order-by/cluster-by for compaction --- .../ExpressionSelectorBenchmark.java | 36 ++- .../ExpressionVectorSelectorBenchmark.java | 13 +- .../benchmark/JoinAndLookupBenchmark.java | 24 +- .../DimensionRangeShardSpecBenchmark.java | 4 + .../compact/CompactionSupervisorTest.java | 216 ++++++++++++++++++ .../embedded/msq/MultiStageQueryTest.java | 206 ++++++++++++++++- .../segment/MapVirtualColumnGroupByTest.java | 12 +- .../segment/MapVirtualColumnTopNTest.java | 8 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../task/NativeCompactionRunnerTest.java | 13 +- .../apache/druid/msq/exec/ControllerImpl.java | 58 +++-- .../msq/indexing/MSQCompactionRunner.java | 182 ++++++++++----- .../druid/msq/logical/stages/JoinStage.java | 3 +- .../druid/msq/querykit/QueryKitUtils.java | 2 +- .../msq/querykit/groupby/GroupByQueryKit.java | 18 +- .../druid/msq/querykit/scan/ScanQueryKit.java | 18 +- .../DartTableInputSpecSlicerTest.java | 4 + .../apache/druid/msq/exec/MSQReplaceTest.java | 6 +- .../table/DataSegmentWithLocationTest.java | 13 +- .../IndexerTableInputSpecSlicerTest.java | 8 +- .../org/apache/druid/frame/key/ClusterBy.java | 31 ++- .../FrameChannelHashPartitioner.java | 2 +- .../java/org/apache/druid/query/Druids.java | 6 +- .../query/filter/FilterSegmentPruner.java | 34 ++- .../druid/query/groupby/GroupByQuery.java | 10 +- .../groupby/orderby/DefaultLimitSpec.java | 4 + .../druid/query/planning/ExecutionVertex.java | 3 +- .../druid/query/topn/TopNQueryBuilder.java | 2 +- .../druid/segment/UnnestCursorFactory.java | 2 +- .../apache/druid/segment/VirtualColumns.java | 3 +- .../BaseDimensionRangeShardSpec.java | 4 + .../BuildingDimensionRangeShardSpec.java | 2 + .../DimensionRangeBucketShardSpec.java | 3 +- .../partition/DimensionRangeShardSpec.java | 22 +- .../druid/timeline/partition/ShardSpec.java | 7 + .../SingleDimensionRangeBucketShardSpec.java | 2 + .../partition/SingleDimensionShardSpec.java | 2 + .../druid/frame/testutil/FrameTestUtil.java | 3 +- .../query/filter/FilterSegmentPrunerTest.java | 60 ++++- .../operator/ScanOperatorFactoryTest.java | 4 +- .../search/SearchQueryQueryToolChestTest.java | 12 +- .../segment/RowBasedCursorFactoryTest.java | 12 +- .../org/apache/druid/segment/TestIndex.java | 5 +- .../druid/segment/VirtualColumnsTest.java | 28 +-- .../druid/segment/filter/BaseFilterTest.java | 113 ++++++--- .../HashJoinSegmentCursorFactoryTest.java | 9 +- .../segment/join/JoinFilterAnalyzerTest.java | 48 ++-- .../NestedFieldColumnSelectorsTest.java | 19 +- .../CompactionTransformSpecTest.java | 13 +- .../ExpressionVectorSelectorsCastTest.java | 13 +- .../ExpressionVectorSelectorsTest.java | 12 +- ...ListFilteredVirtualColumnSelectorTest.java | 7 +- .../VirtualizedColumnSelectorFactoryTest.java | 7 +- .../druid/timeline/DataSegmentTest.java | 48 ++-- .../BuildingDimensionRangeShardSpecTest.java | 2 + .../DimensionRangeShardSpecTest.java | 34 ++- .../PartitionHolderCompletenessTest.java | 7 + .../indexing/ClientCompactionRunnerInfo.java | 48 +++- ...exerSQLMetadataStorageCoordinatorTest.java | 4 +- .../ReindexingDeletionRuleTest.java | 13 +- .../coordination/LoadableDataSegmentTest.java | 13 +- ...eSchemaDataSourceCompactionConfigTest.java | 12 +- .../server/http/DataSegmentPlusTest.java | 12 +- .../sql/calcite/CalciteArraysQueryTest.java | 6 +- 64 files changed, 1131 insertions(+), 419 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java index 9cd89cdfaac0..1dc942e51c2f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -499,25 +499,23 @@ public void caseSearchedWithLookup2(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns( VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "ll", - "lookup(s, 'lookyloo')", - ColumnType.STRING, - LookupEnabledTestExprMacroTable.INSTANCE - ), - new ExpressionVirtualColumn( - "v", - "case_searched(n == 1001, -1, " - + "ll == 'asd1', 1, " - + "ll == 'asd2', 2, " - + "ll == 'asd3', 3, " - + "ll == 'asd4', 4, " - + "ll == 'asd5', 5, " - + "-2)", - ColumnType.LONG, - LookupEnabledTestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "ll", + "lookup(s, 'lookyloo')", + ColumnType.STRING, + LookupEnabledTestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "v", + "case_searched(n == 1001, -1, " + + "ll == 'asd1', 1, " + + "ll == 'asd2', 2, " + + "ll == 'asd3', 3, " + + "ll == 'asd4', 4, " + + "ll == 'asd5', 5, " + + "-2)", + ColumnType.LONG, + LookupEnabledTestExprMacroTable.INSTANCE ) ) ) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java index 5071558e4fad..586a61c67bd7 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -19,7 +19,6 @@ package org.apache.druid.benchmark; -import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; @@ -137,13 +136,11 @@ public void tearDown() throws Exception public void scan(Blackhole blackhole) { final VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - expression, - ExpressionType.toColumnType(outputType), - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "v", + expression, + ExpressionType.toColumnType(outputType), + TestExprMacroTable.INSTANCE ) ); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java index 03278c453e93..9ec046e698ff 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -317,19 +317,17 @@ public String getCanonicalLookupName(String lookupName) ); lookupVirtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - LOOKUP_COUNTRY_CODE_TO_NAME, - "lookup(countryIsoCode, '" + LOOKUP_COUNTRY_CODE_TO_NAME + "')", - ColumnType.STRING, - exprMacroTable - ), - new ExpressionVirtualColumn( - LOOKUP_COUNTRY_NUMBER_TO_NAME, - "lookup(countryNumber, '" + LOOKUP_COUNTRY_NUMBER_TO_NAME + "')", - ColumnType.STRING, - exprMacroTable - ) + new ExpressionVirtualColumn( + LOOKUP_COUNTRY_CODE_TO_NAME, + "lookup(countryIsoCode, '" + LOOKUP_COUNTRY_CODE_TO_NAME + "')", + ColumnType.STRING, + exprMacroTable + ), + new ExpressionVirtualColumn( + LOOKUP_COUNTRY_NUMBER_TO_NAME, + "lookup(countryNumber, '" + LOOKUP_COUNTRY_NUMBER_TO_NAME + "')", + ColumnType.STRING, + exprMacroTable ) ); } diff --git a/benchmarks/src/test/java/org/apache/druid/timeline/DimensionRangeShardSpecBenchmark.java b/benchmarks/src/test/java/org/apache/druid/timeline/DimensionRangeShardSpecBenchmark.java index 9ea11e48eef8..df1bd027a9af 100644 --- a/benchmarks/src/test/java/org/apache/druid/timeline/DimensionRangeShardSpecBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/timeline/DimensionRangeShardSpecBenchmark.java @@ -26,6 +26,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.junit.Assert; import org.openjdk.jmh.annotations.Benchmark; @@ -73,6 +74,7 @@ public class DimensionRangeShardSpecBenchmark // Initial segment (null -> values) private final DimensionRangeShardSpec shardSpec0 = new DimensionRangeShardSpec( Arrays.asList("country", "city"), + VirtualColumns.EMPTY, new StringTuple(new String[]{null, null}), new StringTuple(new String[]{"Germany", "Munich"}), 0, @@ -82,6 +84,7 @@ public class DimensionRangeShardSpecBenchmark // Middle segment (values -> other values) private final DimensionRangeShardSpec shardSpec1 = new DimensionRangeShardSpec( Arrays.asList("country", "city"), + VirtualColumns.EMPTY, new StringTuple(new String[]{"Germany", "Munich"}), new StringTuple(new String[]{"United States", "New York"}), 1, @@ -91,6 +94,7 @@ public class DimensionRangeShardSpecBenchmark // End segment (values -> null) private final DimensionRangeShardSpec shardSpec2 = new DimensionRangeShardSpec( Arrays.asList("country", "city"), + VirtualColumns.EMPTY, new StringTuple(new String[]{"United States", "New York"}), new StringTuple(new String[]{null, null}), 2, diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index e09be28eb517..8a08725cec24 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -23,6 +23,7 @@ import org.apache.druid.catalog.guice.CatalogClientModule; import org.apache.druid.catalog.guice.CatalogCoordinatorModule; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.StringTuple; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -73,6 +74,8 @@ import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.DateTime; @@ -428,6 +431,219 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec verifyNoRowsWithNestedValue("extraInfo", "fieldA", "valueA"); } + + @Test + public void test_compaction_cluster_by_virtualcolumn() + { + // Virtual Columns on nested data is only supported with MSQ compaction engine right now. + CompactionEngine compactionEngine = CompactionEngine.MSQ; + configureCompaction(compactionEngine); + + String jsonDataWithNestedColumn = + """ + {"timestamp": "2023-01-01T00:00:00", "str":"a", "obj":{"a": "ll"}} + {"timestamp": "2023-01-01T00:00:00", "str":"", "obj":{"a": "mm"}} + {"timestamp": "2023-01-01T00:00:00", "str":"null", "obj":{"a": "nn"}} + {"timestamp": "2023-01-01T00:00:00", "str":"b", "obj":{"a": "oo"}} + {"timestamp": "2023-01-01T00:00:00", "str":"c", "obj":{"a": "pp"}} + {"timestamp": "2023-01-01T00:00:00", "str":"d", "obj":{"a": "qq"}} + {"timestamp": "2023-01-01T00:00:00", "str":null, "obj":{"a": "rr"}} + """; + + final TaskBuilder.Index task = TaskBuilder + .ofTypeIndex() + .dataSource(dataSource) + .jsonInputFormat() + .inlineInputSourceWithData(jsonDataWithNestedColumn) + .isoTimestampColumn("timestamp") + .schemaDiscovery() + .granularitySpec("DAY", null, false); + + cluster.callApi().runTask(task.withId(IdUtils.getRandomId()), overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + Assertions.assertEquals(7, getTotalRowCount()); + + VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v0", "json_value(obj, '$.a')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + + InlineSchemaDataSourceCompactionConfig config = + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withTransformSpec( + new CompactionTransformSpec( + null, + virtualColumns + ) + ) + .withTuningConfig( + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + null, + new DimensionRangePartitionsSpec(4, null, List.of("v0"), false), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ) + .build(); + + runCompactionWithSpec(config); + waitForAllCompactionTasksToFinish(); + + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + List segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord).stream().toList(); + Assertions.assertEquals(2, segments.size()); + Assertions.assertEquals( + new DimensionRangeShardSpec( + List.of("v0"), + virtualColumns, + null, + StringTuple.create("oo"), + 0, + 2 + ), + segments.get(0).getShardSpec() + ); + Assertions.assertEquals( + new DimensionRangeShardSpec( + List.of("v0"), + virtualColumns, + StringTuple.create("oo"), + null, + 1, + 2 + ), + segments.get(1).getShardSpec() + ); + } + + @Test + public void test_compaction_cluster_by_virtualcolumn_rollup() + { + // Virtual Columns on nested data is only supported with MSQ compaction engine right now. + CompactionEngine compactionEngine = CompactionEngine.MSQ; + configureCompaction(compactionEngine); + + String jsonDataWithNestedColumn = + """ + {"timestamp": "2023-01-01T00:00:00", "str":"a", "obj":{"a": "ll"}} + {"timestamp": "2023-01-01T00:00:00", "str":"", "obj":{"a": "mm"}} + {"timestamp": "2023-01-01T00:00:00", "str":"null", "obj":{"a": "nn"}} + {"timestamp": "2023-01-01T00:00:00", "str":"b", "obj":{"a": "oo"}} + {"timestamp": "2023-01-01T00:00:00", "str":"c", "obj":{"a": "pp"}} + {"timestamp": "2023-01-01T00:00:00", "str":"d", "obj":{"a": "qq"}} + {"timestamp": "2023-01-01T00:00:00", "str":null, "obj":{"a": "rr"}} + """; + + final TaskBuilder.Index task = TaskBuilder + .ofTypeIndex() + .dataSource(dataSource) + .jsonInputFormat() + .inlineInputSourceWithData(jsonDataWithNestedColumn) + .isoTimestampColumn("timestamp") + .schemaDiscovery() + .granularitySpec("DAY", "MINUTE", true); + + cluster.callApi().runTask(task.withId(IdUtils.getRandomId()), overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + Assertions.assertEquals(7, getTotalRowCount()); + + VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn( + "v0", + "json_value(obj, '$.a')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ); + + InlineSchemaDataSourceCompactionConfig config = + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withTransformSpec( + new CompactionTransformSpec( + null, + virtualColumns + ) + ) + .withTuningConfig( + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + null, + new DimensionRangePartitionsSpec(4, null, List.of("v0"), false), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ) + .build(); + + runCompactionWithSpec(config); + waitForAllCompactionTasksToFinish(); + + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + + List segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord).stream().toList(); + Assertions.assertEquals(2, segments.size()); + Assertions.assertEquals( + new DimensionRangeShardSpec( + List.of("v0"), + virtualColumns, + null, + StringTuple.create("oo"), + 0, + 2 + ), + segments.get(0).getShardSpec() + ); + Assertions.assertEquals( + new DimensionRangeShardSpec( + List.of("v0"), + virtualColumns, + StringTuple.create("oo"), + null, + 1, + 2 + ), + segments.get(1).getShardSpec() + ); + } + /** * Tests that when a compaction task filters out all rows using a transform spec, * tombstones are created to properly drop the old segments. This test covers both diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/MultiStageQueryTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/MultiStageQueryTest.java index d268bb8b897f..1d4ec613e6c3 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/MultiStageQueryTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/MultiStageQueryTest.java @@ -19,10 +19,17 @@ package org.apache.druid.testing.embedded.msq; +import org.apache.druid.data.input.StringTuple; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.http.SqlTaskStatus; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.sql.http.GetQueryReportResponse; import org.apache.druid.testing.embedded.EmbeddedBroker; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; @@ -32,6 +39,8 @@ import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.indexing.Resources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -40,10 +49,15 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.UUID; public class MultiStageQueryTest extends EmbeddedClusterTestBase { - private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedBroker broker = + new EmbeddedBroker().setServerMemory(200_000_000) + .addProperty("druid.msq.dart.controller.maxRetainedReportCount", "10") + .addProperty("druid.query.default.context.maxConcurrentStages", "1") + .addProperty("druid.sql.planner.enableSysQueriesTable", "true"); private final EmbeddedOverlord overlord = new EmbeddedOverlord(); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() @@ -59,6 +73,7 @@ protected EmbeddedDruidCluster createCluster() return EmbeddedDruidCluster .withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() + .addCommonProperty("druid.msq.dart.enabled", "true") .addResource(exportDirectory) .addServer(overlord) .addServer(coordinator) @@ -152,4 +167,193 @@ public void testExport() actualResults ); } + + @Test + public void testClusterByVirtualColumn() + { + final String sqlTemplate = + """ + SET rowsPerSegment = 2; + SET groupByEnableMultiValueUnnesting = FALSE; + REPLACE INTO %s OVERWRITE ALL + WITH "ext" AS ( + SELECT * + FROM TABLE(EXTERN('{"type":"local","files":["%s"]}', '{"type":"json"}')) + EXTEND( + "timestamp" VARCHAR, + "added" BIGINT, + "delta" BIGINT, + "deleted" BIGINT, + "page" VARCHAR, + "city" VARCHAR, + "country" VARCHAR, + "user" VARCHAR + ) + ) + SELECT + TIME_PARSE("timestamp") AS __time, + added, + delta, + deleted, + page, + city, + country, + user + FROM "ext" + PARTITIONED BY DAY + CLUSTERED BY CONCAT(country, ':', city) + """; + final String sql = StringUtils.format( + sqlTemplate, + dataSource, + Resources.DataFile.tinyWiki1Json().getAbsolutePath() + ); + + final SqlTaskStatus taskStatus = msqApis.submitTaskSql(sql); + cluster.callApi().waitForTaskToSucceed(taskStatus.getTaskId(), overlord.latchableEmitter()); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + assertClusterByVirtualColumnSegments(); + assertClusterByVirtualColumnQueries(); + } + + @Test + public void testClusterByVirtualColumnRollup() + { + final String sqlTemplate = + """ + SET rowsPerSegment = 2; + SET groupByEnableMultiValueUnnesting = FALSE; + REPLACE INTO %s OVERWRITE ALL + WITH "ext" AS ( + SELECT * + FROM TABLE(EXTERN('{"type":"local","files":["%s"]}', '{"type":"json"}')) + EXTEND( + "timestamp" VARCHAR, + "added" BIGINT, + "delta" BIGINT, + "deleted" BIGINT, + "page" VARCHAR, + "city" VARCHAR, + "country" VARCHAR, + "user" VARCHAR + ) + ) + SELECT + TIME_PARSE("timestamp") AS __time, + page, + city, + country, + user, + SUM(added) as added, + SUM(delta) as delta, + SUM(deleted) as deleted + FROM "ext" + GROUP BY TIME_PARSE("timestamp"), page, city, country, user, CONCAT(country, ':', city) + PARTITIONED BY DAY + CLUSTERED BY CONCAT(country, ':', city) + """; + final String sql = StringUtils.format( + sqlTemplate, + dataSource, + Resources.DataFile.tinyWiki1Json().getAbsolutePath() + ); + + final SqlTaskStatus taskStatus = msqApis.submitTaskSql(sql); + cluster.callApi().waitForTaskToSucceed(taskStatus.getTaskId(), overlord.latchableEmitter()); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + assertClusterByVirtualColumnSegments(); + assertClusterByVirtualColumnQueries(); + } + + private void assertClusterByVirtualColumnSegments() + { + List segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord).stream().toList(); + Assertions.assertEquals(2, segments.size()); + VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v1", "concat(\"country\",':',\"city\")", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + Assertions.assertEquals( + new DimensionRangeShardSpec( + List.of("v1"), + virtualColumns, + null, + StringTuple.create("Russia:Moscow"), + 0, + 2 + ), + segments.get(0).getShardSpec() + ); + Assertions.assertEquals( + new DimensionRangeShardSpec( + List.of("v1"), + virtualColumns, + StringTuple.create("Russia:Moscow"), + null, + 1, + 2 + ), + segments.get(1).getShardSpec() + ); + } + + private void assertClusterByVirtualColumnQueries() + { + String queryId = UUID.randomUUID().toString(); + cluster.callApi().verifySqlQuery( + "SET engine = 'msq-dart'; SET sqlQueryId = '" + queryId + "'; SELECT __time, country, city, page FROM %s ORDER BY __time", + dataSource, + """ + 2013-08-31T01:02:33.000Z,United States,San Francisco,Gypsy Danger + 2013-08-31T03:32:45.000Z,Australia,Syndey,Striker Eureka + 2013-08-31T07:11:21.000Z,Russia,Moscow,Cherno Alpha""" + ); + Assertions.assertEquals(2, getSegmentsScannedForDartQuery(queryId)); + + queryId = UUID.randomUUID().toString(); + cluster.callApi().verifySqlQuery( + "SET engine = 'msq-dart'; SET sqlQueryId = '" + queryId + "'; SELECT __time, country, city, page FROM %s WHERE CONCAT(country, ':', city) <= 'Russia' ORDER BY __time", + dataSource, + """ + 2013-08-31T03:32:45.000Z,Australia,Syndey,Striker Eureka""" + ); + Assertions.assertEquals(1, getSegmentsScannedForDartQuery(queryId)); + + queryId = UUID.randomUUID().toString(); + cluster.callApi().verifySqlQuery( + "SET engine = 'msq-dart'; SET sqlQueryId = '" + queryId + "'; SELECT __time, country, city, page FROM %s WHERE CONCAT(country, ':', city) <= 'Russia:St. Petersburg' ORDER BY __time", + dataSource, + """ + 2013-08-31T03:32:45.000Z,Australia,Syndey,Striker Eureka + 2013-08-31T07:11:21.000Z,Russia,Moscow,Cherno Alpha""" + ); + Assertions.assertEquals(2, getSegmentsScannedForDartQuery(queryId)); + } + + private long getSegmentsScannedForDartQuery(String sqlQueryId) + { + ChannelCounters.Snapshot segmentChannelCounters = getDartSegmentChannelCounters(sqlQueryId); + return segmentChannelCounters.getFiles()[0]; + } + + private ChannelCounters.Snapshot getDartSegmentChannelCounters(String sqlQueryId) + { + final GetQueryReportResponse reportResponse = msqApis.getDartQueryReport(sqlQueryId, broker); + + Assertions.assertNotNull(reportResponse, "Report response should not be null"); + ChannelCounters.Snapshot segmentChannelCounters = + (ChannelCounters.Snapshot) reportResponse.getReportMap() + .findReport("multiStageQuery") + .map(r -> + ((MSQTaskReportPayload) r.getPayload()).getCounters() + .snapshotForStage(0) + .get(0) + .getMap() + .get("input0") + ).orElse(null); + + Assertions.assertNotNull(segmentChannelCounters); + return segmentChannelCounters; + } } diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java index 46248da5a7ec..695d9964f7ca 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java @@ -134,7 +134,7 @@ public void testWithMapColumn() final GroupByQuery query = new GroupByQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), - VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + VirtualColumns.create(new MapVirtualColumn("keys", "values", "params")), null, Granularities.ALL, ImmutableList.of(new DefaultDimensionSpec("params", "params")), @@ -160,7 +160,7 @@ public void testWithSubColumn() final GroupByQuery query = new GroupByQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), - VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + VirtualColumns.create(new MapVirtualColumn("keys", "values", "params")), null, Granularities.ALL, ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), @@ -190,7 +190,7 @@ public void testWithSubColumnWithFilter() final GroupByQuery query = new GroupByQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), - VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + VirtualColumns.create(new MapVirtualColumn("keys", "values", "params")), new EqualityFilter("params.key3", ColumnType.STRING, "value3", null), Granularities.ALL, ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), @@ -219,7 +219,7 @@ public void testWithSubColumnWithPredicateFilter() final GroupByQuery query = new GroupByQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), - VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + VirtualColumns.create(new MapVirtualColumn("keys", "values", "params")), new InDimFilter("params.key3", ImmutableList.of("value1", "value3"), null), Granularities.ALL, ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), @@ -248,7 +248,7 @@ public void testWithSubColumnWithNotFilter() final GroupByQuery query = new GroupByQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), - VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + VirtualColumns.create(new MapVirtualColumn("keys", "values", "params")), NotDimFilter.of(new EqualityFilter("params.key3", ColumnType.STRING, "value3", null)), Granularities.ALL, ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), @@ -272,7 +272,7 @@ public void testWithSubColumnWithNotPredicateFilter() final GroupByQuery query = new GroupByQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), - VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + VirtualColumns.create(new MapVirtualColumn("keys", "values", "params")), NotDimFilter.of(new InDimFilter("params.key3", ImmutableList.of("value1", "value3"), null)), Granularities.ALL, ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java index 5e3c549a59d4..4cbd0fdd43b8 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java @@ -85,9 +85,7 @@ public void testWithMapColumn() final TopNQuery query = new TopNQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), VirtualColumns.create( - ImmutableList.of( - new MapVirtualColumn("keys", "values", "params") - ) + new MapVirtualColumn("keys", "values", "params") ), new DefaultDimensionSpec("params", "params"), // params is the map type new NumericTopNMetricSpec("count"), @@ -111,9 +109,7 @@ public void testWithSubColumn() final TopNQuery query = new TopNQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), VirtualColumns.create( - ImmutableList.of( - new MapVirtualColumn("keys", "values", "params") - ) + new MapVirtualColumn("keys", "values", "params") ), new DefaultDimensionSpec("params.key3", "params.key3"), // params.key3 is string new NumericTopNMetricSpec("count"), diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index 3523eaf9ef21..3ff298caad24 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -45,6 +45,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; @@ -687,7 +688,7 @@ static DimensionRangeShardSpec createShardSpec( numCorePartitions ); } else { - return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, numCorePartitions); + return new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, start, end, partitionNum, numCorePartitions); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java index dcb801ab4ecf..a9bbafc0eece 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task; -import com.google.common.collect.ImmutableList; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.expression.TestExprMacroTable; @@ -47,13 +46,11 @@ public class NativeCompactionRunnerTest public void testVirtualColumnsInTransformSpecAreNotSupported() { VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "extractedField", - "json_value(metadata, '$.category')", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "extractedField", + "json_value(metadata, '$.category')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE ) ); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 78d021dca6c2..2d476b2b3de8 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -174,6 +174,8 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.rowsandcols.serde.WireTransferableContext; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; @@ -1314,8 +1316,14 @@ private List generateSegmentIdsWithShardSpecsForReplace( segmentNumber == ranges.size() - 1 ? null : makeStringTuple(clusterBy, keyReader, range.getEnd(), shardColumns.size()); - - shardSpec = new DimensionRangeShardSpec(shardColumns, start, end, segmentNumber, ranges.size()); + shardSpec = new DimensionRangeShardSpec( + shardColumns, + VirtualColumns.create(clusterBy.getVirtualColumnMap().values()), + start, + end, + segmentNumber, + ranges.size() + ); } retVal[partitionNumber] = new SegmentIdWithShardSpec(destination.getDataSource(), interval, version, shardSpec); @@ -1824,9 +1832,20 @@ private static Function, Set> addCompactionStateTo ); DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); - CompactionTransformSpec transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) - ? null - : CompactionTransformSpec.of(dataSchema.getTransformSpec()); + + // if the clustered by requires virtual columns, preserve them here so that we can rebuild during compaction + CompactionTransformSpec transformSpec; + if (clusterBy == null || clusterBy.getVirtualColumnMap().isEmpty()) { + transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) + ? null + : CompactionTransformSpec.of(dataSchema.getTransformSpec()); + } else { + transformSpec = new CompactionTransformSpec( + dataSchema.getTransformSpec().getFilter(), + VirtualColumns.create(clusterBy.getVirtualColumnMap().values()) + ); + } + List metricsSpec = buildMSQCompactionMetrics(querySpec, dataSchema); IndexSpec indexSpec = tuningConfig.getIndexSpec(); @@ -1959,19 +1978,24 @@ private static Pair, String> computeShardColumns( ); } - // DimensionRangeShardSpec only handles columns that appear as-is in the output. + // DimensionRangeShardSpec columns may either be explicitly in the table or defined as virtual columns if (outputColumns.isEmpty()) { - return Pair.of( - shardColumns, - StringUtils.format( - "Using only[%d] CLUSTERED BY columns for 'range' shard specs, since the next column was not mapped to " - + "an output column.", - shardColumns.size() - ) - ); + final VirtualColumn vc = clusterBy.getVirtualColumnMap().get(column.columnName()); + if (vc != null) { + shardColumns.add(vc.getOutputName()); + } else { + return Pair.of( + shardColumns, + StringUtils.format( + "Using only[%d] CLUSTERED BY columns for 'range' shard specs, since the next column was not mapped to " + + "an output column or virtual column.", + shardColumns.size() + ) + ); + } + } else { + shardColumns.add(columnMappings.getOutputColumnName(outputColumns.getInt(0))); } - - shardColumns.add(columnMappings.getOutputColumnName(outputColumns.getInt(0))); } return Pair.of(shardColumns, "Using 'range' shard specs with all CLUSTERED BY fields."); @@ -1997,7 +2021,7 @@ private static StringTuple makeStringTuple( final int shardFieldCount ) { - final String[] array = new String[clusterBy.getColumns().size() - clusterBy.getBucketByCount()]; + final String[] array = new String[shardFieldCount]; for (int i = 0; i < shardFieldCount; i++) { final Object val = keyReader.read(key, clusterBy.getBucketByCount() + i); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index e65a1994f4aa..36853137092e 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -30,6 +30,7 @@ import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -66,8 +67,10 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.policy.PolicyEnforcer; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -75,6 +78,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.segment.virtual.VirtualizedColumnInspector; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizationResult; @@ -171,7 +175,10 @@ public CompactionConfigValidationResult validateCompactionTask( validationResults.add( ClientCompactionRunnerInfo.validatePartitionsSpecForMSQ( compactionTask.getTuningConfig().getPartitionsSpec(), - dataSchema.getDimensionsSpec().getDimensions() + dataSchema.getDimensionsSpec().getDimensions(), + compactionTask.getTransformSpec() == null + ? VirtualColumns.EMPTY + : compactionTask.getTransformSpec().getVirtualColumns() ) ); validationResults.add( @@ -407,42 +414,72 @@ private DataSource getInputDataSource(String name) private static List getAggregateDimensions( DataSchema dataSchema, - Map inputColToVirtualCol + Map inputColToVirtualCol, + List orderBy ) { - List dimensionSpecs = new ArrayList<>(); + List dimensions = new ArrayList<>(); + + // build a RowSignature of non-virtual column dimensions of the dataschema to use to resolve virtual column types + RowSignature.Builder baseBuilder = RowSignature.builder().addTimeColumn(); + for (DimensionSchema schema : dataSchema.getDimensionsSpec().getDimensions()) { + if (inputColToVirtualCol.containsKey(schema.getName())) { + continue; + } + baseBuilder.add(schema.getName(), schema.getColumnType()); + } + final RowSignature baseSignature = baseBuilder.build(); + // and virtualized inspector from base signature + final ColumnInspector inspector = new VirtualizedColumnInspector( + baseSignature, + VirtualColumns.create(inputColToVirtualCol.values()) + ); - // if schema is not time-sorted, the time column mapping would already be in inputColToVirtualCol - if (!dataSchema.getDimensionsSpec().getDimensionNames().contains(ColumnHolder.TIME_COLUMN_NAME)) { + // if schema is not time-sorted, the time column will be in the dimensions list, otherwise add time dimension first + if (dataSchema.getDimensionsSpec().getSchema(ColumnHolder.TIME_COLUMN_NAME) == null) { if (isQueryGranularityEmptyOrNone(dataSchema)) { // Dimensions in group-by aren't allowed to have time column name as the output name. - dimensionSpecs.add(new DefaultDimensionSpec( - ColumnHolder.TIME_COLUMN_NAME, - TIME_VIRTUAL_COLUMN, - ColumnType.LONG - )); + dimensions.add(new DefaultDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); } else { // The changed granularity would result in a new virtual column that needs to be aggregated upon. - dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + dimensions.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG)); + } + } + + // If dimensions point to virtual columns, replace dimension columns names with virtual column names. + for (DimensionSchema schema : dataSchema.getDimensionsSpec().getDimensions()) { + String dimension = schema.getName(); + ColumnType colType = schema.getColumnType(); + VirtualColumn vc = inputColToVirtualCol.get(dimension); + if (vc != null) { + if (vc instanceof ExpressionVirtualColumn) { + colType = ((ExpressionVirtualColumn) vc).getOutputType(); + } else { + colType = ColumnType.fromCapabilities(vc.capabilities(inspector, vc.getOutputName())); + } + } + dimensions.add(new DefaultDimensionSpec(dimension, dimension, colType)); + } + + // if any orderby columns refer to a virtual column that was not explicitly a dimension, add it to the list + // this is not really optimal, but it works without requiring any conversion between virtualcolumns and + // postaggregators which doesn't really exist here + for (OrderByColumnSpec order : orderBy) { + if (dataSchema.getDimensionsSpec().getSchema(order.getDimension()) != null) { + continue; + } + VirtualColumn vc = inputColToVirtualCol.get(order.getDimension()); + if (vc != null) { + dimensions.add( + new DefaultDimensionSpec( + vc.getOutputName(), + order.getDimension(), + ColumnType.fromCapabilities(vc.capabilities(baseSignature, vc.getOutputName())) + ) + ); } } - // If virtual columns are created from dimensions, replace dimension columns names with virtual column names. - dimensionSpecs.addAll( - dataSchema.getDimensionsSpec().getDimensions().stream() - .map(dim -> { - String dimension = dim.getName(); - ColumnType colType = dim.getColumnType(); - if (inputColToVirtualCol.containsKey(dim.getName())) { - VirtualColumn virtualColumn = inputColToVirtualCol.get(dimension); - dimension = virtualColumn.getOutputName(); - if (virtualColumn instanceof ExpressionVirtualColumn) { - colType = ((ExpressionVirtualColumn) virtualColumn).getOutputType(); - } - } - return new DefaultDimensionSpec(dimension, dimension, colType); - }) - .collect(Collectors.toList())); - return dimensionSpecs; + return dimensions; } private static ColumnMappings getColumnMappings(DataSchema dataSchema) @@ -467,11 +504,13 @@ private static ColumnMappings getColumnMappings(DataSchema dataSchema) .map(dim -> dim.getName().equals(ColumnHolder.TIME_COLUMN_NAME) ? timeColumnMapping : new ColumnMapping(dim.getName(), dim.getName())) - .collect(Collectors.toList()) + .toList() + ); + columnMappings.addAll( + Arrays.stream(dataSchema.getAggregators()) + .map(agg -> new ColumnMapping(agg.getName(), agg.getName())) + .toList() ); - columnMappings.addAll(Arrays.stream(dataSchema.getAggregators()) - .map(agg -> new ColumnMapping(agg.getName(), agg.getName())) - .collect(Collectors.toList())); return new ColumnMappings(columnMappings); } @@ -506,31 +545,44 @@ private Query buildScanQuery( Map inputColToVirtualCol ) { - RowSignature rowSignature = getRowSignature(dataSchema); - VirtualColumns virtualColumns = VirtualColumns.create(new ArrayList<>(inputColToVirtualCol.values())); + RowSignature baseRowSignature = getRowSignature(dataSchema); + final List columns = new ArrayList<>(baseRowSignature.getColumnNames()); + final List orderBys; + + RowSignature.Builder rowSignatureWithOrderByBuilder = RowSignature.builder().addAll(baseRowSignature); + + // when clustering by a virtual column, we might need to add the virtual column to columns list and row signature + if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { + List orderByColumnSpecs = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()); + orderBys = new ArrayList<>(); + for (OrderByColumnSpec spec : orderByColumnSpecs) { + orderBys.add(new OrderBy(spec.getDimension(), Order.fromString(spec.getDirection().toString()))); + + final VirtualColumn vc = inputColToVirtualCol.get(spec.getDimension()); + if (vc != null) { + columns.add(spec.getDimension()); + final ColumnCapabilities capabilities = vc.capabilities(baseRowSignature, vc.getOutputName()); + DruidException.conditionalDefensive( + capabilities != null, + "virtual column[%s] has null capabilities, cannot determine output type", + vc.getOutputName() + ); + rowSignatureWithOrderByBuilder.add(spec.getDimension(), capabilities.toColumnType()); + } + } + } else { + orderBys = null; + } + Druids.ScanQueryBuilder scanQueryBuilder = new Druids.ScanQueryBuilder() .dataSource(getInputDataSource(dataSchema.getDataSource())) - .columns(rowSignature.getColumnNames()) - .virtualColumns(virtualColumns) - .columnTypes(rowSignature.getColumnTypes()) .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) .filters(dataSchema.getTransformSpec().getFilter()) + .virtualColumns(VirtualColumns.create(inputColToVirtualCol.values())) + .columns(columns) + .columnTypes(rowSignatureWithOrderByBuilder.build().getColumnTypes()) + .orderBy(orderBys) .context(buildQueryContext(compactionTask.getContext(), dataSchema)); - - if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { - List orderByColumnSpecs = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()); - - scanQueryBuilder.orderBy( - orderByColumnSpecs - .stream() - .map(orderByColumnSpec -> - new OrderBy( - orderByColumnSpec.getDimension(), - Order.fromString(orderByColumnSpec.getDirection().toString()) - )) - .collect(Collectors.toList()) - ); - } return scanQueryBuilder.build(); } @@ -645,8 +697,7 @@ private Query buildGroupByQuery( ) { DimFilter dimFilter = dataSchema.getTransformSpec().getFilter(); - - VirtualColumns virtualColumns = VirtualColumns.create(new ArrayList<>(inputColToVirtualCol.values())); + VirtualColumns virtualColumns = VirtualColumns.create(inputColToVirtualCol.values()); // Convert MVDs converted to arrays back to MVDs, with the same name as the input column. // This is safe since input column names no longer exist at post-aggregation stage. @@ -666,20 +717,25 @@ private Query buildGroupByQuery( ) .collect(Collectors.toList()); + final List orderBy; + if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { + orderBy = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()); + } else { + orderBy = List.of(); + } + GroupByQuery.Builder builder = new GroupByQuery.Builder() .setDataSource(getInputDataSource(compactionTask.getDataSource())) - .setVirtualColumns(virtualColumns) - .setDimFilter(dimFilter) + .setInterval(interval) .setGranularity(new AllGranularity()) - .setDimensions(getAggregateDimensions(dataSchema, inputColToVirtualCol)) - .setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators())) + .setDimFilter(dimFilter) + .setVirtualColumns(virtualColumns) + .setDimensions(getAggregateDimensions(dataSchema, inputColToVirtualCol, orderBy)) + .setAggregatorSpecs(dataSchema.getAggregators()) .setPostAggregatorSpecs(postAggregators) - .setContext(buildQueryContext(compactionTask.getContext(), dataSchema)) - .setInterval(interval); + .setOrderByColumns(orderBy) + .setContext(buildQueryContext(compactionTask.getContext(), dataSchema)); - if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { - getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()).forEach(builder::addOrderByColumn); - } return builder.build(); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java index 96cf3d3510b6..42cd3fe4b7e6 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java @@ -46,6 +46,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; /** * Represents a stage that reads data from input sources. @@ -75,7 +76,7 @@ public RowSignature getLogicalRowSignature() @Override public ShuffleSpec buildShuffleSpec() { - final ClusterBy clusterBy = new ClusterBy(keyColumns, 0); + final ClusterBy clusterBy = new ClusterBy(keyColumns, Map.of(), 0); return new HashShuffleSpec(clusterBy, 1); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java index 478a7cfb554c..6f28c9f04931 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java @@ -122,7 +122,7 @@ public static ClusterBy clusterByWithSegmentGranularity( final List newColumns = new ArrayList<>(clusterBy.getColumns().size() + 1); newColumns.add(new KeyColumn(QueryKitUtils.SEGMENT_GRANULARITY_COLUMN, KeyOrder.ASCENDING)); newColumns.addAll(clusterBy.getColumns()); - return new ClusterBy(newColumns, 1); + return new ClusterBy(newColumns, clusterBy.getVirtualColumnMap(), 1); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java index 8ace680938c1..2b96bd81c26a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java @@ -49,11 +49,14 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.Optional; public class GroupByQueryKit implements QueryKit @@ -267,6 +270,7 @@ private static ClusterBy computeResultClusterBy( )); return new ClusterBy( resultClusterByWithPartitionBoostColumns, + resultClusterByWithoutPartitionBoost.getVirtualColumnMap(), resultClusterByWithoutPartitionBoost.getBucketByCount() ); } @@ -334,6 +338,7 @@ static ClusterBy computeClusterByForResults(final GroupByQuery query) if (!defaultLimitSpec.getColumns().isEmpty()) { final List clusterByColumns = new ArrayList<>(); + final Map clusterByVirtualColumnMap = new LinkedHashMap<>(); for (final OrderByColumnSpec orderBy : defaultLimitSpec.getColumns()) { clusterByColumns.add( @@ -344,9 +349,20 @@ static ClusterBy computeClusterByForResults(final GroupByQuery query) : KeyOrder.ASCENDING ) ); + + // check for virtual columns + final DimensionSpec dim = query.getDimensions() + .stream() + .filter(x -> x.getOutputName().equals(orderBy.getDimension())) + .findFirst() + .orElse(null); + final VirtualColumn vc = dim == null ? null : query.getVirtualColumns().getVirtualColumn(dim.getDimension()); + if (vc != null) { + clusterByVirtualColumnMap.put(orderBy.getDimension(), vc); + } } - return new ClusterBy(clusterByColumns, 0); + return new ClusterBy(clusterByColumns, clusterByVirtualColumnMap, 0); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index f6cc9222df99..eb87c7a85b4a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -42,11 +42,14 @@ import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; public class ScanQueryKit implements QueryKit { @@ -103,6 +106,7 @@ public QueryDefinition makeQueryDefinition( final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, queryToRun.getContext()); final List clusterByColumns = new ArrayList<>(); + final Map clusterByVirtualColumnMap = new LinkedHashMap<>(); // Add regular orderBys. for (final OrderBy orderBy : queryToRun.getOrderBys()) { @@ -112,13 +116,23 @@ public QueryDefinition makeQueryDefinition( orderBy.getOrder() == Order.DESCENDING ? KeyOrder.DESCENDING : KeyOrder.ASCENDING ) ); + VirtualColumn vc = queryToRun.getVirtualColumns().getVirtualColumn(orderBy.getColumnName()); + if (vc != null) { + clusterByVirtualColumnMap.put(orderBy.getColumnName(), vc); + } } clusterByColumns.add(new KeyColumn(QueryKitUtils.PARTITION_BOOST_COLUMN, KeyOrder.ASCENDING)); signatureBuilder.add(QueryKitUtils.PARTITION_BOOST_COLUMN, ColumnType.LONG); - final ClusterBy clusterBy = - QueryKitUtils.clusterByWithSegmentGranularity(new ClusterBy(clusterByColumns, 0), segmentGranularity); + final ClusterBy clusterBy = QueryKitUtils.clusterByWithSegmentGranularity( + new ClusterBy( + clusterByColumns, + clusterByVirtualColumnMap, + 0 + ), + segmentGranularity + ); final ShuffleSpec finalShuffleSpec = resultShuffleSpecFactory.build(clusterBy, false); final RowSignature signatureToUse = QueryKitUtils.sortableSignature( diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicerTest.java index b18ee8dde537..07a4ad9769ca 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicerTest.java @@ -46,6 +46,7 @@ import org.apache.druid.query.TableDataSource; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.FilterSegmentPruner; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -105,6 +106,7 @@ public class DartTableInputSpecSlicerTest extends InitializedNullHandlingTest .interval(Intervals.of("2000/2001")) .shardSpec(new DimensionRangeShardSpec( ImmutableList.of(PARTITION_DIM), + VirtualColumns.EMPTY, null, new StringTuple(new String[]{"foo"}), 0, @@ -119,6 +121,7 @@ public class DartTableInputSpecSlicerTest extends InitializedNullHandlingTest .interval(Intervals.of("2000/2001")) .shardSpec(new DimensionRangeShardSpec( ImmutableList.of(PARTITION_DIM), + VirtualColumns.EMPTY, new StringTuple(new String[]{"foo"}), null, 1, @@ -449,6 +452,7 @@ public void test_sliceStatic_dimensionFilter_twoSlices() ); final FilterSegmentPruner pruner = new FilterSegmentPruner( new EqualityFilter(PARTITION_DIM, ColumnType.STRING, "abc", null), + null, null ); diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index c532e8334744..e6bd5ded7f15 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -328,7 +328,7 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName SegmentId.of("foo", Intervals.ETERNITY, "test", 0) ) ) - .setExpectedShardSpec(NumberedShardSpec.class) + .setExpectedShardSpec(DimensionRangeShardSpec.class) .setExpectedResultRows( ImmutableList.of( new Object[]{946684800000L, "", 1.0f}, @@ -347,7 +347,7 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName .setExpectedLastCompactionState( expectedCompactionState( context, - Collections.emptyList(), + List.of("v0"), DimensionsSpec.builder() .setDimensions( ImmutableList.of( @@ -363,7 +363,7 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName ) .verifyResults(); } - + @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithAllClusteredByExpression(String contextName, Map context) diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataSegmentWithLocationTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataSegmentWithLocationTest.java index 4fef5fd8d3d1..dd75caaffa3d 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataSegmentWithLocationTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataSegmentWithLocationTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -80,13 +79,11 @@ public void testSerde_dataSegmentWithLocation() throws Exception new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ), diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java index 55ba95a9d98f..61fadac716a7 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java @@ -32,6 +32,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.FilterSegmentPruner; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; @@ -60,6 +61,7 @@ public class IndexerTableInputSpecSlicerTest extends InitializedNullHandlingTest Collections.emptyList(), new DimensionRangeShardSpec( ImmutableList.of("dim"), + VirtualColumns.EMPTY, null, new StringTuple(new String[]{"foo"}), 0, @@ -79,6 +81,7 @@ public class IndexerTableInputSpecSlicerTest extends InitializedNullHandlingTest Collections.emptyList(), new DimensionRangeShardSpec( ImmutableList.of("dim"), + VirtualColumns.EMPTY, new StringTuple(new String[]{"foo"}), null, 1, @@ -276,6 +279,7 @@ public void test_sliceStatic_dimFilter() ); final FilterSegmentPruner pruner = new FilterSegmentPruner( new EqualityFilter("dim", ColumnType.STRING, "bar", null), + null, null ); @@ -309,7 +313,8 @@ public void test_sliceStatic_dimFilterNotUsed() ); final FilterSegmentPruner segmentPruner = new FilterSegmentPruner( new EqualityFilter("dim", ColumnType.STRING, "bar", null), - Collections.emptySet() + Collections.emptySet(), + null ); Assert.assertEquals( @@ -350,6 +355,7 @@ public void test_sliceStatic_intervalAndDimFilter() ); final FilterSegmentPruner segmentPruner = new FilterSegmentPruner( new EqualityFilter("dim", ColumnType.STRING, "bar", null), + null, null ); diff --git a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java index 44e9151692a0..fb90cfe471bf 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java +++ b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java @@ -27,13 +27,16 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -45,16 +48,27 @@ public class ClusterBy { private final List columns; + private final Map virtualColumnMap; private final int bucketByCount; private final boolean sortable; + public ClusterBy( + List keyColumns, + int bucketByCount + ) + { + this(keyColumns, Map.of(), bucketByCount); + } + @JsonCreator public ClusterBy( @JsonProperty("columns") List columns, + @JsonProperty("virtualColumnMap") @Nullable Map virtualColumnMap, @JsonProperty("bucketByCount") int bucketByCount ) { this.columns = Preconditions.checkNotNull(columns, "columns"); + this.virtualColumnMap = virtualColumnMap == null ? Map.of() : virtualColumnMap; this.bucketByCount = bucketByCount; if (bucketByCount < 0 || bucketByCount > columns.size()) { @@ -63,17 +77,14 @@ public ClusterBy( // Key must be 100% sortable or 100% nonsortable. If empty, call it sortable. boolean sortable = true; - for (int i = 0; i < columns.size(); i++) { final KeyColumn column = columns.get(i); - if (i == 0) { sortable = column.order().sortable(); } else if (sortable != column.order().sortable()) { throw new IAE("Cannot mix sortable and unsortable key columns"); } } - this.sortable = sortable; } @@ -94,6 +105,13 @@ public List getColumns() return columns; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public Map getVirtualColumnMap() + { + return virtualColumnMap; + } + /** * How many fields from {@link #getColumns()} comprise the "bucket key". Bucketing is like strict partitioning: all * rows in a given partition will have the exact same bucket key. It is most commonly used to implement @@ -189,13 +207,15 @@ public boolean equals(Object o) return false; } ClusterBy clusterBy = (ClusterBy) o; - return bucketByCount == clusterBy.bucketByCount && Objects.equals(columns, clusterBy.columns); + return bucketByCount == clusterBy.bucketByCount && + Objects.equals(columns, clusterBy.columns) && + Objects.equals(virtualColumnMap, clusterBy.virtualColumnMap); } @Override public int hashCode() { - return Objects.hash(columns, bucketByCount); + return Objects.hash(columns, virtualColumnMap, bucketByCount); } @Override @@ -203,6 +223,7 @@ public String toString() { return "ClusterBy{" + "columns=" + columns + + ", virtualColumns=" + virtualColumnMap + ", bucketByCount=" + bucketByCount + '}'; } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelHashPartitioner.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelHashPartitioner.java index 033e8f921ea9..d5f52465a08a 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelHashPartitioner.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelHashPartitioner.java @@ -209,7 +209,7 @@ private void readNextFrame(final IntSet readableInputs) cursor = FrameProcessors.makeCursor( frame, frameReader, - VirtualColumns.create(Collections.singletonList(hashPartitionVirtualColumn)) + VirtualColumns.create(hashPartitionVirtualColumn) ); cursorRowPartitionNumberSupplier = diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 24765f551e28..2bd173b22e01 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -200,7 +200,7 @@ public TimeseriesQueryBuilder virtualColumns(VirtualColumns virtualColumns) public TimeseriesQueryBuilder virtualColumns(VirtualColumn... virtualColumns) { - return virtualColumns(VirtualColumns.create(Arrays.asList(virtualColumns))); + return virtualColumns(VirtualColumns.create(virtualColumns)); } public TimeseriesQueryBuilder filters(String dimensionName, String value) @@ -445,7 +445,7 @@ public SearchQueryBuilder dimensions(DimensionSpec d) public SearchQueryBuilder virtualColumns(VirtualColumn... vc) { - virtualColumns = VirtualColumns.create(Arrays.asList(vc)); + virtualColumns = VirtualColumns.create(vc); return this; } @@ -906,7 +906,7 @@ public ScanQueryBuilder virtualColumns(VirtualColumns virtualColumns) public ScanQueryBuilder virtualColumns(VirtualColumn... virtualColumns) { - return virtualColumns(VirtualColumns.create(Arrays.asList(virtualColumns))); + return virtualColumns(VirtualColumns.create(virtualColumns)); } public ScanQueryBuilder context(Map c) diff --git a/processing/src/main/java/org/apache/druid/query/filter/FilterSegmentPruner.java b/processing/src/main/java/org/apache/druid/query/filter/FilterSegmentPruner.java index 6dce9018b7a6..232469612dec 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/FilterSegmentPruner.java +++ b/processing/src/main/java/org/apache/druid/query/filter/FilterSegmentPruner.java @@ -21,6 +21,8 @@ import com.google.common.collect.RangeSet; import org.apache.druid.error.InvalidInput; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.ShardSpec; @@ -44,16 +46,19 @@ public class FilterSegmentPruner implements SegmentPruner { private final DimFilter filter; private final Set filterFields; + private final VirtualColumns virtualColumns; private final Map>> rangeCache; public FilterSegmentPruner( DimFilter filter, - @Nullable Set filterFields + @Nullable Set filterFields, + @Nullable VirtualColumns virtualColumns ) { InvalidInput.conditionalException(filter != null, "filter must not be null"); this.filter = filter; this.filterFields = filterFields == null ? filter.getRequiredColumns() : filterFields; + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; this.rangeCache = new HashMap<>(); } @@ -89,10 +94,25 @@ public Collection prune(Iterable input, Function conve Map> filterDomain = new HashMap<>(); List dimensions = shard.getDomainDimensions(); for (String dimension : dimensions) { - if (filterFields == null || filterFields.contains(dimension)) { + final VirtualColumn shardVirtualColumn = shard.getDomainVirtualColumns().getVirtualColumn(dimension); + if (shardVirtualColumn != null) { + final VirtualColumn queryEquivalent = virtualColumns.findEquivalent(shardVirtualColumn); + if (queryEquivalent != null) { + if (filterFields == null || filterFields.contains(queryEquivalent.getOutputName())) { + Optional> optFilterRangeSet = rangeCache + .computeIfAbsent( + queryEquivalent.getOutputName(), + d -> Optional.ofNullable(filter.getDimensionRangeSet(d)) + ); + optFilterRangeSet.ifPresent(stringRangeSet -> filterDomain.put( + shardVirtualColumn.getOutputName(), + stringRangeSet + )); + } + } + } else if (filterFields == null || filterFields.contains(dimension)) { Optional> optFilterRangeSet = rangeCache.computeIfAbsent(dimension, d -> Optional.ofNullable(filter.getDimensionRangeSet(d))); - optFilterRangeSet.ifPresent(stringRangeSet -> filterDomain.put(dimension, stringRangeSet)); } } @@ -115,13 +135,15 @@ public boolean equals(Object o) return false; } FilterSegmentPruner that = (FilterSegmentPruner) o; - return Objects.equals(filter, that.filter) && Objects.equals(filterFields, that.filterFields); + return Objects.equals(filter, that.filter) && + Objects.equals(filterFields, that.filterFields) && + Objects.equals(virtualColumns, that.virtualColumns); } @Override public int hashCode() { - return Objects.hash(filter, filterFields); + return Objects.hash(filter, filterFields, virtualColumns); } @Override @@ -130,7 +152,7 @@ public String toString() return "FilterSegmentPruner{" + "filter=" + filter + ", filterFields=" + filterFields + - ", rangeCache=" + rangeCache + + ", virtualColumns=" + virtualColumns + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java index 4de4e592c7f1..d31f130f6654 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java @@ -1052,7 +1052,7 @@ public Builder setVirtualColumns(VirtualColumns virtualColumns) public Builder setVirtualColumns(VirtualColumn... virtualColumns) { - this.virtualColumns = VirtualColumns.create(Arrays.asList(virtualColumns)); + this.virtualColumns = VirtualColumns.create(virtualColumns); return this; } @@ -1088,6 +1088,14 @@ public Builder addOrderByColumn(OrderByColumnSpec columnSpec) return this; } + public Builder setOrderByColumns(List columnSpec) + { + ensureExplicitLimitSpecNotSet(); + this.orderByColumnSpecs = new ArrayList<>(columnSpec); + this.postProcessingFn = null; + return this; + } + public Builder setLimitSpec(LimitSpec limitSpec) { Preconditions.checkNotNull(limitSpec); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java index 0c04839c3d2c..88f48af16f1a 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -203,6 +203,10 @@ public Function, Sequence> build(final GroupByQue sortingNeeded = true; break; } + if (query.getVirtualColumns().getVirtualColumn(columnSpec.getDimension()) != null) { + sortingNeeded = true; + break; + } final ColumnType columnType = getOrderByType(columnSpec, dimensions); final StringComparator naturalComparator; diff --git a/processing/src/main/java/org/apache/druid/query/planning/ExecutionVertex.java b/processing/src/main/java/org/apache/druid/query/planning/ExecutionVertex.java index d5baa72dd084..64a974771a20 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/ExecutionVertex.java +++ b/processing/src/main/java/org/apache/druid/query/planning/ExecutionVertex.java @@ -232,7 +232,8 @@ public SegmentPruner getSegmentPruner() return new FilterSegmentPruner( topQuery.getFilter(), - baseFields + baseFields, + topQuery.getVirtualColumns() ); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java index 437f9fab1015..8f2a7d445409 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java @@ -167,7 +167,7 @@ public TopNQueryBuilder virtualColumns(VirtualColumns virtualColumns) public TopNQueryBuilder virtualColumns(VirtualColumn... virtualColumns) { - return virtualColumns(VirtualColumns.create(Arrays.asList(virtualColumns))); + return virtualColumns(VirtualColumns.create(virtualColumns)); } public TopNQueryBuilder dataSource(DataSource d) diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java index 177f2c024163..8359b8664ab0 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java @@ -514,7 +514,7 @@ static CursorBuildSpec transformCursorBuildSpec( .setInterval(spec.getInterval()) .setFilter(baseTableFilter) .setPhysicalColumns(physicalColumns) - .setVirtualColumns(VirtualColumns.create(List.of(unnestColumn))) + .setVirtualColumns(VirtualColumns.create(unnestColumn)) .setPreferredOrdering(maybeOrderByTime) .setQueryContext(spec.getQueryContext()) .setQueryMetrics(spec.getQueryMetrics()) diff --git a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java index c0a3c885eeaa..c7d5cc54cb44 100644 --- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java +++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java @@ -53,6 +53,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -88,7 +89,7 @@ public static Pair splitColumnName(String columnName) } @JsonCreator - public static VirtualColumns create(@Nullable List virtualColumns) + public static VirtualColumns create(@Nullable Collection virtualColumns) { if (virtualColumns == null || virtualColumns.isEmpty()) { return EMPTY; diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BaseDimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BaseDimensionRangeShardSpec.java index 9d5abd6f76e6..b1e0d01c7476 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/BaseDimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/BaseDimensionRangeShardSpec.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; import java.util.Arrays; @@ -33,6 +34,7 @@ public abstract class BaseDimensionRangeShardSpec implements ShardSpec { protected final List dimensions; + protected final VirtualColumns virtualColumns; @Nullable protected final StringTuple start; @Nullable @@ -40,11 +42,13 @@ public abstract class BaseDimensionRangeShardSpec implements ShardSpec protected BaseDimensionRangeShardSpec( List dimensions, + @Nullable VirtualColumns virtualColumns, @Nullable StringTuple start, @Nullable StringTuple end ) { this.dimensions = dimensions; + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; this.start = start; this.end = end; } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpec.java index 308e36474a6b..c15f58370a78 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; import java.util.List; @@ -109,6 +110,7 @@ public DimensionRangeShardSpec convert(int numCorePartitions) numCorePartitions ) : new DimensionRangeShardSpec( dimensions, + VirtualColumns.EMPTY, start, end, partitionId, diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpec.java index 6d187eef8ae5..58a2003c11f0 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpec.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; import java.util.List; @@ -51,7 +52,7 @@ public DimensionRangeBucketShardSpec( @JsonProperty("end") @Nullable StringTuple end ) { - super(dimensions, start, end); + super(dimensions, VirtualColumns.EMPTY, start, end); // Verify that the tuple sizes and number of dimensions are the same Preconditions.checkArgument( start == null || start.size() == dimensions.size(), diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java index 4054a618a752..dba0ad1ad332 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java @@ -20,12 +20,14 @@ package org.apache.druid.timeline.partition; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.TreeRangeSet; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; import java.util.Collections; @@ -53,13 +55,14 @@ public class DimensionRangeShardSpec extends BaseDimensionRangeShardSpec @JsonCreator public DimensionRangeShardSpec( @JsonProperty("dimensions") List dimensions, + @JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns, @JsonProperty("start") @Nullable StringTuple start, @JsonProperty("end") @Nullable StringTuple end, @JsonProperty("partitionNum") int partitionNum, @JsonProperty("numCorePartitions") @Nullable Integer numCorePartitions // nullable for backward compatibility ) { - super(dimensions, start, end); + super(dimensions, virtualColumns, start, end); Preconditions.checkArgument(partitionNum >= 0, "partitionNum >= 0"); Preconditions.checkArgument( dimensions != null && !dimensions.isEmpty(), @@ -76,6 +79,13 @@ public List getDimensions() return dimensions; } + @JsonProperty("virtualColumns") + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + @Nullable @JsonProperty("start") public StringTuple getStartTuple() @@ -115,6 +125,12 @@ public List getDomainDimensions() return Collections.unmodifiableList(dimensions); } + @Override + public VirtualColumns getDomainVirtualColumns() + { + return virtualColumns; + } + /** * Set[:i] is the cartesian product of Set[0],...,Set[i - 1] * EffectiveDomain[:i] is defined as QueryDomain[:i] INTERSECTION SegmentRange[:i] @@ -271,6 +287,7 @@ public boolean equals(Object o) return partitionNum == shardSpec.partitionNum && numCorePartitions == shardSpec.numCorePartitions && Objects.equals(dimensions, shardSpec.dimensions) && + Objects.equals(virtualColumns, shardSpec.virtualColumns) && Objects.equals(start, shardSpec.start) && Objects.equals(end, shardSpec.end); } @@ -278,7 +295,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(dimensions, start, end, partitionNum, numCorePartitions); + return Objects.hash(dimensions, virtualColumns, start, end, partitionNum, numCorePartitions); } @Override @@ -286,6 +303,7 @@ public String toString() { return "DimensionRangeShardSpec{" + "dimensions='" + dimensions + '\'' + + ", virtualColumns=" + virtualColumns + ", start='" + start + '\'' + ", end='" + end + '\'' + ", partitionNum=" + partitionNum + diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java index ffa2198f4497..6f60d36b9517 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.RangeSet; +import org.apache.druid.segment.VirtualColumns; import java.util.List; import java.util.Map; @@ -117,6 +118,12 @@ default short getAtomicUpdateGroupSize() @JsonIgnore List getDomainDimensions(); + @JsonIgnore + default VirtualColumns getDomainVirtualColumns() + { + return VirtualColumns.EMPTY; + } + /** * if given domain ranges are not possible in this shard, return false; otherwise return true; * @return possibility of in domain diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionRangeBucketShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionRangeBucketShardSpec.java index e47b5aafaeae..ae9ea1fb416e 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionRangeBucketShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionRangeBucketShardSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; import java.util.Collections; @@ -52,6 +53,7 @@ public SingleDimensionRangeBucketShardSpec( { super( dimension == null ? Collections.emptyList() : Collections.singletonList(dimension), + VirtualColumns.EMPTY, start == null ? null : StringTuple.create(start), end == null ? null : StringTuple.create(end) ); diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java index 59ef1be3d6ac..aa346adabbf4 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java @@ -25,6 +25,7 @@ import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; import java.util.Collections; @@ -62,6 +63,7 @@ public SingleDimensionShardSpec( { super( dimension == null ? Collections.emptyList() : Collections.singletonList(dimension), + VirtualColumns.EMPTY, start == null ? null : StringTuple.create(start), end == null ? null : StringTuple.create(end), partitionNum, diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java index 9ec348d7ea63..73f71375de4a 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java @@ -62,7 +62,6 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -308,7 +307,7 @@ public static CursorHolder makeCursorForCursorFactory( if (populateRowNumber) { rowNumberVirtualColumn = new SettableLongVirtualColumn(ROW_NUMBER_COLUMN); - virtualColumns = VirtualColumns.create(Collections.singletonList(rowNumberVirtualColumn)); + virtualColumns = VirtualColumns.create(rowNumberVirtualColumn); } else { rowNumberVirtualColumn = null; virtualColumns = VirtualColumns.EMPTY; diff --git a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java index 131337ad4683..222a2c2dd7fe 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java @@ -24,7 +24,9 @@ import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; @@ -46,7 +48,7 @@ void testNullFilter() { Throwable t = Assertions.assertThrows( DruidException.class, - () -> new FilterSegmentPruner(null, null) + () -> new FilterSegmentPruner(null, null, null) ); Assertions.assertEquals("filter must not be null", t.getMessage()); } @@ -70,15 +72,47 @@ void testPrune() List segs = List.of(seg1, seg2, seg3, seg4, seg5, seg6, seg7); - FilterSegmentPruner prunerRange = new FilterSegmentPruner(range_a, null); - FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet()); - FilterSegmentPruner prunerExpression = new FilterSegmentPruner(expression_b, null); + FilterSegmentPruner prunerRange = new FilterSegmentPruner(range_a, null, null); + FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), null); + FilterSegmentPruner prunerExpression = new FilterSegmentPruner(expression_b, null, null); Assertions.assertEquals(Set.of(seg1, seg4, seg5, seg6, seg7), prunerRange.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerExpression.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); } + @Test + void testPruneVirtualColumn() + { + VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("vdim1", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + DimFilter range_a = new RangeFilter("vdim1", ColumnType.STRING, null, "aaa", null, null, null); + + String interval1 = "2026-02-18T00:00:00Z/2026-02-19T00:00:00Z"; + + DataSegment seg1 = makeDataSegment( + interval1, + makeRange(List.of("vdim1"), virtualColumns, 0, null, StringTuple.create("abcfoo")) + ); + DataSegment seg2 = makeDataSegment( + interval1, + makeRange(List.of("vdim1"), virtualColumns, 1, StringTuple.create("abcfoo"), StringTuple.create("lmnfoo")) + ); + DataSegment seg3 = makeDataSegment( + interval1, + makeRange(List.of("vdim1"), virtualColumns, 2, StringTuple.create("lmnfoo"), null) + ); + + List segs = List.of(seg1, seg2, seg3); + + FilterSegmentPruner prunerRange = new FilterSegmentPruner(range_a, null, virtualColumns); + FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), virtualColumns); + + Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); + } + @Test void testEqualsAndHashcode() { @@ -106,9 +140,27 @@ private ShardSpec makeRange( @Nullable StringTuple start, @Nullable StringTuple end ) + { + return makeRange( + columns, + null, + partitionNumber, + start, + end + ); + } + + private ShardSpec makeRange( + List columns, + VirtualColumns virtualColumns, + int partitionNumber, + @Nullable StringTuple start, + @Nullable StringTuple end + ) { return new DimensionRangeShardSpec( columns, + virtualColumns, start, end, partitionNumber, diff --git a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java index 7e5fdc0e414d..575d44d59186 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java @@ -88,9 +88,9 @@ public void testWrappedOperatorCarriesThroughValues() throws JsonProcessingExcep Collections.emptyList() }; VirtualColumns[] virtualCols = new VirtualColumns[]{ - VirtualColumns.create(Collections.singletonList( + VirtualColumns.create( new ExpressionVirtualColumn("test", "2", null, TestExprMacroTable.INSTANCE) - )) + ) }; for (int i = 0; i <= intervals.length; ++i) { diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryQueryToolChestTest.java index fc2f4d249319..bff440e4720f 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryQueryToolChestTest.java @@ -94,9 +94,7 @@ public void testCacheStrategyVirtualColumns() new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2015-01-01/2015-01-02"))), ImmutableList.of(Druids.DIMENSION_IDENTITY.apply("v0")), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("v0", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) - ) + new ExpressionVirtualColumn("v0", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) ), new FragmentSearchQuerySpec(ImmutableList.of("a", "b")), null, @@ -111,9 +109,7 @@ public void testCacheStrategyVirtualColumns() new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2015-01-01/2015-01-02"))), ImmutableList.of(Druids.DIMENSION_IDENTITY.apply("v0")), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("v0", "concat(dim2, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) - ) + new ExpressionVirtualColumn("v0", "concat(dim2, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) ), new FragmentSearchQuerySpec(ImmutableList.of("a", "b")), null, @@ -128,9 +124,7 @@ public void testCacheStrategyVirtualColumns() new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2015-01-01/2015-01-02"))), ImmutableList.of(Druids.DIMENSION_IDENTITY.apply("v0")), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("v0", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) - ) + new ExpressionVirtualColumn("v0", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) ), new FragmentSearchQuerySpec(ImmutableList.of("a", "b")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java index 2b2613465084..288603bead07 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java @@ -432,13 +432,11 @@ public void test_makeCursor_filterOnVirtualColumn() .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) .setVirtualColumns( VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "vc", - "\"LONG\" + 1", - ColumnType.LONG, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "vc", + "\"LONG\" + 1", + ColumnType.LONG, + ExprMacroTable.nil() ) ) ) diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index c4d363b8ae8b..bc9006954a08 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -73,7 +73,6 @@ import java.net.URL; import java.nio.charset.StandardCharsets; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -172,9 +171,7 @@ public class TestIndex public static final Interval DATA_INTERVAL = Intervals.of("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"); private static final Logger log = new Logger(TestIndex.class); private static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create( - Collections.singletonList( - new ExpressionVirtualColumn("expr", "index + 10", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) - ) + new ExpressionVirtualColumn("expr", "index + 10", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) ); public static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory(DOUBLE_METRICS[0], "index"), diff --git a/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java b/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java index 5466748a76ae..c3ffe2676387 100644 --- a/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java @@ -347,7 +347,7 @@ public void testTimeNotAllowed() Throwable t = Assert.assertThrows( IllegalArgumentException.class, - () -> VirtualColumns.create(ImmutableList.of(expr)) + () -> VirtualColumns.create(expr) ); Assert.assertEquals("virtualColumn name[__time] not allowed", t.getMessage()); } @@ -371,7 +371,7 @@ public void testDuplicateNameDetection() Throwable t = Assert.assertThrows( IllegalArgumentException.class, - () -> VirtualColumns.create(ImmutableList.of(expr, expr2)) + () -> VirtualColumns.create(expr, expr2) ); Assert.assertEquals("Duplicate virtualColumn name[expr]", t.getMessage()); } @@ -398,7 +398,7 @@ public void testCycleDetection() Throwable t = Assert.assertThrows( IllegalArgumentException.class, - () -> VirtualColumns.create(ImmutableList.of(expr, expr2)) + () -> VirtualColumns.create(expr, expr2) ); Assert.assertEquals("Self-referential column[expr]", t.getMessage()); } @@ -431,7 +431,7 @@ public void testNotACycle() TestExprMacroTable.INSTANCE ); - final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(expr, expr2, expr3)); + final VirtualColumns virtualColumns = VirtualColumns.create(expr, expr2, expr3); Assert.assertEquals(3, virtualColumns.getColumnNames().size()); } @@ -439,15 +439,11 @@ public void testNotACycle() public void testGetCacheKey() { final VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("expr", "x + y", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) - ) + new ExpressionVirtualColumn("expr", "x + y", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) ); final VirtualColumns virtualColumns2 = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("expr", "x + y", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) - ) + new ExpressionVirtualColumn("expr", "x + y", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) ); Assert.assertArrayEquals(virtualColumns.getCacheKey(), virtualColumns2.getCacheKey()); @@ -478,7 +474,7 @@ public void testEquivalence() ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); - final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(v0)); + final VirtualColumns virtualColumns = VirtualColumns.create(v0); final VirtualColumn v1 = new ExpressionVirtualColumn( "differentNameExpr", @@ -538,7 +534,7 @@ public void testCompositeVirtualColumnsCycles() final ExpressionVirtualColumn expr1 = new ExpressionVirtualColumn("v1", "1 + x", ColumnType.LONG, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn("v2", "1 + y", ColumnType.LONG, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr0 = new ExpressionVirtualColumn("v0", "case_searched(notnull(1 + x), v1, v2)", ColumnType.LONG, TestExprMacroTable.INSTANCE); - final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(expr0, expr1, expr2)); + final VirtualColumns virtualColumns = VirtualColumns.create(expr0, expr1, expr2); Assert.assertTrue(virtualColumns.exists("v0")); Assert.assertTrue(virtualColumns.exists("v1")); @@ -551,7 +547,7 @@ public void testCompositeVirtualColumnsCyclesSiblings() final ExpressionVirtualColumn expr1 = new ExpressionVirtualColumn("v1", "1 + x", ColumnType.LONG, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn("v2", "1 + y", ColumnType.LONG, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr0 = new ExpressionVirtualColumn("v0", "case_searched(notnull(v1), v1, v2)", ColumnType.LONG, TestExprMacroTable.INSTANCE); - final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(expr0, expr1, expr2)); + final VirtualColumns virtualColumns = VirtualColumns.create(expr0, expr1, expr2); Assert.assertTrue(virtualColumns.exists("v0")); Assert.assertTrue(virtualColumns.exists("v1")); @@ -564,7 +560,7 @@ public void testCompositeVirtualColumnsCyclesTree() final ExpressionVirtualColumn expr1 = new ExpressionVirtualColumn("v1", "1 + x", ColumnType.LONG, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn("v2", "1 + v1", ColumnType.LONG, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr0 = new ExpressionVirtualColumn("v0", "v1 + v2", ColumnType.LONG, TestExprMacroTable.INSTANCE); - final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(expr0, expr1, expr2)); + final VirtualColumns virtualColumns = VirtualColumns.create(expr0, expr1, expr2); Assert.assertTrue(virtualColumns.exists("v0")); Assert.assertTrue(virtualColumns.exists("v1")); @@ -593,7 +589,7 @@ public ColumnCapabilities getColumnCapabilities(String column) final NestedFieldVirtualColumn v1 = new NestedFieldVirtualColumn("n", "$.y", "v1", ColumnType.LONG); final ExpressionVirtualColumn expr1 = new ExpressionVirtualColumn("v2", "v0 * v1", null, TestExprMacroTable.INSTANCE); final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn("v3", "v0 * x", null, TestExprMacroTable.INSTANCE); - final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(v0, v1, expr1, expr2)); + final VirtualColumns virtualColumns = VirtualColumns.create(v0, v1, expr1, expr2); Assert.assertEquals(ColumnType.STRING, virtualColumns.getColumnCapabilitiesWithoutFallback(baseInspector, "v0").toColumnType()); Assert.assertEquals(ColumnType.LONG, virtualColumns.getColumnCapabilitiesWithoutFallback(baseInspector, "v1").toColumnType()); @@ -628,7 +624,7 @@ private VirtualColumns makeVirtualColumns() TestExprMacroTable.INSTANCE ); final DottyVirtualColumn dotty = new DottyVirtualColumn("foo"); - return VirtualColumns.create(ImmutableList.of(expr, expr2i, expr2, dotty)); + return VirtualColumns.create(expr, expr2i, expr2, dotty); } static class DottyVirtualColumn implements VirtualColumn diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index a198e22f7a15..01f1aa4b4276 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -130,41 +130,84 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest static final String TIMESTAMP_COLUMN = "timestamp"; static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("expr", "1.0 + 0.1", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("exprDouble", "1.0 + 1.1", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("exprLong", "1 + 2", ColumnType.LONG, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vdim0", "dim0", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vdim1", "dim1", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vs0", "s0", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vd0", "d0", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vf0", "f0", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vl0", "l0", ColumnType.LONG, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vd0-nvl-2", "nvl(vd0, 2.0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vd0-add-sub", "d0 + (d0 - d0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vf0-add-sub", "f0 + (f0 - f0)", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vl0-add-sub", "l0 + (l0 - l0)", ColumnType.LONG, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("double-vd0-add-sub", "vd0 + (vd0 - vd0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("double-vf0-add-sub", "vf0 + (vf0 - vf0)", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("double-vl0-add-sub", "vl0 + (vl0 - vl0)", ColumnType.LONG, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vdim3-concat", "dim3 + dim3", ColumnType.LONG, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vdim2-offset", "array_offset(dim2, 1)", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("nestedArrayLong", "array(arrayLong)", ColumnType.ofArray(ColumnType.LONG_ARRAY), TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("fake-nvl", "nvl(fake, 'hello')", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ListFilteredVirtualColumn("allow-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), true), - new ListFilteredVirtualColumn("deny-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), false), - new ListFilteredVirtualColumn("allow-dim2", DefaultDimensionSpec.of("dim2"), ImmutableSet.of("a"), true), - new ListFilteredVirtualColumn("deny-dim2", DefaultDimensionSpec.of("dim2"), ImmutableSet.of("a"), false), - new NestedFieldVirtualColumn("nested", "$.s0", "nested.s0", ColumnType.STRING), - new NestedFieldVirtualColumn("nested", "$.d0", "nested.d0", ColumnType.DOUBLE), - new NestedFieldVirtualColumn("nested", "$.l0", "nested.l0", ColumnType.LONG), - new NestedFieldVirtualColumn("nested", "$.arrayLong", "nested.arrayLong", ColumnType.LONG_ARRAY), - new NestedFieldVirtualColumn("nested", "$.arrayDouble", "nested.arrayDouble", ColumnType.DOUBLE_ARRAY), - new NestedFieldVirtualColumn("nested", "$.arrayString", "nested.arrayString", ColumnType.STRING_ARRAY), - new ExpressionVirtualColumn("arrayLongAsMvd", "array_to_mv(arrayLong)", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("arrayDoubleAsMvd", "array_to_mv(arrayDouble)", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("arrayStringAsMvd", "array_to_mv(arrayString)", ColumnType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("arrayConstantAsMvd", "array_to_mv(array(1,2,3))", ColumnType.STRING, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("expr", "1.0 + 0.1", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("exprDouble", "1.0 + 1.1", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("exprLong", "1 + 2", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vdim0", "dim0", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vdim1", "dim1", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vs0", "s0", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vd0", "d0", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vf0", "f0", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vl0", "l0", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vd0-nvl-2", "nvl(vd0, 2.0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vd0-add-sub", "d0 + (d0 - d0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vf0-add-sub", "f0 + (f0 - f0)", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vl0-add-sub", "l0 + (l0 - l0)", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn( + "double-vd0-add-sub", + "vd0 + (vd0 - vd0)", + ColumnType.DOUBLE, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "double-vf0-add-sub", + "vf0 + (vf0 - vf0)", + ColumnType.FLOAT, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "double-vl0-add-sub", + "vl0 + (vl0 - vl0)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn("vdim3-concat", "dim3 + dim3", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn( + "vdim2-offset", + "array_offset(dim2, 1)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "nestedArrayLong", + "array(arrayLong)", + ColumnType.ofArray(ColumnType.LONG_ARRAY), + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn("fake-nvl", "nvl(fake, 'hello')", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ListFilteredVirtualColumn("allow-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), true), + new ListFilteredVirtualColumn("deny-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), false), + new ListFilteredVirtualColumn("allow-dim2", DefaultDimensionSpec.of("dim2"), ImmutableSet.of("a"), true), + new ListFilteredVirtualColumn("deny-dim2", DefaultDimensionSpec.of("dim2"), ImmutableSet.of("a"), false), + new NestedFieldVirtualColumn("nested", "$.s0", "nested.s0", ColumnType.STRING), + new NestedFieldVirtualColumn("nested", "$.d0", "nested.d0", ColumnType.DOUBLE), + new NestedFieldVirtualColumn("nested", "$.l0", "nested.l0", ColumnType.LONG), + new NestedFieldVirtualColumn("nested", "$.arrayLong", "nested.arrayLong", ColumnType.LONG_ARRAY), + new NestedFieldVirtualColumn("nested", "$.arrayDouble", "nested.arrayDouble", ColumnType.DOUBLE_ARRAY), + new NestedFieldVirtualColumn("nested", "$.arrayString", "nested.arrayString", ColumnType.STRING_ARRAY), + new ExpressionVirtualColumn( + "arrayLongAsMvd", + "array_to_mv(arrayLong)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "arrayDoubleAsMvd", + "array_to_mv(arrayDouble)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "arrayStringAsMvd", + "array_to_mv(arrayString)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "arrayConstantAsMvd", + "array_to_mv(array(1,2,3))", + ColumnType.STRING, + TestExprMacroTable.INSTANCE ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java index 32acee4cd73e..e1b4ffdf5410 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java @@ -46,7 +46,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.Collections; import java.util.List; public class HashJoinSegmentCursorFactoryTest extends BaseHashJoinSegmentCursorFactoryTest @@ -1295,9 +1294,7 @@ public void test_makeCursor_factToCountryUsingVirtualColumn() ); VirtualColumns virtualColumns = VirtualColumns.create( - Collections.singletonList( - makeExpressionVirtualColumn("concat(substring(countryIsoCode, 0, 1),'L')") - ) + makeExpressionVirtualColumn("concat(substring(countryIsoCode, 0, 1),'L')") ); JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis( @@ -1347,9 +1344,7 @@ public void test_makeCursor_factToCountryUsingVirtualColumnUsingLookup() ); VirtualColumns virtualColumns = VirtualColumns.create( - Collections.singletonList( - makeExpressionVirtualColumn("concat(substring(countryIsoCode, 0, 1),'L')") - ) + makeExpressionVirtualColumn("concat(substring(countryIsoCode, 0, 1),'L')") ); JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis( diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java index 4cbec2dbe9a6..2bf14b749835 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java @@ -378,13 +378,11 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ); VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v1", - "concat('virtual-column-', \"channel\")", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "v1", + "concat('virtual-column-', \"channel\")", + ColumnType.STRING, + TestExprMacroTable.INSTANCE ) ); @@ -448,13 +446,11 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu // a selector filter on an ExpressionVirtualColumn Filter originalFilter = new SelectorFilter("v0", "VIRGINIA"); VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v0", - "upper(\"r1.regionName\")", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "v0", + "upper(\"r1.regionName\")", + ColumnType.STRING, + TestExprMacroTable.INSTANCE ) ); @@ -514,19 +510,17 @@ public void test_filterPushDown_orFilterWithPostJoinVirtualColumnIsNotPushedDown ); final VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v0", - "concat('virtual-column-', \"channel\")", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ), - new ExpressionVirtualColumn( - "v1", - "upper(\"r1.regionName\")", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "v0", + "concat('virtual-column-', \"channel\")", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "v1", + "upper(\"r1.regionName\")", + ColumnType.STRING, + TestExprMacroTable.INSTANCE ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index 7db1445f60f4..8e694cbcd930 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.nested; import com.fasterxml.jackson.databind.Module; -import com.google.common.collect.ImmutableList; import org.apache.druid.error.DruidException; import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; @@ -317,16 +316,14 @@ private VirtualColumns makeNestedNumericVirtualColumns() ); return VirtualColumns.create( - ImmutableList.of( - longVirtualColumn, - doubleVirtualColumn, - mixedNumericVirtualColumn, - mixedVirtualColumn, - sparseLongVirtualColumn, - sparseDoubleVirtualColumn, - sparseMixedNumericVirtualColumn, - sparseMixedVirtualColumn - ) + longVirtualColumn, + doubleVirtualColumn, + mixedNumericVirtualColumn, + mixedVirtualColumn, + sparseLongVirtualColumn, + sparseDoubleVirtualColumn, + sparseMixedNumericVirtualColumn, + sparseMixedVirtualColumn ); } diff --git a/processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java b/processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java index 110605286bd1..5f966a824248 100644 --- a/processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.math.expr.ExprMacroTable; @@ -52,13 +51,11 @@ public void testSerde() throws IOException final CompactionTransformSpec expected = new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java index e103413ae9cd..90c7e56c329b 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.virtual; -import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExpressionType; @@ -107,13 +106,11 @@ public static void testCast( ) { final VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "cast(" + column + ", '" + ExpressionType.fromColumnType(castTo) + "')", - castTo, - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "v", + "cast(" + column + ", '" + ExpressionType.fromColumnType(castTo) + "')", + castTo, + TestExprMacroTable.INSTANCE ) ); final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java index 56466868f406..8c81c6b1fefa 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java @@ -224,13 +224,11 @@ public static void sanityTestVectorizedExpressionSelectors( { final List results = new ArrayList<>(rowsPerSegment); final VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - expression, - ExpressionType.toColumnType(outputType), - TestExprMacroTable.INSTANCE - ) + new ExpressionVirtualColumn( + "v", + expression, + ExpressionType.toColumnType(outputType), + TestExprMacroTable.INSTANCE ) ); final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java index 54670e7fca5d..3337729b4e35 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java @@ -51,7 +51,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.List; public class ListFilteredVirtualColumnSelectorTest extends InitializedNullHandlingTest { @@ -205,7 +204,7 @@ public void testFilterListFilteredVirtualColumnAllowIndex() throws IOException try (final Closer closer = Closer.create()) { ColumnIndexSelector bitmapIndexSelector = new ColumnCache( queryableIndex, - VirtualColumns.create(List.of(virtualColumn)), + VirtualColumns.create(virtualColumn), closer ); @@ -266,7 +265,7 @@ public void testFilterListFilteredVirtualColumnDenyIndex() try (final Closer closer = Closer.create()) { ColumnIndexSelector bitmapIndexSelector = new ColumnCache( queryableIndex, - VirtualColumns.create(List.of(virtualColumn)), + VirtualColumns.create(virtualColumn), closer ); @@ -302,7 +301,7 @@ private VirtualizedColumnSelectorFactory makeSelectorFactory(ListFilteredVirtual rowSignature, false ), - VirtualColumns.create(ImmutableList.of(virtualColumn)) + VirtualColumns.create(virtualColumn) ); return selectorFactory; diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualizedColumnSelectorFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualizedColumnSelectorFactoryTest.java index 02bcb59fe771..8323888f51d9 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualizedColumnSelectorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualizedColumnSelectorFactoryTest.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.virtual; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.math.expr.ExprMacroTable; @@ -44,10 +43,8 @@ public class VirtualizedColumnSelectorFactoryTest extends InitializedNullHandlin false ), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn("v0", "x + 1", null, ExprMacroTable.nil()), - new ExpressionVirtualColumn("v1", "v0 + y", null, ExprMacroTable.nil()) - ) + new ExpressionVirtualColumn("v0", "x + 1", null, ExprMacroTable.nil()), + new ExpressionVirtualColumn("v1", "v0 + y", null, ExprMacroTable.nil()) ) ); diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 37c175b57f27..73a9628ce19b 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -91,13 +91,11 @@ public void testSerializationWithLatestFormat() throws Exception new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ), @@ -168,13 +166,11 @@ public void testV1Serialization() throws Exception new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ), @@ -380,13 +376,11 @@ public void testWithLastCompactionState() new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ), @@ -421,13 +415,11 @@ public void testAnnotateWithLastCompactionState() CompactionTransformSpec transformSpec = new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ); diff --git a/processing/src/test/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpecTest.java b/processing/src/test/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpecTest.java index b8099e227a4f..6f7af3fecb8b 100644 --- a/processing/src/test/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpecTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/partition/BuildingDimensionRangeShardSpecTest.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.segment.VirtualColumns; import org.junit.Assert; import org.junit.Test; @@ -39,6 +40,7 @@ public void testConvert() Assert.assertEquals( new DimensionRangeShardSpec( Arrays.asList("dim1", "dim2"), + VirtualColumns.EMPTY, StringTuple.create("start1", "start2"), StringTuple.create("end1", "end2"), 5, diff --git a/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java b/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java index ade40e6ae38f..0f62da3518b1 100644 --- a/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.StringTuple; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.VirtualColumns; import org.junit.Assert; import org.junit.Test; @@ -50,15 +51,30 @@ public void testShardSpecLookup() setDimensions("dim1", "dim2"); final List shardSpecs = ImmutableList.of( - new DimensionRangeShardSpec(dimensions, null, StringTuple.create("India", "Delhi"), 1, 1), new DimensionRangeShardSpec( dimensions, + VirtualColumns.EMPTY, + null, + StringTuple.create("India", "Delhi"), + 1, + 1 + ), + new DimensionRangeShardSpec( + dimensions, + VirtualColumns.EMPTY, StringTuple.create("India", "Delhi"), StringTuple.create("Spain", "Valencia"), 2, 1 ), - new DimensionRangeShardSpec(dimensions, StringTuple.create("Spain", "Valencia"), null, 3, 1) + new DimensionRangeShardSpec( + dimensions, + VirtualColumns.EMPTY, + StringTuple.create("Spain", "Valencia"), + null, + 3, + 1 + ) ); final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); final long currentTime = DateTimes.nowUtc().getMillis(); @@ -143,6 +159,7 @@ public void testShardSpecLookupWithNull() final DimensionRangeShardSpec shard0 = new DimensionRangeShardSpec( dimensions, + VirtualColumns.EMPTY, null, StringTuple.create("India", null), 1, @@ -151,6 +168,7 @@ public void testShardSpecLookupWithNull() final DimensionRangeShardSpec shard1 = new DimensionRangeShardSpec( dimensions, + VirtualColumns.EMPTY, StringTuple.create("India", null), StringTuple.create("Spain", "Valencia"), 10, @@ -159,6 +177,7 @@ public void testShardSpecLookupWithNull() final DimensionRangeShardSpec shard2 = new DimensionRangeShardSpec( dimensions, + VirtualColumns.EMPTY, StringTuple.create("Spain", "Valencia"), StringTuple.create("Tokyo", null), 10, @@ -167,6 +186,7 @@ public void testShardSpecLookupWithNull() final DimensionRangeShardSpec shard3 = new DimensionRangeShardSpec( dimensions, + VirtualColumns.EMPTY, StringTuple.create("Tokyo", null), null, 100, @@ -202,7 +222,7 @@ public void testPossibleInDomain_withNullStart() final RangeSet universalSet = TreeRangeSet.create(); universalSet.add(Range.all()); - ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null); + ShardSpec shard = new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, start, end, 0, null); Map> domain = new HashMap<>(); // {Mars} * {Zoo, Zuu} * {Blah, Random} @@ -265,7 +285,7 @@ public void testPossibleInDomain_withNullValues() final RangeSet universalSet = TreeRangeSet.create(); universalSet.add(Range.all()); - ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null); + ShardSpec shard = new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, start, end, 0, null); Map> domain = new HashMap<>(); // (-INF, INF) * (-INF, INF) * (-INF, INF) @@ -345,7 +365,7 @@ public void testPossibleInDomain_nonNullValues_acceptanceScenarios() final RangeSet universalSet = TreeRangeSet.create(); universalSet.add(Range.all()); - ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null); + ShardSpec shard = new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, start, end, 0, null); Map> domain = new HashMap<>(); // (-INF, INF) * (-INF, INF) * (-INF, INF) @@ -442,7 +462,7 @@ public void testPossibleInDomain_nonNullValues_pruningScenarios() final RangeSet universalSet = TreeRangeSet.create(); universalSet.add(Range.all()); - ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null); + ShardSpec shard = new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, start, end, 0, null); Map> domain = new HashMap<>(); // (-INF, Earth) U (Earth, INF) * (-INF, INF) * (-INF, INF) @@ -504,7 +524,7 @@ public void testPossibleInDomain_falsePruning() final RangeSet universalSet = TreeRangeSet.create(); universalSet.add(Range.all()); - ShardSpec shard = new DimensionRangeShardSpec(dimensions, start, end, 0, null); + ShardSpec shard = new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, start, end, 0, null); Map> domain = new HashMap<>(); // {Earth} U {Mars} * (USA, INF) * (-INF, INF) diff --git a/processing/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java b/processing/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java index 2c3442645d26..daad7aafc94f 100644 --- a/processing/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Test; @@ -86,6 +87,7 @@ public static Iterable constructorFeeder() ImmutableList.of( new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, null, StringTuple.create("aaa"), 0, @@ -93,6 +95,7 @@ public static Iterable constructorFeeder() ), new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, StringTuple.create("ttt"), StringTuple.create("zzz"), 2, @@ -100,6 +103,7 @@ public static Iterable constructorFeeder() ), new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, StringTuple.create("bbb"), StringTuple.create("fff"), 1, @@ -116,6 +120,7 @@ public static Iterable constructorFeeder() ImmutableList.of( new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, StringTuple.create("bbb"), StringTuple.create("fff"), 1, @@ -123,6 +128,7 @@ public static Iterable constructorFeeder() ), new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, StringTuple.create("fff"), null, 2, @@ -130,6 +136,7 @@ public static Iterable constructorFeeder() ), new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, null, StringTuple.create("bbb"), 0, diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index 532c71977189..a77fa1f8c553 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -28,7 +28,12 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.virtual.VirtualizedColumnInspector; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -38,7 +43,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; @@ -119,7 +123,8 @@ private static CompactionConfigValidationResult compactionConfigSupportedByMSQEn if (newConfig.getTuningConfig() != null) { validationResults.add(validatePartitionsSpecForMSQ( newConfig.getTuningConfig().getPartitionsSpec(), - newConfig.getDimensionsSpec() == null ? null : newConfig.getDimensionsSpec().getDimensions() + newConfig.getDimensionsSpec() == null ? null : newConfig.getDimensionsSpec().getDimensions(), + newConfig.getTransformSpec() == null ? VirtualColumns.EMPTY : newConfig.getTransformSpec().getVirtualColumns() )); } if (newConfig.getGranularitySpec() != null) { @@ -142,7 +147,8 @@ private static CompactionConfigValidationResult compactionConfigSupportedByMSQEn */ public static CompactionConfigValidationResult validatePartitionsSpecForMSQ( @Nullable PartitionsSpec partitionsSpec, - @Nullable List dimensionSchemas + @Nullable List dimensionSchemas, + VirtualColumns virtualColumns ) { if (partitionsSpec == null) { @@ -168,16 +174,36 @@ public static CompactionConfigValidationResult validatePartitionsSpecForMSQ( Map dimensionSchemaMap = dimensionSchemas.stream().collect( Collectors.toMap(DimensionSchema::getName, Function.identity()) ); - Optional nonStringDimension = ((DimensionRangePartitionsSpec) partitionsSpec) - .getPartitionDimensions() - .stream() - .filter(dim -> !ColumnType.STRING.equals(dimensionSchemaMap.get(dim).getColumnType())) - .findAny(); - if (nonStringDimension.isPresent()) { + RowSignature.Builder baseSignatureBuilder = RowSignature.builder().addTimeColumn(); + for (DimensionSchema dimensionSchema : dimensionSchemas) { + baseSignatureBuilder.add(dimensionSchema.getName(), dimensionSchema.getColumnType()); + } + final RowSignature baseSignature = baseSignatureBuilder.build(); + final ColumnInspector inspector = new VirtualizedColumnInspector(baseSignature, virtualColumns); + + String nonString = null; + ColumnType nonStringType = null; + for (String dim : ((DimensionRangePartitionsSpec) partitionsSpec).getPartitionDimensions()) { + ColumnType partitionType = baseSignature.getColumnType(dim).orElse(null); + if (partitionType == null) { + VirtualColumn virtualColumn = virtualColumns.getVirtualColumn(dim); + if (virtualColumn != null) { + partitionType = ColumnType.fromCapabilities( + virtualColumn.capabilities(inspector, virtualColumn.getOutputName()) + ); + } + } + if (!ColumnType.STRING.equals(partitionType)) { + nonString = dim; + nonStringType = partitionType; + break; + } + } + if (nonString != null) { return CompactionConfigValidationResult.failure( "MSQ: Non-string partition dimension[%s] of type[%s] not supported with 'range' partition spec", - nonStringDimension.get(), - dimensionSchemaMap.get(nonStringDimension.get()).getTypeName() + nonString, + nonStringType ); } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 9add89949c83..25c87b32f07d 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestDataSource; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; @@ -3480,6 +3481,7 @@ public void testAddNumberedShardSpecAfterMultiDimensionsShardSpecWithUnknownCore metrics, new DimensionRangeShardSpec( Collections.singletonList("dim"), + VirtualColumns.EMPTY, i == 0 ? null : StringTuple.create(String.valueOf(i - 1)), i == 5 ? null : StringTuple.create(String.valueOf(i)), i, @@ -4174,7 +4176,7 @@ public void testRetrieveUsedSegmentsForSegmentAllocation() loadspec, dimensions, metrics, - new DimensionRangeShardSpec(dimensions, null, null, 0, 1), + new DimensionRangeShardSpec(dimensions, VirtualColumns.EMPTY, null, null, 0, 1), 0, 100 ); diff --git a/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java index a99f3157527a..7df4fef3c547 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java @@ -19,7 +19,6 @@ package org.apache.druid.server.compaction; -import com.google.common.collect.ImmutableList; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -43,13 +42,11 @@ public class ReindexingDeletionRuleTest private final DimFilter testFilter = new EqualityFilter("isRobot", ColumnType.STRING, "true", null); private final VirtualColumns virtualColumns = VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java b/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java index 21e5626cae72..fada2acaa4bc 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -77,13 +76,11 @@ public void testSerde_LoadableDataSegment() throws Exception new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java index 4c58d551509d..35b325d7ae30 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java @@ -454,13 +454,11 @@ public void testSerdeTransformSpec() throws IOException new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ) diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index 84d8021700aa..3ce6c4b418bc 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -104,13 +104,11 @@ public void testSerde() throws JsonProcessingException new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null), VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "isRobotFiltered", - "concat(isRobot, '_filtered')", - ColumnType.STRING, - ExprMacroTable.nil() - ) + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() ) ) ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index a4b99952fd06..2ca95310e1cc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -1555,10 +1555,8 @@ public void testScalarInArrayFilter() .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( VirtualColumns.create( - ImmutableList.of( - expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ColumnType.STRING), - expressionVirtualColumn("v1", "(\"cnt\" * 2)", ColumnType.LONG) - ) + expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ColumnType.STRING), + expressionVirtualColumn("v1", "(\"cnt\" * 2)", ColumnType.LONG) ) ) .filters( From 17122434314f673a93041ac4ce85229a24b3287f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 26 Feb 2026 13:29:51 -0800 Subject: [PATCH 02/12] fix lost line in refactor, suppress checkstyle for now --- .../java/org/apache/druid/msq/indexing/MSQCompactionRunner.java | 1 + pom.xml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 36853137092e..0f377145f3b9 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -452,6 +452,7 @@ private static List getAggregateDimensions( ColumnType colType = schema.getColumnType(); VirtualColumn vc = inputColToVirtualCol.get(dimension); if (vc != null) { + dimension = vc.getOutputName(); if (vc instanceof ExpressionVirtualColumn) { colType = ((ExpressionVirtualColumn) vc).getOutputType(); } else { diff --git a/pom.xml b/pom.xml index e0e7cdb73c85..2dc4bd04fb9c 100644 --- a/pom.xml +++ b/pom.xml @@ -1587,7 +1587,7 @@ true true - *com/fasterxml/jackson/databind/*,**/NestedDataFormatsTest.java + *com/fasterxml/jackson/databind/*,**/NestedDataFormatsTest.java,**/CompactionSupervisorTest.java,**/MultiStageQueryTest.java From d5d63c753cf5c3216081ebd0dc9797deb8c72876 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 26 Feb 2026 15:41:10 -0800 Subject: [PATCH 03/12] fixes --- .../apache/druid/msq/indexing/MSQCompactionRunnerTest.java | 2 +- .../partition/DimensionRangeBucketShardSpecTest.java | 5 ++++- .../druid/client/indexing/ClientCompactionRunnerInfo.java | 5 ----- .../client/indexing/ClientCompactionRunnerInfoTest.java | 2 +- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index a906f3dc0016..e8456317fed3 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -230,7 +230,7 @@ public void testLongDimensionInRangePartitionsSpecIsInvalid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "MSQ: Non-string partition dimension[long_dim] of type[long] not supported with 'range' partition spec", + "MSQ: Non-string partition dimension[long_dim] of type[LONG] not supported with 'range' partition spec", validationResult.getReason() ); } diff --git a/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpecTest.java b/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpecTest.java index f4a2d88b5cd9..7eb607ecffb8 100644 --- a/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpecTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeBucketShardSpecTest.java @@ -198,6 +198,9 @@ public void testInvalidEndTupleSize() @Test public void testEquals() { - EqualsVerifier.forClass(DimensionRangeBucketShardSpec.class).usingGetClass().verify(); + EqualsVerifier.forClass(DimensionRangeBucketShardSpec.class) + .usingGetClass() + .withIgnoredFields("virtualColumns") + .verify(); } } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index a77fa1f8c553..c66def987934 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -43,8 +43,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.function.Function; -import java.util.stream.Collectors; /** @@ -171,9 +169,6 @@ public static CompactionConfigValidationResult validatePartitionsSpecForMSQ( ); } if (partitionsSpec instanceof DimensionRangePartitionsSpec && dimensionSchemas != null) { - Map dimensionSchemaMap = dimensionSchemas.stream().collect( - Collectors.toMap(DimensionSchema::getName, Function.identity()) - ); RowSignature.Builder baseSignatureBuilder = RowSignature.builder().addTimeColumn(); for (DimensionSchema dimensionSchema : dimensionSchemas) { baseSignatureBuilder.add(dimensionSchema.getName(), dimensionSchema.getColumnType()); diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index f0d35ef324b3..2730c98899e6 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -161,7 +161,7 @@ public void testMSQEngineWithLongDimensionsInRangePartitionsSpecIsInvalid() ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "MSQ: Non-string partition dimension[partitionDim] of type[long] not supported with 'range' partition spec", + "MSQ: Non-string partition dimension[partitionDim] of type[LONG] not supported with 'range' partition spec", validationResult.getReason() ); } From e2054193b342f37e15148cc8c6cf7fad4b6d336f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 13 Mar 2026 17:05:51 -0700 Subject: [PATCH 04/12] rework stuff, virtual column map now lives on segment generator processor --- .../compact/CompactionSupervisorTest.java | 5 +- .../apache/druid/msq/exec/ControllerImpl.java | 58 +++++++++++-------- .../msq/exec/QueryKitBasedMSQPlanner.java | 2 - .../msq/indexing/MSQCompactionRunner.java | 4 +- .../SegmentGenerationStageSpec.java | 51 +++++++++++++--- .../SegmentGeneratorStageProcessor.java | 13 +++++ .../druid/msq/logical/stages/JoinStage.java | 2 +- .../druid/msq/querykit/QueryKitUtils.java | 2 +- .../msq/querykit/groupby/GroupByQueryKit.java | 19 +----- .../druid/msq/querykit/scan/ScanQueryKit.java | 15 +---- .../druid/msq/sql/MSQTaskQueryMaker.java | 9 ++- .../org/apache/druid/frame/key/ClusterBy.java | 27 +-------- 12 files changed, 105 insertions(+), 102 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index e58b86e0f1a4..b611df969c24 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -50,6 +50,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.NotDimFilter; @@ -670,6 +671,7 @@ public void test_compaction_cluster_by_virtualcolumn_rollup() .inlineInputSourceWithData(jsonDataWithNestedColumn) .isoTimestampColumn("timestamp") .schemaDiscovery() + .dataSchema(builder -> builder.withAggregators(new CountAggregatorFactory("count"))) .granularitySpec("DAY", "MINUTE", true); cluster.callApi().runTask(task.withId(IdUtils.getRandomId()), overlord); @@ -727,7 +729,6 @@ public void test_compaction_cluster_by_virtualcolumn_rollup() cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - List segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord).stream().toList(); Assertions.assertEquals(2, segments.size()); Assertions.assertEquals( @@ -888,7 +889,7 @@ private void configureCompaction(CompactionEngine compactionEngine, @Nullable Co ); Assertions.assertTrue(updateResponse.isSuccess()); } - + private void waitUntilPublishedRecordsAreIngested(int expectedRowCount) { indexer.latchableEmitter().waitForEventAggregate( diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 24adfefadeec..b75590c33135 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1064,6 +1064,7 @@ private List generateSegmentIdsWithShardSpecs( final ClusterBy clusterBy, final RowKeyReader keyReader, final ClusterByPartitions partitionBoundaries, + final Map clusterByVirtualColumnMappings, final boolean mayHaveMultiValuedClusterByFields, @Nullable final Boolean isStageOutputEmpty ) throws IOException @@ -1075,6 +1076,7 @@ private List generateSegmentIdsWithShardSpecs( clusterBy, keyReader, partitionBoundaries, + clusterByVirtualColumnMappings, mayHaveMultiValuedClusterByFields, isStageOutputEmpty ); @@ -1242,6 +1244,7 @@ private List generateSegmentIdsWithShardSpecsForReplace( final ClusterBy clusterBy, final RowKeyReader keyReader, final ClusterByPartitions partitionBoundaries, + final Map clusterByVirtualColumnMappings, final boolean mayHaveMultiValuedClusterByFields, @Nullable final Boolean isStageOutputEmpty ) throws IOException @@ -1256,6 +1259,7 @@ private List generateSegmentIdsWithShardSpecsForReplace( signature, clusterBy, querySpec.getColumnMappings(), + clusterByVirtualColumnMappings, mayHaveMultiValuedClusterByFields ); final List shardColumns = shardReasonPair.lhs; @@ -1318,7 +1322,7 @@ private List generateSegmentIdsWithShardSpecsForReplace( : makeStringTuple(clusterBy, keyReader, range.getEnd(), shardColumns.size()); shardSpec = new DimensionRangeShardSpec( shardColumns, - VirtualColumns.create(clusterBy.getVirtualColumnMap().values()), + VirtualColumns.create(clusterByVirtualColumnMappings.values()), start, end, segmentNumber, @@ -1718,19 +1722,13 @@ private void handleQueryResults( queryDef.getQueryId() ); } else { - DataSchema dataSchema = ((SegmentGeneratorStageProcessor) queryKernel - .getStageDefinition(finalStageId).getProcessor()).getDataSchema(); - - ShardSpec shardSpec = segments.isEmpty() ? null : segments.stream().findFirst().get().getShardSpec(); - ClusterBy clusterBy = queryKernel.getStageDefinition(finalStageId).getClusterBy(); + final ShardSpec shardSpec = segments.isEmpty() ? null : segments.stream().findFirst().get().getShardSpec(); compactionStateAnnotateFunction = addCompactionStateToSegments( + queryDef, querySpec, context.jsonMapper(), - dataSchema, - shardSpec, - clusterBy, - queryDef.getQueryId() + shardSpec ); } } @@ -1769,17 +1767,21 @@ private void handleQueryResults( } private static Function, Set> addCompactionStateToSegments( + QueryDefinition queryDef, MSQSpec querySpec, ObjectMapper jsonMapper, - DataSchema dataSchema, - @Nullable ShardSpec shardSpec, - @Nullable ClusterBy clusterBy, - String queryId + @Nullable ShardSpec shardSpec ) { + final ClusterBy clusterBy = queryDef.getFinalStageDefinition().getClusterBy(); final MSQTuningConfig tuningConfig = querySpec.getTuningConfig(); - PartitionsSpec partitionSpec; + final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); + final SegmentGeneratorStageProcessor segmentProcessor = + (SegmentGeneratorStageProcessor) queryDef.getFinalStageDefinition().getProcessor(); + final DataSchema dataSchema = segmentProcessor.getDataSchema(); + + final PartitionsSpec partitionSpec; // shardSpec is absent in the absence of segments, which happens when only tombstones are generated by an // MSQControllerTask. if (shardSpec != null) { @@ -1801,7 +1803,7 @@ private static Function, Set> addCompactionStateTo UnknownFault.forMessage( StringUtils.format( "Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].", - queryId, + queryDef.getQueryId(), shardSpec.getType() ))); } @@ -1819,30 +1821,32 @@ private static Function, Set> addCompactionStateTo partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); } - Granularity segmentGranularity = ((DataSourceMSQDestination) querySpec.getDestination()) - .getSegmentGranularity(); + Granularity segmentGranularity = destination.getSegmentGranularity(); GranularitySpec granularitySpec = new UniformGranularitySpec( segmentGranularity, - querySpec.getContext() - .getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), + querySpec.getContext().getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper), dataSchema.getGranularitySpec().isRollup(), // Not using dataSchema.getGranularitySpec().inputIntervals() as that always has ETERNITY - ((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks() + destination.getReplaceTimeChunks() ); DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); // if the clustered by requires virtual columns, preserve them here so that we can rebuild during compaction CompactionTransformSpec transformSpec; - if (clusterBy == null || clusterBy.getVirtualColumnMap().isEmpty()) { + // this is true if we are in here + + final Map clusterByVirtualColumnMappings = segmentProcessor.getClusterByVirtualColumnMappings(); + + if (clusterBy == null || clusterByVirtualColumnMappings.isEmpty()) { transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) ? null : CompactionTransformSpec.of(dataSchema.getTransformSpec()); } else { transformSpec = new CompactionTransformSpec( dataSchema.getTransformSpec().getFilter(), - VirtualColumns.create(clusterBy.getVirtualColumnMap().values()) + VirtualColumns.create(clusterByVirtualColumnMappings.values()) ); } @@ -1850,7 +1854,7 @@ private static Function, Set> addCompactionStateTo IndexSpec indexSpec = tuningConfig.getIndexSpec(); - log.info("Query[%s] storing compaction state in segments.", queryId); + log.info("Query[%s] storing compaction state in segments.", queryDef.getQueryId()); return CompactionState.addCompactionStateToSegments( partitionSpec, @@ -1928,6 +1932,7 @@ private static Pair, String> computeShardColumns( final RowSignature signature, final ClusterBy clusterBy, final ColumnMappings columnMappings, + final Map clusterByVirtualColumns, boolean mayHaveMultiValuedClusterByFields ) { @@ -1980,7 +1985,7 @@ private static Pair, String> computeShardColumns( // DimensionRangeShardSpec columns may either be explicitly in the table or defined as virtual columns if (outputColumns.isEmpty()) { - final VirtualColumn vc = clusterBy.getVirtualColumnMap().get(column.columnName()); + final VirtualColumn vc = clusterByVirtualColumns.get(column.columnName()); if (vc != null) { shardColumns.add(vc.getOutputName()); } else { @@ -2657,6 +2662,8 @@ private void populateSegmentsToGenerate() throws IOException final boolean mayHaveMultiValuedClusterByFields = !shuffleStageDef.mustGatherResultKeyStatistics() || queryKernel.hasStageCollectorEncounteredAnyMultiValueField(shuffleStageId); + final SegmentGeneratorStageProcessor segmentGeneratorStageProcessor = + (SegmentGeneratorStageProcessor) queryDef.getFinalStageDefinition().getProcessor(); segmentsToGenerate = generateSegmentIdsWithShardSpecs( (DataSourceMSQDestination) querySpec.getDestination(), @@ -2664,6 +2671,7 @@ private void populateSegmentsToGenerate() throws IOException shuffleStageDef.getClusterBy(), shuffleStageDef.getClusterBy().keyReader(shuffleStageDef.getSignature(), rowBasedFrameType), partitionBoundaries, + segmentGeneratorStageProcessor.getClusterByVirtualColumnMappings(), mayHaveMultiValuedClusterByFields, isShuffleStageOutputEmpty ); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java index 7ec3f24e3de2..6a9cfab6793a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java @@ -161,7 +161,6 @@ private QueryDefinition makeQueryDefinitionInternal(final Query queryToPlan) // Add all query stages. // Set shuffleCheckHasMultipleValues on the stage that serves as input to the final segment-generation stage. final QueryDefinitionBuilder builder = QueryDefinition.builder(queryKitSpec.getQueryId()); - for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { if (stageDef.equals(finalShuffleStageDef)) { builder.add(StageDefinition.builder(stageDef).shuffleCheckHasMultipleValues(true)); @@ -169,7 +168,6 @@ private QueryDefinition makeQueryDefinitionInternal(final Query queryToPlan) builder.add(StageDefinition.builder(stageDef)); } } - final DataSourceMSQDestination destination1 = (DataSourceMSQDestination) destination; return builder.add( destination1.getTerminalStageSpec() diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 61baf0bad4df..46f30295ea5a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -67,9 +67,8 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.policy.PolicyEnforcer; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.segment.ColumnInspector; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -740,6 +739,7 @@ private Query buildGroupByQuery( .setDimensions(getAggregateDimensions(dataSchema, inputColToVirtualCol, orderBy)) .setAggregatorSpecs(dataSchema.getAggregators()) .setPostAggregatorSpecs(postAggregators) + .setOrderByColumns(orderBy) .setContext(buildQueryContext(compactionTask.getContext(), dataSchema)); return builder.build(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java index 269a118f6d88..5d133a4cf447 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java @@ -25,6 +25,7 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.processor.SegmentGeneratorStageProcessor; @@ -35,7 +36,12 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageDefinitionBuilder; import org.apache.druid.msq.kernel.controller.WorkerInputs; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -43,7 +49,9 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; public class SegmentGenerationStageSpec implements TerminalStageSpec { @@ -73,16 +81,41 @@ public StageDefinitionBuilder constructFinalStage(QueryDefinition queryDef, MSQS final DataSchema dataSchema = SegmentGenerationUtils.makeDataSchemaForIngestion(querySpec, querySignature, queryClusterBy, columnMappings, jsonMapper, query); + final Map clusterByVirtualColumns = new LinkedHashMap<>(); + if (query instanceof GroupByQuery groupByQuery) { + final Map outputToVc = new LinkedHashMap<>(); + for (DimensionSpec spec : groupByQuery.getDimensions()) { + final VirtualColumn vc = groupByQuery.getVirtualColumns().getVirtualColumn(spec.getDimension()); + if (vc != null) { + outputToVc.put(spec.getOutputName(), vc); + } + } + for (KeyColumn column : queryClusterBy.getColumns()) { + final VirtualColumn vc = outputToVc.get(column.columnName()); + if (vc != null) { + clusterByVirtualColumns.put(column.columnName(), vc); + } + } + } else if (query instanceof ScanQuery scanQuery) { + for (OrderBy orderBy : scanQuery.getOrderBys()) { + final VirtualColumn vc = scanQuery.getVirtualColumns().getVirtualColumn(orderBy.getColumnName()); + if (vc != null) { + clusterByVirtualColumns.put(orderBy.getColumnName(), vc); + } + } + } + return StageDefinition.builder(queryDef.getNextStageNumber()) - .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) - .maxWorkerCount(tuningConfig.getMaxNumWorkers()) - .processor( - new SegmentGeneratorStageProcessor( - dataSchema, - columnMappings, - tuningConfig - ) - ); + .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) + .maxWorkerCount(tuningConfig.getMaxNumWorkers()) + .processor( + new SegmentGeneratorStageProcessor( + dataSchema, + columnMappings, + clusterByVirtualColumns, + tuningConfig + ) + ); } public Int2ObjectMap> getWorkerInfo( diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java index fe9418ee3fb5..25bd70fed8de 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing.processor; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.type.TypeReference; @@ -51,6 +52,7 @@ import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.AppendableIndexSpec; @@ -72,6 +74,7 @@ import java.io.File; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -80,17 +83,20 @@ public class SegmentGeneratorStageProcessor implements StageProcessor clusterByVirtualColumnMappings; private final MSQTuningConfig tuningConfig; @JsonCreator public SegmentGeneratorStageProcessor( @JsonProperty("dataSchema") final DataSchema dataSchema, @JsonProperty("columnMappings") final ColumnMappings columnMappings, + @JsonProperty("clusterByVirtualColumnsMappings") @Nullable final Map clusterByVirtualColumnMappings, @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.columnMappings = Preconditions.checkNotNull(columnMappings, "columnMappings"); + this.clusterByVirtualColumnMappings = clusterByVirtualColumnMappings == null ? Map.of() : clusterByVirtualColumnMappings; this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); } @@ -106,6 +112,13 @@ public ColumnMappings getColumnMappings() return columnMappings; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public Map getClusterByVirtualColumnMappings() + { + return clusterByVirtualColumnMappings; + } + @JsonProperty public MSQTuningConfig getTuningConfig() { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java index 42cd3fe4b7e6..3ed232d0bd32 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java @@ -76,7 +76,7 @@ public RowSignature getLogicalRowSignature() @Override public ShuffleSpec buildShuffleSpec() { - final ClusterBy clusterBy = new ClusterBy(keyColumns, Map.of(), 0); + final ClusterBy clusterBy = new ClusterBy(keyColumns, 0); return new HashShuffleSpec(clusterBy, 1); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java index 6f28c9f04931..478a7cfb554c 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java @@ -122,7 +122,7 @@ public static ClusterBy clusterByWithSegmentGranularity( final List newColumns = new ArrayList<>(clusterBy.getColumns().size() + 1); newColumns.add(new KeyColumn(QueryKitUtils.SEGMENT_GRANULARITY_COLUMN, KeyOrder.ASCENDING)); newColumns.addAll(clusterBy.getColumns()); - return new ClusterBy(newColumns, clusterBy.getVirtualColumnMap(), 1); + return new ClusterBy(newColumns, 1); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java index 2b96bd81c26a..b3f9a6beafb1 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java @@ -49,14 +49,11 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import java.util.ArrayList; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.Optional; public class GroupByQueryKit implements QueryKit @@ -270,7 +267,6 @@ private static ClusterBy computeResultClusterBy( )); return new ClusterBy( resultClusterByWithPartitionBoostColumns, - resultClusterByWithoutPartitionBoost.getVirtualColumnMap(), resultClusterByWithoutPartitionBoost.getBucketByCount() ); } @@ -338,8 +334,6 @@ static ClusterBy computeClusterByForResults(final GroupByQuery query) if (!defaultLimitSpec.getColumns().isEmpty()) { final List clusterByColumns = new ArrayList<>(); - final Map clusterByVirtualColumnMap = new LinkedHashMap<>(); - for (final OrderByColumnSpec orderBy : defaultLimitSpec.getColumns()) { clusterByColumns.add( new KeyColumn( @@ -349,20 +343,9 @@ static ClusterBy computeClusterByForResults(final GroupByQuery query) : KeyOrder.ASCENDING ) ); - - // check for virtual columns - final DimensionSpec dim = query.getDimensions() - .stream() - .filter(x -> x.getOutputName().equals(orderBy.getDimension())) - .findFirst() - .orElse(null); - final VirtualColumn vc = dim == null ? null : query.getVirtualColumns().getVirtualColumn(dim.getDimension()); - if (vc != null) { - clusterByVirtualColumnMap.put(orderBy.getDimension(), vc); - } } - return new ClusterBy(clusterByColumns, clusterByVirtualColumnMap, 0); + return new ClusterBy(clusterByColumns, 0); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index eb87c7a85b4a..3824ba971133 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -42,14 +42,11 @@ import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import java.util.ArrayList; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; public class ScanQueryKit implements QueryKit { @@ -106,8 +103,6 @@ public QueryDefinition makeQueryDefinition( final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, queryToRun.getContext()); final List clusterByColumns = new ArrayList<>(); - final Map clusterByVirtualColumnMap = new LinkedHashMap<>(); - // Add regular orderBys. for (final OrderBy orderBy : queryToRun.getOrderBys()) { clusterByColumns.add( @@ -116,21 +111,13 @@ public QueryDefinition makeQueryDefinition( orderBy.getOrder() == Order.DESCENDING ? KeyOrder.DESCENDING : KeyOrder.ASCENDING ) ); - VirtualColumn vc = queryToRun.getVirtualColumns().getVirtualColumn(orderBy.getColumnName()); - if (vc != null) { - clusterByVirtualColumnMap.put(orderBy.getColumnName(), vc); - } } clusterByColumns.add(new KeyColumn(QueryKitUtils.PARTITION_BOOST_COLUMN, KeyOrder.ASCENDING)); signatureBuilder.add(QueryKitUtils.PARTITION_BOOST_COLUMN, ColumnType.LONG); final ClusterBy clusterBy = QueryKitUtils.clusterByWithSegmentGranularity( - new ClusterBy( - clusterByColumns, - clusterByVirtualColumnMap, - 0 - ), + new ClusterBy(clusterByColumns, 0), segmentGranularity ); final ShuffleSpec finalShuffleSpec = resultShuffleSpecFactory.build(clusterBy, false); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index 0032a6051ac2..c8b9546f0177 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -211,9 +211,12 @@ public static LegacyMSQSpec makeLegacyMSQSpec( return querySpec; } - private static MSQDestination buildMSQDestination(final IngestDestination targetDataSource, - final ColumnMappings columnMappings, final PlannerContext plannerContext, - final MSQTerminalStageSpecFactory terminalStageSpecFactory) + private static MSQDestination buildMSQDestination( + final IngestDestination targetDataSource, + final ColumnMappings columnMappings, + final PlannerContext plannerContext, + final MSQTerminalStageSpecFactory terminalStageSpecFactory + ) { final QueryContext sqlQueryContext = plannerContext.queryContext(); final Object segmentGranularity = getSegmentGranularity(plannerContext); diff --git a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java index fb90cfe471bf..07eed3205262 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java +++ b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java @@ -27,16 +27,13 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.segment.ColumnInspector; -import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import javax.annotation.Nullable; import java.util.Collections; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.Objects; /** @@ -48,27 +45,16 @@ public class ClusterBy { private final List columns; - private final Map virtualColumnMap; private final int bucketByCount; private final boolean sortable; - public ClusterBy( - List keyColumns, - int bucketByCount - ) - { - this(keyColumns, Map.of(), bucketByCount); - } - @JsonCreator public ClusterBy( @JsonProperty("columns") List columns, - @JsonProperty("virtualColumnMap") @Nullable Map virtualColumnMap, @JsonProperty("bucketByCount") int bucketByCount ) { this.columns = Preconditions.checkNotNull(columns, "columns"); - this.virtualColumnMap = virtualColumnMap == null ? Map.of() : virtualColumnMap; this.bucketByCount = bucketByCount; if (bucketByCount < 0 || bucketByCount > columns.size()) { @@ -105,13 +91,6 @@ public List getColumns() return columns; } - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_EMPTY) - public Map getVirtualColumnMap() - { - return virtualColumnMap; - } - /** * How many fields from {@link #getColumns()} comprise the "bucket key". Bucketing is like strict partitioning: all * rows in a given partition will have the exact same bucket key. It is most commonly used to implement @@ -208,14 +187,13 @@ public boolean equals(Object o) } ClusterBy clusterBy = (ClusterBy) o; return bucketByCount == clusterBy.bucketByCount && - Objects.equals(columns, clusterBy.columns) && - Objects.equals(virtualColumnMap, clusterBy.virtualColumnMap); + Objects.equals(columns, clusterBy.columns); } @Override public int hashCode() { - return Objects.hash(columns, virtualColumnMap, bucketByCount); + return Objects.hash(columns, bucketByCount); } @Override @@ -223,7 +201,6 @@ public String toString() { return "ClusterBy{" + "columns=" + columns + - ", virtualColumns=" + virtualColumnMap + ", bucketByCount=" + bucketByCount + '}'; } From c744f2ddf046cab60cbf257e8d166de7977d4741 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 13 Mar 2026 17:12:02 -0700 Subject: [PATCH 05/12] tidy up --- .../SegmentGenerationStageSpec.java | 69 +++++++++++-------- 1 file changed, 42 insertions(+), 27 deletions(-) diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java index 5d133a4cf447..58661dd82ced 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java @@ -78,32 +78,19 @@ public StageDefinitionBuilder constructFinalStage(QueryDefinition queryDef, MSQS final ClusterBy queryClusterBy = queryDef.getFinalStageDefinition().getClusterBy(); // Add a segment-generation stage. - final DataSchema dataSchema = - SegmentGenerationUtils.makeDataSchemaForIngestion(querySpec, querySignature, queryClusterBy, columnMappings, jsonMapper, query); - - final Map clusterByVirtualColumns = new LinkedHashMap<>(); - if (query instanceof GroupByQuery groupByQuery) { - final Map outputToVc = new LinkedHashMap<>(); - for (DimensionSpec spec : groupByQuery.getDimensions()) { - final VirtualColumn vc = groupByQuery.getVirtualColumns().getVirtualColumn(spec.getDimension()); - if (vc != null) { - outputToVc.put(spec.getOutputName(), vc); - } - } - for (KeyColumn column : queryClusterBy.getColumns()) { - final VirtualColumn vc = outputToVc.get(column.columnName()); - if (vc != null) { - clusterByVirtualColumns.put(column.columnName(), vc); - } - } - } else if (query instanceof ScanQuery scanQuery) { - for (OrderBy orderBy : scanQuery.getOrderBys()) { - final VirtualColumn vc = scanQuery.getVirtualColumns().getVirtualColumn(orderBy.getColumnName()); - if (vc != null) { - clusterByVirtualColumns.put(orderBy.getColumnName(), vc); - } - } - } + final DataSchema dataSchema = SegmentGenerationUtils.makeDataSchemaForIngestion( + querySpec, + querySignature, + queryClusterBy, + columnMappings, + jsonMapper, + query + ); + + final Map clusterByVirtualColumnMappings = getClusterByVirtualColumnMappings( + query, + queryClusterBy + ); return StageDefinition.builder(queryDef.getNextStageNumber()) .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) @@ -112,7 +99,7 @@ public StageDefinitionBuilder constructFinalStage(QueryDefinition queryDef, MSQS new SegmentGeneratorStageProcessor( dataSchema, columnMappings, - clusterByVirtualColumns, + clusterByVirtualColumnMappings, tuningConfig ) ); @@ -146,4 +133,32 @@ public Int2ObjectMap> getWorkerInfo( return retVal; } + + private static Map getClusterByVirtualColumnMappings(Query query, ClusterBy queryClusterBy) + { + final Map clusterByVirtualColumns = new LinkedHashMap<>(); + if (query instanceof GroupByQuery groupByQuery) { + final Map outputToVc = new LinkedHashMap<>(); + for (DimensionSpec spec : groupByQuery.getDimensions()) { + final VirtualColumn vc = groupByQuery.getVirtualColumns().getVirtualColumn(spec.getDimension()); + if (vc != null) { + outputToVc.put(spec.getOutputName(), vc); + } + } + for (KeyColumn column : queryClusterBy.getColumns()) { + final VirtualColumn vc = outputToVc.get(column.columnName()); + if (vc != null) { + clusterByVirtualColumns.put(column.columnName(), vc); + } + } + } else if (query instanceof ScanQuery scanQuery) { + for (OrderBy orderBy : scanQuery.getOrderBys()) { + final VirtualColumn vc = scanQuery.getVirtualColumns().getVirtualColumn(orderBy.getColumnName()); + if (vc != null) { + clusterByVirtualColumns.put(orderBy.getColumnName(), vc); + } + } + } + return clusterByVirtualColumns; + } } From 0aaaf6189e22e38d58e9c63b7ccd04d1f4b62347 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 13 Mar 2026 19:31:08 -0700 Subject: [PATCH 06/12] fix style --- .../main/java/org/apache/druid/msq/logical/stages/JoinStage.java | 1 - 1 file changed, 1 deletion(-) diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java index 3ed232d0bd32..96cf3d3510b6 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/JoinStage.java @@ -46,7 +46,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Map; /** * Represents a stage that reads data from input sources. From 0a7dca026de065b6c232f2110c57e731fc58793d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 13 Mar 2026 19:40:43 -0700 Subject: [PATCH 07/12] fix test --- .../apache/druid/msq/exec/ControllerImpl.java | 6 +- .../SegmentGenerationStageSpec.java | 7 +-- .../apache/druid/msq/exec/MSQReplaceTest.java | 55 ++++++++++++++++++- 3 files changed, 61 insertions(+), 7 deletions(-) diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index b75590c33135..ad180e8af671 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -61,6 +61,7 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLock; @@ -1837,9 +1838,10 @@ private static Function, Set> addCompactionStateTo CompactionTransformSpec transformSpec; // this is true if we are in here - final Map clusterByVirtualColumnMappings = segmentProcessor.getClusterByVirtualColumnMappings(); + final Map clusterByVirtualColumnMappings = + segmentProcessor.getClusterByVirtualColumnMappings(); - if (clusterBy == null || clusterByVirtualColumnMappings.isEmpty()) { + if (clusterByVirtualColumnMappings.isEmpty() || !SecondaryPartitionType.RANGE.equals(partitionSpec.getType())) { transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) ? null : CompactionTransformSpec.of(dataSchema.getTransformSpec()); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java index 58661dd82ced..7898cd8ab6a9 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationStageSpec.java @@ -36,7 +36,6 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageDefinitionBuilder; import org.apache.druid.msq.kernel.controller.WorkerInputs; -import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; @@ -152,10 +151,10 @@ private static Map getClusterByVirtualColumnMappings(Quer } } } else if (query instanceof ScanQuery scanQuery) { - for (OrderBy orderBy : scanQuery.getOrderBys()) { - final VirtualColumn vc = scanQuery.getVirtualColumns().getVirtualColumn(orderBy.getColumnName()); + for (KeyColumn column : queryClusterBy.getColumns()) { + final VirtualColumn vc = scanQuery.getVirtualColumns().getVirtualColumn(column.columnName()); if (vc != null) { - clusterByVirtualColumns.put(orderBy.getColumnName(), vc); + clusterByVirtualColumns.put(column.columnName(), vc); } } } diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 2875a1749e89..fdf58b3e8747 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -56,13 +56,16 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.RangeFilter; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -357,6 +360,17 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName ) .setDimensionExclusions(Collections.singletonList("__time")) .build(), + new CompactionTransformSpec( + null, + VirtualColumns.create( + new ExpressionVirtualColumn( + "v0", + "lower(\"dim1\")", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ), GranularityType.ALL, Intervals.ETERNITY ) @@ -2813,6 +2827,25 @@ private CompactionState expectedCompactionState( GranularityType segmentGranularity, Interval interval ) + { + return expectedCompactionState( + context, + partitionDimensions, + dimensions, + null, + segmentGranularity, + interval + ); + } + + private CompactionState expectedCompactionState( + Map context, + List partitionDimensions, + List dimensions, + CompactionTransformSpec transformSpec, + GranularityType segmentGranularity, + Interval interval + ) { return expectedCompactionState( context, @@ -2821,6 +2854,7 @@ private CompactionState expectedCompactionState( .setDimensions(dimensions) .setDimensionExclusions(Collections.singletonList("__time")) .build(), + transformSpec, segmentGranularity, interval ); @@ -2833,6 +2867,25 @@ private CompactionState expectedCompactionState( GranularityType segmentGranularity, Interval interval ) + { + return expectedCompactionState( + context, + partitionDimensions, + dimensionsSpec, + null, + segmentGranularity, + interval + ); + } + + private CompactionState expectedCompactionState( + Map context, + List partitionDimensions, + DimensionsSpec dimensionsSpec, + CompactionTransformSpec transformSpec, + GranularityType segmentGranularity, + Interval interval + ) { if (!context.containsKey(Tasks.STORE_COMPACTION_STATE_KEY) || !((Boolean) context.get(Tasks.STORE_COMPACTION_STATE_KEY))) { @@ -2866,7 +2919,7 @@ private CompactionState expectedCompactionState( partitionsSpec, dimensionsSpec, metricsSpec, - null, + transformSpec, indexSpec, granularitySpec, null From dc7587cec47ebd7b1cd77bd7494cd996215efd22 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 16 Mar 2026 01:18:25 -0700 Subject: [PATCH 08/12] review stuff --- .../apache/druid/msq/exec/ControllerImpl.java | 3 +- .../msq/indexing/MSQCompactionRunner.java | 3 - .../SegmentGeneratorStageProcessor.java | 5 +- .../SegmentGeneratorStageProcessorTest.java | 85 +++++++++++++++++++ .../org/apache/druid/frame/key/ClusterBy.java | 6 +- .../groupby/orderby/DefaultLimitSpec.java | 4 - .../druid/timeline/partition/ShardSpec.java | 14 ++- .../query/filter/FilterSegmentPrunerTest.java | 40 +++++++-- .../druid/segment/indexing/DataSchema.java | 72 +++++++++++----- .../segment/indexing/DataSchemaTest.java | 14 +++ 10 files changed, 201 insertions(+), 45 deletions(-) create mode 100644 multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index ad180e8af671..6a064c82c5dd 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1836,11 +1836,10 @@ private static Function, Set> addCompactionStateTo // if the clustered by requires virtual columns, preserve them here so that we can rebuild during compaction CompactionTransformSpec transformSpec; - // this is true if we are in here - final Map clusterByVirtualColumnMappings = segmentProcessor.getClusterByVirtualColumnMappings(); + // only range partitioning can have virtual columns if (clusterByVirtualColumnMappings.isEmpty() || !SecondaryPartitionType.RANGE.equals(partitionSpec.getType())) { transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) ? null diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 46f30295ea5a..8b4cd39879b2 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -579,9 +579,6 @@ private Query buildScanQuery( Druids.ScanQueryBuilder scanQueryBuilder = new Druids.ScanQueryBuilder() .dataSource(getInputDataSource(dataSchema.getDataSource())) - .columns(columns) - .virtualColumns(VirtualColumns.create(inputColToVirtualCol.values())) - .columnTypes(rowSignatureWithOrderByBuilder.build().getColumnTypes()) .intervals(segmentSpec) .filters(dataSchema.getTransformSpec().getFilter()) .virtualColumns(VirtualColumns.create(inputColToVirtualCol.values())) diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java index 25bd70fed8de..c80c03268fc3 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java @@ -90,7 +90,7 @@ public class SegmentGeneratorStageProcessor implements StageProcessor clusterByVirtualColumnMappings, + @JsonProperty("clusterByVirtualColumnMappings") @Nullable final Map clusterByVirtualColumnMappings, @JsonProperty("tuningConfig") final MSQTuningConfig tuningConfig ) { @@ -268,13 +268,14 @@ public boolean equals(Object o) SegmentGeneratorStageProcessor that = (SegmentGeneratorStageProcessor) o; return Objects.equals(dataSchema, that.dataSchema) && Objects.equals(columnMappings, that.columnMappings) + && Objects.equals(clusterByVirtualColumnMappings, that.clusterByVirtualColumnMappings) && Objects.equals(tuningConfig, that.tuningConfig); } @Override public int hashCode() { - return Objects.hash(dataSchema, columnMappings, tuningConfig); + return Objects.hash(dataSchema, columnMappings, clusterByVirtualColumnMappings, tuningConfig); } @Override diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java new file mode 100644 index 000000000000..4fddecfff217 --- /dev/null +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java @@ -0,0 +1,85 @@ +/* + * 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. + */ + +package org.apache.druid.msq.indexing.processor; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.msq.exec.StageProcessor; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.sql.calcite.planner.ColumnMapping; +import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; + +class SegmentGeneratorStageProcessorTest +{ + + @Test + public void testSerde() throws JsonProcessingException + { + ObjectMapper mapper = TestHelper.makeJsonMapper(); + mapper.registerModules(new MSQIndexingModule().getJacksonModules()); + SegmentGeneratorStageProcessor processor = new SegmentGeneratorStageProcessor( + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("timestamp", "auto", null)) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null)) + .withDimensions(new StringDimensionSchema("a"), new StringDimensionSchema("b")) + .withAggregators(new CountAggregatorFactory("cnt")) + .build(), + new ColumnMappings( + List.of( + new ColumnMapping("d0", "__time"), + new ColumnMapping("d1", "a"), + new ColumnMapping("d2", "b"), + new ColumnMapping("a0", "cnt") + ) + ), + Map.of( + "v0", + new ExpressionVirtualColumn("v0", "concat(\"a\",'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ), + MSQTuningConfig.defaultConfig() + ); + + Assertions.assertEquals(processor, mapper.readValue(mapper.writeValueAsString(processor), StageProcessor.class)); + } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(SegmentGeneratorStageProcessor.class).usingGetClass().verify(); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java index 07eed3205262..44e9151692a0 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java +++ b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java @@ -63,14 +63,17 @@ public ClusterBy( // Key must be 100% sortable or 100% nonsortable. If empty, call it sortable. boolean sortable = true; + for (int i = 0; i < columns.size(); i++) { final KeyColumn column = columns.get(i); + if (i == 0) { sortable = column.order().sortable(); } else if (sortable != column.order().sortable()) { throw new IAE("Cannot mix sortable and unsortable key columns"); } } + this.sortable = sortable; } @@ -186,8 +189,7 @@ public boolean equals(Object o) return false; } ClusterBy clusterBy = (ClusterBy) o; - return bucketByCount == clusterBy.bucketByCount && - Objects.equals(columns, clusterBy.columns); + return bucketByCount == clusterBy.bucketByCount && Objects.equals(columns, clusterBy.columns); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java index 88f48af16f1a..0c04839c3d2c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -203,10 +203,6 @@ public Function, Sequence> build(final GroupByQue sortingNeeded = true; break; } - if (query.getVirtualColumns().getVirtualColumn(columnSpec.getDimension()) != null) { - sortingNeeded = true; - break; - } final ColumnType columnType = getOrderByType(columnSpec, dimensions); final StringComparator naturalComparator; diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java index 19024ff39015..da1c046465bf 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.RangeSet; import org.apache.druid.error.DruidException; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import java.util.List; @@ -137,13 +138,22 @@ default short getAtomicUpdateGroupSize() ShardSpecLookup getLookup(List shardSpecs); /** - * Get dimensions who have possible range for the rows this shard contains. + * Get dimensions who have possible range for the rows this shard contains. These columns might be physical columns + * stored in the shard, or computed expressions, in which case the manner in which they were computed is available in + * {@link #getDomainVirtualColumns()}. * - * @return list of dimensions who has its possible range. Dimensions with unknown possible range are not listed + * @return list of dimensions who has its possible range. Dimensions with unknown possible range are not listed. */ @JsonIgnore List getDomainDimensions(); + /** + * If any of the columns in {@link #getDomainDimensions()} was computed with an expression and was not stored, the + * {@link org.apache.druid.segment.VirtualColumn} which computes it is stored here. This allows matching ranges even + * when the value is not stored in the shard so long as {@link VirtualColumns#findEquivalent(VirtualColumn)} exists. + * + * @return {@link VirtualColumns} associated with columns listed in {@link #getDomainDimensions()}. + */ @JsonIgnore default VirtualColumns getDomainVirtualColumns() { diff --git a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java index 222a2c2dd7fe..03dde7257f52 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java @@ -84,30 +84,54 @@ void testPrune() @Test void testPruneVirtualColumn() { - VirtualColumns virtualColumns = VirtualColumns.create( + VirtualColumns shardVirtualColumns = VirtualColumns.create( new ExpressionVirtualColumn("vdim1", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) ); - DimFilter range_a = new RangeFilter("vdim1", ColumnType.STRING, null, "aaa", null, null, null); - String interval1 = "2026-02-18T00:00:00Z/2026-02-19T00:00:00Z"; DataSegment seg1 = makeDataSegment( interval1, - makeRange(List.of("vdim1"), virtualColumns, 0, null, StringTuple.create("abcfoo")) + makeRange(List.of("vdim1"), shardVirtualColumns, 0, null, StringTuple.create("abcfoo")) ); DataSegment seg2 = makeDataSegment( interval1, - makeRange(List.of("vdim1"), virtualColumns, 1, StringTuple.create("abcfoo"), StringTuple.create("lmnfoo")) + makeRange(List.of("vdim1"), shardVirtualColumns, 1, StringTuple.create("abcfoo"), StringTuple.create("lmnfoo")) ); DataSegment seg3 = makeDataSegment( interval1, - makeRange(List.of("vdim1"), virtualColumns, 2, StringTuple.create("lmnfoo"), null) + makeRange(List.of("vdim1"), shardVirtualColumns, 2, StringTuple.create("lmnfoo"), null) ); List segs = List.of(seg1, seg2, seg3); - FilterSegmentPruner prunerRange = new FilterSegmentPruner(range_a, null, virtualColumns); - FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), virtualColumns); + // same expression, same name + VirtualColumns queryVirtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("vdim1", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + DimFilter range_a = new RangeFilter("vdim1", ColumnType.STRING, null, "aaa", null, null, null); + FilterSegmentPruner prunerRange = new FilterSegmentPruner(range_a, null, queryVirtualColumns); + FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), queryVirtualColumns); + Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); + + // same expression, different name + queryVirtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v0", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + range_a = new RangeFilter("v0", ColumnType.STRING, null, "aaa", null, null, null); + prunerRange = new FilterSegmentPruner(range_a, null, queryVirtualColumns); + prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), queryVirtualColumns); + + Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); + + // same expression, different name + queryVirtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v10", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + range_a = new RangeFilter("v10", ColumnType.STRING, null, "aaa", null, null, null); + prunerRange = new FilterSegmentPruner(range_a, null, queryVirtualColumns); + prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), queryVirtualColumns); Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index a3df6001f056..994ced8f9b8b 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -57,6 +57,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.stream.Collectors; @@ -86,14 +87,13 @@ public static Builder builder(DataSchema schema) private final Map parserMap; private final ObjectMapper objectMapper; - // The below fields can be initialized lazily from parser for backward compatibility. - private TimestampSpec timestampSpec; - private DimensionsSpec dimensionsSpec; + private final TimestampSpec timestampSpec; + private final DimensionsSpec dimensionsSpec; // This is used for backward compatibility private InputRowParser inputRowParser; @Nullable - private List projections; + private final List projections; @JsonCreator public DataSchema( @@ -111,15 +111,21 @@ public DataSchema( validateDatasourceName(dataSource); this.dataSource = dataSource; - this.timestampSpec = timestampSpec; + if (timestampSpec == null) { + this.timestampSpec = Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getTimestampSpec(); + } else { + this.timestampSpec = timestampSpec; + } this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; - this.dimensionsSpec = dimensionsSpec == null - ? null - : computeDimensionsSpec( - Preconditions.checkNotNull(timestampSpec, "timestampSpec"), - dimensionsSpec, - this.aggregators - ); + if (dimensionsSpec == null) { + this.dimensionsSpec = computeDimensionsSpec( + this.timestampSpec, + Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getDimensionsSpec(), + this.aggregators + ); + } else { + this.dimensionsSpec = computeDimensionsSpec(this.timestampSpec, dimensionsSpec, this.aggregators); + } if (granularitySpec == null) { log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default."); @@ -165,9 +171,6 @@ private TimestampSpec getGivenTimestampSpec() public TimestampSpec getTimestampSpec() { - if (timestampSpec == null) { - timestampSpec = Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getTimestampSpec(); - } return timestampSpec; } @@ -180,13 +183,6 @@ private DimensionsSpec getGivenDimensionsSpec() public DimensionsSpec getDimensionsSpec() { - if (dimensionsSpec == null) { - dimensionsSpec = computeDimensionsSpec( - getTimestampSpec(), - Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getDimensionsSpec(), - aggregators - ); - } return dimensionsSpec; } @@ -527,6 +523,38 @@ private static Set getFieldsOrThrowIfErrors(Map } } + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) { + return false; + } + DataSchema that = (DataSchema) o; + return Objects.equals(dataSource, that.dataSource) && + Objects.deepEquals(aggregators, that.aggregators) && + Objects.equals(granularitySpec, that.granularitySpec) && + Objects.equals(transformSpec, that.transformSpec) && + Objects.equals(parserMap, that.parserMap) && + Objects.equals(timestampSpec, that.timestampSpec) && + Objects.equals(dimensionsSpec, that.dimensionsSpec) && + Objects.equals(projections, that.projections); + } + + @Override + public int hashCode() + { + return Objects.hash( + dataSource, + Arrays.hashCode(aggregators), + granularitySpec, + transformSpec, + parserMap, + timestampSpec, + dimensionsSpec, + projections + ); + } + public static class Builder { private String dataSource; diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 9e83ab71ccf8..eb2deb55e129 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -25,10 +25,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.commons.text.StringEscapeUtils; import org.apache.druid.common.utils.IdUtilsTest; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; @@ -40,6 +42,7 @@ import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -1018,4 +1021,15 @@ void testInvalidProjectionDupeAggNames() t.getMessage() ); } + + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(DataSchema.class) + .usingGetClass() + .withIgnoredFields("objectMapper", "inputRowParser") + .withPrefabValues(ObjectMapper.class, jsonMapper, new DefaultObjectMapper()) + .withPrefabValues(DimensionSchema.class, new StringDimensionSchema("foo"), new LongDimensionSchema("bar")) + .verify(); + } } From e43a26b984e5dc55a3a9d854c4e455f0275038e7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 16 Mar 2026 14:54:44 -0700 Subject: [PATCH 09/12] fix up test --- .../druid/segment/indexing/DataSchema.java | 27 ++++---- .../segment/indexing/DataSchemaTest.java | 67 +++++++++---------- 2 files changed, 46 insertions(+), 48 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 994ced8f9b8b..1999f1919b22 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -111,12 +111,25 @@ public DataSchema( validateDatasourceName(dataSource); this.dataSource = dataSource; + this.objectMapper = objectMapper; + this.parserMap = parserMap; + + if (granularitySpec == null) { + log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default."); + this.granularitySpec = new UniformGranularitySpec(null, null, null); + } else { + this.granularitySpec = granularitySpec; + } + this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; + this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; + this.projections = projections; + + // do these 2 last to populate stuff if (timestampSpec == null) { this.timestampSpec = Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getTimestampSpec(); } else { this.timestampSpec = timestampSpec; } - this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; if (dimensionsSpec == null) { this.dimensionsSpec = computeDimensionsSpec( this.timestampSpec, @@ -127,17 +140,6 @@ public DataSchema( this.dimensionsSpec = computeDimensionsSpec(this.timestampSpec, dimensionsSpec, this.aggregators); } - if (granularitySpec == null) { - log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default."); - this.granularitySpec = new UniformGranularitySpec(null, null, null); - } else { - this.granularitySpec = granularitySpec; - } - this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; - this.projections = projections; - this.parserMap = parserMap; - this.objectMapper = objectMapper; - // Fail-fast if there are output name collisions. Note: because of the pull-from-parser magic in getDimensionsSpec, // this validation is not necessarily going to be able to catch everything. It will run again in getDimensionsSpec. computeAndValidateOutputFieldNames(this.dimensionsSpec, this.aggregators); @@ -153,7 +155,6 @@ public DataSchema( dataSource ); } - } @JsonProperty diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index eb2deb55e129..79c0853d36e7 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.indexing; -import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.exc.ValueInstantiationException; import com.google.common.collect.ImmutableList; @@ -244,19 +243,18 @@ void testOverlapMetricNameAndDim() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = DataSchema.builder() - .withDataSource(IdUtilsTest.VALID_ID_CHARS) - .withParserMap(parser) - .withAggregators( - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2") - ) - .withGranularity(ARBITRARY_GRANULARITY) - .withObjectMapper(jsonMapper) - .build(); Throwable t = Assertions.assertThrows( DruidException.class, - () -> schema.getParser() + () -> DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build() ); Assertions.assertEquals( @@ -420,16 +418,14 @@ void testOverlapTimeAndDimLegacy() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = DataSchema.builder() - .withDataSource(IdUtilsTest.VALID_ID_CHARS) - .withParserMap(parser) - .withGranularity(ARBITRARY_GRANULARITY) - .withObjectMapper(jsonMapper) - .build(); - Throwable t = Assertions.assertThrows( DruidException.class, - () -> schema.getParser() + () -> DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build() ); Assertions.assertEquals( @@ -494,25 +490,18 @@ void testSerdeWithInvalidParserMap() throws Exception + "\"intervals\":[\"2014-01-01T00:00:00.000Z/2015-01-01T00:00:00.000Z\"]}}"; - //no error on serde as parser is converted to InputRowParser lazily when really needed - DataSchema schema = jsonMapper.readValue( - jsonMapper.writeValueAsString( - jsonMapper.readValue(jsonStr, DataSchema.class) - ), - DataSchema.class - ); - Throwable t = Assertions.assertThrows( - IllegalArgumentException.class, - () -> schema.getParser() + ValueInstantiationException.class, + () -> jsonMapper.readValue( + jsonMapper.writeValueAsString( + jsonMapper.readValue(jsonStr, DataSchema.class) + ), + DataSchema.class + ) ); MatcherAssert.assertThat( t.getMessage(), - Matchers.startsWith("Cannot construct instance of `org.apache.druid.data.input.impl.StringInputRowParser`, problem: parseSpec") - ); - MatcherAssert.assertThat( - t.getCause(), - Matchers.instanceOf(JsonMappingException.class) + Matchers.startsWith("Cannot construct instance of `org.apache.druid.segment.indexing.DataSchema`") ); } @@ -868,6 +857,8 @@ void testInvalidProjectionDupeNames() DruidException.class, () -> DataSchema.builder() .withDataSource("dataSource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) .withGranularity( new UniformGranularitySpec( Granularities.HOUR, @@ -913,6 +904,8 @@ void testInvalidProjectionGranularity() DruidException.class, () -> DataSchema.builder() .withDataSource("dataSource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) .withGranularity( new UniformGranularitySpec( Granularities.HOUR, @@ -956,6 +949,8 @@ void testInvalidProjectionDupeGroupingNames() DruidException.class, () -> DataSchema.builder() .withDataSource("dataSource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) .withGranularity( new UniformGranularitySpec( Granularities.HOUR, @@ -991,6 +986,8 @@ void testInvalidProjectionDupeAggNames() DruidException.class, () -> DataSchema.builder() .withDataSource("dataSource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) .withGranularity( new UniformGranularitySpec( Granularities.HOUR, From a9c06730c8836353047cae62c23962c0f38b8699 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 18 Mar 2026 11:46:42 -0700 Subject: [PATCH 10/12] restore some formatting on now unrelated files, fix unused imports --- .../testing/embedded/compact/CompactionSupervisorTest.java | 1 - .../org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java | 2 ++ .../apache/druid/msq/querykit/groupby/GroupByQueryKit.java | 1 + .../org/apache/druid/msq/querykit/scan/ScanQueryKit.java | 7 +++---- .../processor/SegmentGeneratorStageProcessorTest.java | 1 - .../org/apache/druid/segment/indexing/DataSchemaTest.java | 1 - 6 files changed, 6 insertions(+), 7 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index ca2061ecb020..2a8dff78b0cf 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -527,7 +527,6 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec Assertions.assertTrue(count > 0); } - @Test public void test_compaction_cluster_by_virtualcolumn() { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java index 6a9cfab6793a..7ec3f24e3de2 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/QueryKitBasedMSQPlanner.java @@ -161,6 +161,7 @@ private QueryDefinition makeQueryDefinitionInternal(final Query queryToPlan) // Add all query stages. // Set shuffleCheckHasMultipleValues on the stage that serves as input to the final segment-generation stage. final QueryDefinitionBuilder builder = QueryDefinition.builder(queryKitSpec.getQueryId()); + for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { if (stageDef.equals(finalShuffleStageDef)) { builder.add(StageDefinition.builder(stageDef).shuffleCheckHasMultipleValues(true)); @@ -168,6 +169,7 @@ private QueryDefinition makeQueryDefinitionInternal(final Query queryToPlan) builder.add(StageDefinition.builder(stageDef)); } } + final DataSourceMSQDestination destination1 = (DataSourceMSQDestination) destination; return builder.add( destination1.getTerminalStageSpec() diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java index b3f9a6beafb1..8ace680938c1 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java @@ -334,6 +334,7 @@ static ClusterBy computeClusterByForResults(final GroupByQuery query) if (!defaultLimitSpec.getColumns().isEmpty()) { final List clusterByColumns = new ArrayList<>(); + for (final OrderByColumnSpec orderBy : defaultLimitSpec.getColumns()) { clusterByColumns.add( new KeyColumn( diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index 3824ba971133..f6cc9222df99 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -103,6 +103,7 @@ public QueryDefinition makeQueryDefinition( final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, queryToRun.getContext()); final List clusterByColumns = new ArrayList<>(); + // Add regular orderBys. for (final OrderBy orderBy : queryToRun.getOrderBys()) { clusterByColumns.add( @@ -116,10 +117,8 @@ public QueryDefinition makeQueryDefinition( clusterByColumns.add(new KeyColumn(QueryKitUtils.PARTITION_BOOST_COLUMN, KeyOrder.ASCENDING)); signatureBuilder.add(QueryKitUtils.PARTITION_BOOST_COLUMN, ColumnType.LONG); - final ClusterBy clusterBy = QueryKitUtils.clusterByWithSegmentGranularity( - new ClusterBy(clusterByColumns, 0), - segmentGranularity - ); + final ClusterBy clusterBy = + QueryKitUtils.clusterByWithSegmentGranularity(new ClusterBy(clusterByColumns, 0), segmentGranularity); final ShuffleSpec finalShuffleSpec = resultShuffleSpecFactory.build(clusterBy, false); final RowSignature signatureToUse = QueryKitUtils.sortableSignature( diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java index 4fddecfff217..7a06c419dd68 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessorTest.java @@ -45,7 +45,6 @@ class SegmentGeneratorStageProcessorTest { - @Test public void testSerde() throws JsonProcessingException { diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index a5e47e866fea..1aa8a296ddcf 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -28,7 +28,6 @@ import org.apache.commons.text.StringEscapeUtils; import org.apache.druid.common.utils.IdUtilsTest; import org.apache.druid.data.input.impl.AggregateProjectionSpec; -import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; From 583d2cb506aa55894be1cc645b6b90c968a99dc5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 18 Mar 2026 16:26:00 -0700 Subject: [PATCH 11/12] use builder, adjust test to more useful --- .../compact/CompactionSupervisorTest.java | 50 +++---------------- .../query/filter/FilterSegmentPrunerTest.java | 19 ++++--- 2 files changed, 17 insertions(+), 52 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 2a8dff78b0cf..838bb5599893 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -575,27 +575,10 @@ public void test_compaction_cluster_by_virtualcolumn() ) ) .withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(4, null, List.of("v0"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + UserCompactionTaskQueryTuningConfig + .builder() + .partitionsSpec(new DimensionRangePartitionsSpec(4, null, List.of("v0"), false)) + .build() ) .build(); @@ -684,27 +667,10 @@ public void test_compaction_cluster_by_virtualcolumn_rollup() ) ) .withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(4, null, List.of("v0"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + UserCompactionTaskQueryTuningConfig + .builder() + .partitionsSpec(new DimensionRangePartitionsSpec(4, null, List.of("v0"), false)) + .build() ) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java index 03dde7257f52..b141608ad4f3 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java @@ -76,9 +76,13 @@ void testPrune() FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), null); FilterSegmentPruner prunerExpression = new FilterSegmentPruner(expression_b, null, null); + // prune twice to exercise cache Assertions.assertEquals(Set.of(seg1, seg4, seg5, seg6, seg7), prunerRange.prune(segs, Function.identity())); + Assertions.assertEquals(Set.of(seg1, seg4, seg5, seg6, seg7), prunerRange.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerExpression.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerExpression.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); } @Test @@ -111,8 +115,11 @@ void testPruneVirtualColumn() DimFilter range_a = new RangeFilter("vdim1", ColumnType.STRING, null, "aaa", null, null, null); FilterSegmentPruner prunerRange = new FilterSegmentPruner(range_a, null, queryVirtualColumns); FilterSegmentPruner prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), queryVirtualColumns); + // prune twice to exercise cache + Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); // same expression, different name queryVirtualColumns = VirtualColumns.create( @@ -122,19 +129,11 @@ void testPruneVirtualColumn() prunerRange = new FilterSegmentPruner(range_a, null, queryVirtualColumns); prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), queryVirtualColumns); + // prune twice to exercise cache Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); - Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); - - // same expression, different name - queryVirtualColumns = VirtualColumns.create( - new ExpressionVirtualColumn("v10", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) - ); - range_a = new RangeFilter("v10", ColumnType.STRING, null, "aaa", null, null, null); - prunerRange = new FilterSegmentPruner(range_a, null, queryVirtualColumns); - prunerEmptyFields = new FilterSegmentPruner(range_a, Collections.emptySet(), queryVirtualColumns); - Assertions.assertEquals(Set.of(seg1), prunerRange.prune(segs, Function.identity())); Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); + Assertions.assertEquals(Set.copyOf(segs), prunerEmptyFields.prune(segs, Function.identity())); } @Test From 614ba60b067f995f34eb2a35a10ab2e38ed9a3c5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 18 Mar 2026 16:29:06 -0700 Subject: [PATCH 12/12] more better --- .../apache/druid/query/filter/FilterSegmentPrunerTest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java index b141608ad4f3..a1a1dba0c281 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/FilterSegmentPrunerTest.java @@ -91,6 +91,10 @@ void testPruneVirtualColumn() VirtualColumns shardVirtualColumns = VirtualColumns.create( new ExpressionVirtualColumn("vdim1", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) ); + VirtualColumns shardVirtualColumnsDifferentName = VirtualColumns.create( + new ExpressionVirtualColumn("vdifferentname", "concat(dim1, 'foo')", ColumnType.STRING, TestExprMacroTable.INSTANCE) + ); + String interval1 = "2026-02-18T00:00:00Z/2026-02-19T00:00:00Z"; DataSegment seg1 = makeDataSegment( @@ -101,9 +105,10 @@ void testPruneVirtualColumn() interval1, makeRange(List.of("vdim1"), shardVirtualColumns, 1, StringTuple.create("abcfoo"), StringTuple.create("lmnfoo")) ); + // same virtual column with a different name in this segment DataSegment seg3 = makeDataSegment( interval1, - makeRange(List.of("vdim1"), shardVirtualColumns, 2, StringTuple.create("lmnfoo"), null) + makeRange(List.of("vdifferentname"), shardVirtualColumnsDifferentName, 2, StringTuple.create("lmnfoo"), null) ); List segs = List.of(seg1, seg2, seg3);