From 9fcc1b0446736116af805efd369d81a5347b100e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 1 Jun 2024 01:35:38 -0700 Subject: [PATCH 01/74] rework cursor creation changes: * Added `CursorBuildSpec` which captures all of the 'interesting' stuff that goes into producing a cursor as a replacement for the method arguments of `CursorFactory.canVectorize`, `CursorFactory.makeCursors`, and `CursorFactory.makeVectorCursors` * added new interfaces `CursorMaker` and new method `asCursorMaker` to `CursorFactory`, which takes a `CursorBuildSpec` as an argument and replaces `CursorFactory.canVectorize`, `CursorFactory.makeCursors`, and `CursorFactory.makeVectorCursors` * Deprecated `CursorFactory.canVectorize`, `CursorFactory.makeCursors`, and `CursorFactory.makeVectorCursors` * updated all `CursorFactory` implementations to implement `asCursorMaker` * updated all query engines to use `asCursorMaker` --- .../ExpressionAggregationBenchmark.java | 15 +- .../benchmark/ExpressionFilterBenchmark.java | 31 +- .../ExpressionSelectorBenchmark.java | 458 +++++++++--------- .../ExpressionVectorSelectorBenchmark.java | 27 +- .../benchmark/FilterPartitionBenchmark.java | 10 +- .../IndexedTableJoinCursorBenchmark.java | 10 +- .../benchmark/JoinAndLookupBenchmark.java | 200 ++++---- .../IncrementalIndexReadBenchmark.java | 17 +- ...bjectVectorColumnProcessorFactoryTest.java | 22 +- .../SegmentGeneratorFrameProcessor.java | 7 +- .../results/ExportResultsFrameProcessor.java | 7 +- .../scan/ScanQueryFrameProcessor.java | 18 +- .../indexing/input/DruidSegmentReader.java | 16 +- .../common/task/CompactionTaskRunTest.java | 47 +- .../indexing/common/task/IndexTaskTest.java | 32 +- .../frame/processor/FrameProcessors.java | 9 +- .../frame/segment/FrameStorageAdapter.java | 26 +- .../segment/columnar/FrameCursorFactory.java | 142 +++--- .../frame/segment/row/FrameCursorFactory.java | 87 ++-- .../query/FrameBasedInlineDataSource.java | 8 +- .../java/org/apache/druid/query/Query.java | 29 ++ .../org/apache/druid/query/QueryContexts.java | 29 +- .../apache/druid/query/QueryRunnerHelper.java | 34 -- .../apache/druid/query/filter/DimFilter.java | 4 +- .../org/apache/druid/query/filter/Filter.java | 3 +- .../druid/query/filter/FilterBundle.java | 19 +- .../druid/query/groupby/GroupByQuery.java | 29 ++ .../druid/query/groupby/GroupingEngine.java | 36 +- .../epinephelinae/GroupByQueryEngine.java | 18 +- .../vector/VectorGroupByEngine.java | 42 +- .../druid/query/metadata/SegmentAnalyzer.java | 19 +- .../LazilyDecoratedRowsAndColumns.java | 23 +- .../StorageAdapterRowsAndColumns.java | 13 +- .../apache/druid/query/scan/ScanQuery.java | 30 ++ .../druid/query/scan/ScanQueryEngine.java | 17 +- .../query/search/CursorOnlyStrategy.java | 10 +- .../TimeBoundaryQueryRunnerFactory.java | 27 +- .../query/timeseries/TimeseriesQuery.java | 31 ++ .../timeseries/TimeseriesQueryEngine.java | 119 ++--- .../apache/druid/query/topn/TopNQuery.java | 31 ++ .../druid/query/topn/TopNQueryEngine.java | 24 +- .../druid/segment/ArrayListSegment.java | 4 +- .../java/org/apache/druid/segment/Cursor.java | 2 +- .../apache/druid/segment/CursorBuildSpec.java | 240 +++++++++ .../apache/druid/segment/CursorFactory.java | 112 +++++ .../org/apache/druid/segment/CursorMaker.java | 74 +++ .../druid/segment/FilteredStorageAdapter.java | 30 +- ...er.java => QueryableIndexCursorMaker.java} | 345 ++++++++----- .../segment/QueryableIndexStorageAdapter.java | 76 +-- .../apache/druid/segment/RowBasedSegment.java | 4 +- .../apache/druid/segment/SegmentWrangler.java | 3 +- .../druid/segment/UnnestStorageAdapter.java | 120 +++-- .../apache/druid/segment/VirtualColumns.java | 11 - .../druid/segment/filter/AndFilter.java | 11 + .../apache/druid/segment/filter/OrFilter.java | 23 + .../IncrementalIndexCursorMaker.java | 264 ++++++++++ .../IncrementalIndexStorageAdapter.java | 204 +------- .../join/HashJoinSegmentStorageAdapter.java | 213 ++++---- .../druid/segment/join/PostJoinCursor.java | 5 +- .../table/BroadcastSegmentIndexedTable.java | 19 +- .../join/table/FrameBasedIndexedTable.java | 12 +- .../druid/segment/vector/VectorCursor.java | 7 +- .../druid/frame/TestArrayStorageAdapter.java | 30 +- .../processor/OutputChannelFactoryTest.java | 29 +- .../segment/FrameStorageAdapterTest.java | 51 +- .../druid/frame/testutil/FrameTestUtil.java | 20 +- .../druid/frame/write/FrameWriterTest.java | 6 +- .../groupby/NestedDataGroupByQueryTest.java | 22 +- .../VectorGroupByEngineIteratorTest.java | 12 +- .../druid/query/lookup/LookupSegmentTest.java | 16 +- .../semantic/RowsAndColumnsDecoratorTest.java | 18 +- .../scan/ScanQueryResultOrderingTest.java | 3 +- .../NestedDataTimeseriesQueryTest.java | 20 +- .../segment/AutoTypeColumnIndexerTest.java | 64 +-- .../NestedDataColumnIndexerV4Test.java | 56 +-- ...ava => QueryableIndexCursorMakerTest.java} | 22 +- .../QueryableIndexStorageAdapterTest.java | 31 +- .../segment/RowBasedStorageAdapterTest.java | 185 +++---- .../segment/UnnestStorageAdapterTest.java | 127 +++-- .../druid/segment/filter/BaseFilterTest.java | 91 ++-- .../filter/FloatAndDoubleFilteringTest.java | 103 ---- .../segment/filter/LongFilteringTest.java | 87 ---- .../IncrementalIndexStorageAdapterTest.java | 94 ++-- ...BaseHashJoinSegmentStorageAdapterTest.java | 3 +- .../HashJoinSegmentStorageAdapterTest.java | 409 ++++------------ .../segment/join/JoinFilterAnalyzerTest.java | 384 +++++---------- .../segment/join/PostJoinCursorTest.java | 34 +- .../NestedFieldColumnSelectorsTest.java | 27 +- .../virtual/ExpressionSelectorsTest.java | 64 +-- .../ExpressionVectorSelectorsCastTest.java | 15 +- .../ExpressionVectorSelectorsTest.java | 24 +- .../firehose/IngestSegmentFirehose.java | 20 +- .../org/apache/druid/cli/DumpSegment.java | 20 +- 93 files changed, 2734 insertions(+), 2908 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java create mode 100644 processing/src/main/java/org/apache/druid/segment/CursorMaker.java rename processing/src/main/java/org/apache/druid/segment/{QueryableIndexCursorSequenceBuilder.java => QueryableIndexCursorMaker.java} (65%) create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java rename processing/src/test/java/org/apache/druid/segment/{QueryableIndexCursorSequenceBuilderTest.java => QueryableIndexCursorMakerTest.java} (72%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java index 8afcea4a4c32..7c1d9994d0df 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -35,9 +35,9 @@ import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -165,14 +165,11 @@ private double compute(final Function a { final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final Sequence cursors = adapter.makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build(); + final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); final List results = cursors .map(cursor -> { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java index e7d8ad00bc5e..c9eaf06f8414 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -33,9 +33,9 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -146,14 +146,13 @@ public void tearDown() throws Exception @Benchmark public void expressionFilter(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - expressionFilter.toFilter(), - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(expressionFilter.toFilter()) + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build(); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); + final List results = cursors .map(cursor -> { final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); @@ -168,14 +167,12 @@ public void expressionFilter(Blackhole blackhole) @Benchmark public void nativeFilter(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - nativeFilter.toFilter(), - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(nativeFilter.toFilter()) + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build(); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); final List results = cursors .map(cursor -> { final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); 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 dd33b6606440..b087b3fedd22 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -35,6 +35,7 @@ import org.apache.druid.query.extraction.TimeFormatExtractionFn; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -141,23 +142,22 @@ public void tearDown() throws Exception @Benchmark public void timeFloorUsingExpression(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "timestamp_floor(__time, 'PT1H')", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "timestamp_floor(__time, 'PT1H')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -173,14 +173,12 @@ public void timeFloorUsingExpression(Blackhole blackhole) @Benchmark public void timeFloorUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -204,14 +202,12 @@ public void timeFloorUsingExtractionFn(Blackhole blackhole) @Benchmark public void timeFloorUsingCursor(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.HOUR, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.HOUR) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -235,23 +231,22 @@ public void timeFloorUsingCursor(Blackhole blackhole) @Benchmark public void timeFormatUsingExpression(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "timestamp_format(__time, 'yyyy-MM-dd')", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "timestamp_format(__time, 'yyyy-MM-dd')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -269,14 +264,12 @@ public void timeFormatUsingExpression(Blackhole blackhole) @Benchmark public void timeFormatUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -300,23 +293,22 @@ public void timeFormatUsingExtractionFn(Blackhole blackhole) @Benchmark public void strlenUsingExpressionAsLong(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "strlen(s)", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "strlen(s)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -332,23 +324,22 @@ public void strlenUsingExpressionAsLong(Blackhole blackhole) @Benchmark public void strlenUsingExpressionAsString(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "strlen(s)", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "strlen(s)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -367,14 +358,12 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) @Benchmark public void strlenUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -393,23 +382,22 @@ public void strlenUsingExtractionFn(Blackhole blackhole) @Benchmark public void arithmeticOnLong(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "n + 1", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "n + 1", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -425,23 +413,22 @@ public void arithmeticOnLong(Blackhole blackhole) @Benchmark public void stringConcatAndCompareOnLong(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "concat(n, ' is my favorite number') == '3 is my favorite number'", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "concat(n, ' is my favorite number') == '3 is my favorite number'", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -457,23 +444,22 @@ public void stringConcatAndCompareOnLong(Blackhole blackhole) @Benchmark public void caseSearched1(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "case_searched(s == 'asd' || isnull(s) || s == 'xxx', 1, s == 'foo' || s == 'bar', 2, 3)", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || s == 'xxx', 1, s == 'foo' || s == 'bar', 2, 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -489,23 +475,22 @@ public void caseSearched1(Blackhole blackhole) @Benchmark public void caseSearched2(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "case_searched(s == 'asd' || isnull(s) || n == 1, 1, n == 2, 2, 3)", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || n == 1, 1, n == 2, 2, 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -534,23 +519,22 @@ public void caseSearched100(Blackhole blackhole) ); } - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "case_searched(s == 'asd' || isnull(s) || n == 1, 1, " + caseBranches + " 3)", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || n == 1, 1, " + caseBranches + " 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -566,29 +550,28 @@ public void caseSearched100(Blackhole blackhole) @Benchmark public void caseSearchedWithLookup(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "case_searched(n == 1001, -1, " - + "lookup(s, 'lookyloo') == 'asd1', 1, " - + "lookup(s, 'lookyloo') == 'asd2', 2, " - + "lookup(s, 'lookyloo') == 'asd3', 3, " - + "lookup(s, 'lookyloo') == 'asd4', 4, " - + "lookup(s, 'lookyloo') == 'asd5', 5, " - + "-2)", - ColumnType.LONG, - LookupEnabledTestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(n == 1001, -1, " + + "lookup(s, 'lookyloo') == 'asd1', 1, " + + "lookup(s, 'lookyloo') == 'asd2', 2, " + + "lookup(s, 'lookyloo') == 'asd3', 3, " + + "lookup(s, 'lookyloo') == 'asd4', 4, " + + "lookup(s, 'lookyloo') == 'asd5', 5, " + + "-2)", + ColumnType.LONG, + LookupEnabledTestExprMacroTable.INSTANCE + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { @@ -604,35 +587,36 @@ public void caseSearchedWithLookup(Blackhole blackhole) @Benchmark public void caseSearchedWithLookup2(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - 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 - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( + CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .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 + ) + ) + ) + ) + .build() + ).makeCursors(); final List results = cursors .map(cursor -> { 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 754eb7b6d381..908f1c8f4cad 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -33,6 +33,8 @@ import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -150,15 +152,14 @@ public void scan(Blackhole blackhole) ) ) ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns(virtualColumns) + .build(); + final CursorMaker cursorMaker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec); if (vectorize) { - VectorCursor cursor = new QueryableIndexStorageAdapter(index).makeVectorCursor( - null, - index.getDataInterval(), - virtualColumns, - false, - 512, - null - ); + VectorCursor cursor = cursorMaker.makeVectorCursor(); if (outputType.isNumeric()) { VectorValueSelector selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); if (outputType.is(ExprType.DOUBLE)) { @@ -177,15 +178,7 @@ public void scan(Blackhole blackhole) closer.register(cursor); } } else { - Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - virtualColumns, - Granularities.ALL, - false, - null - ); - + final Sequence cursors = cursorMaker.makeCursors(); int rowCount = cursors .map(cursor -> { final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 98c7ddf4f856..041f71697f0e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -48,6 +48,7 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -55,7 +56,6 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; @@ -451,7 +451,13 @@ public void readComplexOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosio private Sequence makeCursors(StorageAdapter sa, Filter filter) { - return sa.makeCursors(filter, schemaInfo.getDataInterval(), VirtualColumns.EMPTY, Granularities.ALL, false, null); + return sa.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(schemaInfo.getDataInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); } private void readCursors(Sequence cursors, Blackhole blackhole) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java index c2dd8ae9996a..00126ac2b5ec 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -233,14 +234,7 @@ public void hashJoinCursorDimensionSelectors(Blackhole blackhole) private Sequence makeCursors() { - return hashJoinSegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + return hashJoinSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); } 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 7ac9931da48f..c803cfb786e3 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -23,18 +23,19 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.LookupExprMacro; +import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.query.lookup.MapLookupExtractorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -367,14 +368,9 @@ private static String getLastValue(final Sequence cursors, final String @OutputTimeUnit(TimeUnit.MILLISECONDS) public void baseSegment(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = baseSegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, "countryIsoCode")); } @@ -384,14 +380,15 @@ public void baseSegment(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void baseSegmentWithFilter(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter().makeCursors( - new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(); + final Sequence cursors = baseSegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); blackhole.consume(getLastValue(cursors, "countryIsoCode")); } @@ -401,14 +398,9 @@ public void baseSegmentWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupStringKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.v")); } @@ -418,14 +410,15 @@ public void joinLookupStringKey(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupStringKeyWithFilter(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter().makeCursors( - new SelectorDimFilter("c.v", "Canada", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter(); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.v")); } @@ -435,14 +428,9 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupLongKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupLongKeySegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.v")); } @@ -452,14 +440,15 @@ public void joinLookupLongKey(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupLongKeyWithFilter(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupLongKeySegment.asStorageAdapter().makeCursors( - new SelectorDimFilter("c.v", "Canada", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter(); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.v")); } @@ -469,14 +458,9 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableLongKey(Blackhole blackhole) { - final Sequence cursors = hashJoinIndexedTableLongKeySegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.countryName")); } @@ -486,14 +470,15 @@ public void joinIndexedTableLongKey(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) { - final Sequence cursors = hashJoinIndexedTableLongKeySegment.asStorageAdapter().makeCursors( - new SelectorDimFilter("c.countryName", "Canada", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter(); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.countryName")); } @@ -503,14 +488,9 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableStringKey(Blackhole blackhole) { - final Sequence cursors = hashJoinIndexedTableStringKeySegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.countryName")); } @@ -520,14 +500,15 @@ public void joinIndexedTableStringKey(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) { - final Sequence cursors = hashJoinIndexedTableStringKeySegment.asStorageAdapter().makeCursors( - new SelectorDimFilter("c.countryName", "Canada", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter(); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); blackhole.consume(getLastValue(cursors, "c.countryName")); } @@ -537,14 +518,9 @@ public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void lookupVirtualColumnStringKey(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - lookupVirtualColumns, - Granularities.ALL, - false, - null - ); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); } @@ -554,14 +530,15 @@ public void lookupVirtualColumnStringKey(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter().makeCursors( - new SelectorDimFilter(LOOKUP_COUNTRY_CODE_TO_NAME, "Canada", null).toFilter(), - Intervals.ETERNITY, - lookupVirtualColumns, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_CODE_TO_NAME, "Canada", null).toFilter(); + final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); } @@ -571,14 +548,9 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void lookupVirtualColumnLongKey(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - lookupVirtualColumns, - Granularities.ALL, - false, - null - ); + final Sequence cursors = baseSegment.asStorageAdapter() + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } @@ -588,14 +560,16 @@ public void lookupVirtualColumnLongKey(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void lookupVirtualColumnLongKeyWithFilter(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter().makeCursors( - new SelectorDimFilter(LOOKUP_COUNTRY_NUMBER_TO_NAME, "Canada", null).toFilter(), - Intervals.ETERNITY, - lookupVirtualColumns, - Granularities.ALL, - false, - null - ); + final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_NUMBER_TO_NAME, "Canada", null).toFilter(); + final Sequence cursors = baseSegment.asStorageAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build() + ) + .makeCursors(); + blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 379885983241..9f765f54d7ff 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -36,8 +36,8 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.generator.DataGenerator; import org.apache.druid.segment.generator.GeneratorBasicSchemas; @@ -205,14 +205,13 @@ public void readWithFilters(Blackhole blackhole) private Sequence makeCursors(IncrementalIndexStorageAdapter sa, DimFilter filter) { - return sa.makeCursors( - filter == null ? null : filter.toFilter(), - schemaInfo.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setInterval(schemaInfo.getDataInterval()); + if (filter != null) { + builder.setFilter(filter.toFilter()); + } + return sa.asCursorMaker(builder.build()).makeCursors(); } private static DimensionSelector makeDimensionSelector(Cursor cursor, String name) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index e147cbc0377f..c0a5d3c962c8 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -19,15 +19,17 @@ package org.apache.druid.query.aggregation.datasketches.util; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.druid.hll.HyperLogLogCollector; -import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; @@ -169,14 +171,14 @@ public void testComplexSketch() private VectorCursor makeCursor() { - return adapter.makeVectorCursor( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - false, - 3, /* vector size */ - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setQueryContext( + QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) + ) + ) + .build(); + return adapter.asCursorMaker(buildSpec).makeVectorCursor(); } private List readColumn(final String column, final int limit) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index 78b3e16f6702..0c7b1db5ee50 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java @@ -38,7 +38,6 @@ import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; @@ -48,7 +47,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -183,8 +182,8 @@ private void addFrame(final Frame frame) final MSQInputRow inputRow = new MSQInputRow(); final Sequence cursorSequence = - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); SequenceUtils.forEach( cursorSequence, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java index e3635338231e..bd986c98e19a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java @@ -33,14 +33,13 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.util.SequenceUtils; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -145,8 +144,8 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx private void exportFrame(final Frame frame) { final Sequence cursorSequence = - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors(); SequenceUtils.forEach( cursorSequence, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index c570fdc29b91..e295013df48d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -44,7 +44,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Unit; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; @@ -62,7 +61,6 @@ import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.query.Druids; import org.apache.druid.query.IterableRowsCursorHelper; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -77,9 +75,7 @@ import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.timeline.SegmentId; -import org.joda.time.Interval; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; @@ -445,18 +441,6 @@ private static Sequence makeCursors(final ScanQuery query, final Storage ); } - final List intervals = query.getQuerySegmentSpec().getIntervals(); - Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals); - - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - - return adapter.makeCursors( - filter, - intervals.get(0), - query.getVirtualColumns(), - Granularities.ALL, - ScanQuery.Order.DESCENDING.equals(query.getTimeOrder()), - null - ); + return adapter.asCursorMaker(query.asCursorBuildSpec(null)).makeCursors(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index d048cf4d8b91..e4b62ed845e7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -48,10 +48,10 @@ import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; @@ -123,15 +123,13 @@ protected CloseableIterator> intermediateRowIterator() throw ), intervalFilter ); + final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() + .setFilter(Filters.toFilter(dimFilter)) + .setInterval(storageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); - final Sequence cursors = storageAdapter.getAdapter().makeCursors( - Filters.toFilter(dimFilter), - storageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = storageAdapter.getAdapter().asCursorMaker(cursorBuildSpec).makeCursors(); // Retain order of columns from the original segments. Useful for preserving dimension order if we're in // schemaless mode. diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 2893ef476a63..ecb75f26a1d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -75,11 +75,11 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; @@ -1753,14 +1753,13 @@ public void testRunWithSpatialDimensions() throws Exception new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - final Sequence cursorSequence = adapter.getAdapter().makeCursors( - null, - segment.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setInterval(segment.getInterval()) + .build(); + final Sequence cursorSequence = adapter.getAdapter() + .asCursorMaker(buildSpec) + .makeCursors(); cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { cursor.reset(); @@ -1888,14 +1887,13 @@ public void testRunWithAutoCastDimensions() throws Exception new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - final Sequence cursorSequence = adapter.getAdapter().makeCursors( - null, - segment.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setInterval(segment.getInterval()) + .build(); + final Sequence cursorSequence = adapter.getAdapter() + .asCursorMaker(buildSpec) + .makeCursors(); cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { cursor.reset(); @@ -2118,14 +2116,13 @@ private List getCSVFormatRowsFromSegments(List segments) th new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - final Sequence cursorSequence = adapter.getAdapter().makeCursors( - null, - segment.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setInterval(segment.getInterval()) + .build(); + final Sequence cursorSequence = adapter.getAdapter() + .asCursorMaker(buildSpec) + .makeCursors(); cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { cursor.reset(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 336b4d499bc8..93cba7e9e351 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -68,13 +68,13 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SegmentSchemaMapping; -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.data.CompressionStrategy; @@ -535,14 +535,13 @@ public void testTransformSpec() throws Exception new QueryableIndexStorageAdapter(indexIO.loadIndex(segmentFile)), segment.getInterval() ); - final Sequence cursorSequence = adapter.getAdapter().makeCursors( - null, - segment.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setInterval(segment.getInterval()) + .build(); + final Sequence cursorSequence = adapter.getAdapter() + .asCursorMaker(buildSpec) + .makeCursors(); final List> transforms = cursorSequence .map(cursor -> { final DimensionSelector selector1 = cursor.getColumnSelectorFactory() @@ -771,14 +770,13 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception segment.getInterval() ); - final Sequence cursorSequence = adapter.getAdapter().makeCursors( - null, - segment.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setInterval(segment.getInterval()) + .build(); + final Sequence cursorSequence = adapter.getAdapter() + .asCursorMaker(buildSpec) + .makeCursors(); final List hashes = cursorSequence .map(cursor -> { final DimensionSelector selector = cursor.getColumnSelectorFactory() diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index 2e21bbc312bf..7ee48d00bde4 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; import java.io.Closeable; @@ -115,12 +116,14 @@ public static FrameCursor makeCursor( final VirtualColumns virtualColumns ) { + final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setVirtualColumns(virtualColumns) + .build(); // Safe to never close the Sequence that the FrameCursor comes from, because it does not need to be closed. // Refer to FrameStorageAdapter#makeCursors. - return (FrameCursor) Yielders.each( - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursors(null, Intervals.ETERNITY, virtualColumns, Granularities.ALL, false, null) + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec).makeCursors() ).get(); } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java index 9733c548d422..df9d3bf5720d 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java @@ -26,7 +26,9 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; @@ -146,6 +148,12 @@ public Metadata getMetadata() return null; } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + return cursorFactory.asCursorMaker(spec); + } + @Override public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) { @@ -162,14 +170,7 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - return cursorFactory.makeCursors( - filter, - interval, - virtualColumns, - gran, - descending, - queryMetrics - ); + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } @Nullable @@ -183,13 +184,6 @@ public VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - return cursorFactory.makeVectorCursor( - filter, - interval, - virtualColumns, - descending, - vectorSize, - queryMetrics - ); + return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java index 3497ed39898a..3714622740c4 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java @@ -37,7 +37,9 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; @@ -80,6 +82,90 @@ public FrameCursorFactory( this.columnReaders = columnReaders; } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + return new CursorMaker() + { + @Override + public boolean canVectorize() + { + return (spec.getFilter() == null || spec.getFilter().canVectorizeMatcher(signature)) + && spec.getVirtualColumns().canVectorize(signature) + && !spec.isDescending(); + } + + @Override + public Sequence makeCursors() + { + final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); + + if (Granularities.ALL.equals(spec.getGranularity())) { + final Closer closer = Closer.create(); + final Cursor cursor = makeGranularityAllCursor( + new ColumnCache(index, closer), + frame.numRows(), + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + spec.isDescending() + ); + + return Sequences.simple(Collections.singletonList(cursor)).withBaggage(closer); + } else { + // Not currently needed for the intended use cases of frame-based cursors. + throw new UOE("Granularity [%s] not supported", spec.getGranularity()); + } + } + + @Nullable + @Override + public VectorCursor makeVectorCursor() + { + if (!canVectorize()) { + throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); + } + + final Closer closer = Closer.create(); + final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); + final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); + final VectorOffset baseOffset = new NoFilterVectorOffset( + spec.getQueryContext().getVectorSize(), + 0, + frame.numRows() + ); + final ColumnCache columnCache = new ColumnCache(index, closer); + + // baseColumnSelectorFactory using baseOffset is the column selector for filtering. + final VectorColumnSelectorFactory baseColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( + index, + baseOffset, + columnCache, + spec.getVirtualColumns() + ); + + if (filterToUse == null) { + return new FrameVectorCursor(baseOffset, baseColumnSelectorFactory, closer); + } else { + final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseColumnSelectorFactory); + final FilteredVectorOffset filteredOffset = FilteredVectorOffset.create( + baseOffset, + matcher + ); + + final VectorColumnSelectorFactory filteredColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( + index, + filteredOffset, + columnCache, + spec.getVirtualColumns() + ); + + return new FrameVectorCursor(filteredOffset, filteredColumnSelectorFactory, closer); + } + } + }; + } + @Override public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) { @@ -98,24 +184,7 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); - - if (Granularities.ALL.equals(gran)) { - final Closer closer = Closer.create(); - final Cursor cursor = makeGranularityAllCursor( - new ColumnCache(index, closer), - frame.numRows(), - filter, - interval, - virtualColumns, - descending - ); - - return Sequences.simple(Collections.singletonList(cursor)).withBaggage(closer); - } else { - // Not currently needed for the intended use cases of frame-based cursors. - throw new UOE("Granularity [%s] not supported", gran); - } + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } @Nullable @@ -129,42 +198,7 @@ public VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - if (!canVectorize(filter, virtualColumns, descending)) { - throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); - } - - final Closer closer = Closer.create(); - final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); - final Filter filterToUse = FrameCursorUtils.buildFilter(filter, interval); - final VectorOffset baseOffset = new NoFilterVectorOffset(vectorSize, 0, frame.numRows()); - final ColumnCache columnCache = new ColumnCache(index, closer); - - // baseColumnSelectorFactory using baseOffset is the column selector for filtering. - final VectorColumnSelectorFactory baseColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( - index, - baseOffset, - columnCache, - virtualColumns - ); - - if (filterToUse == null) { - return new FrameVectorCursor(baseOffset, baseColumnSelectorFactory, closer); - } else { - final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseColumnSelectorFactory); - final FilteredVectorOffset filteredOffset = FilteredVectorOffset.create( - baseOffset, - matcher - ); - - final VectorColumnSelectorFactory filteredColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( - index, - filteredOffset, - columnCache, - virtualColumns - ); - - return new FrameVectorCursor(filteredOffset, filteredColumnSelectorFactory, closer); - } + return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); } private static Cursor makeGranularityAllCursor( diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java index b3232e15d7f6..d5977ba6d478 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java @@ -35,7 +35,9 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; @@ -71,58 +73,55 @@ public FrameCursorFactory( } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - if (Granularities.ALL.equals(gran)) { - final Cursor cursor = makeGranularityAllCursor(filter, interval, virtualColumns, descending); - - // Note: if anything closeable is ever added to this Sequence, make sure to update FrameProcessors.makeCursor. - // Currently, it assumes that closing the Sequence does nothing. - return Sequences.simple(Collections.singletonList(cursor)); - } else { + if (!Granularities.ALL.equals(spec.getGranularity())) { // Not currently needed for the intended use cases of frame-based cursors. - throw new UOE("Granularity [%s] not supported", gran); + throw new UOE("Granularity [%s] not supported", spec.getGranularity()); } - } + return () -> { + final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); - private Cursor makeGranularityAllCursor( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final boolean descending - ) - { - final Filter filterToUse = FrameCursorUtils.buildFilter(filter, interval); + final SimpleSettableOffset baseOffset = spec.isDescending() + ? new SimpleDescendingOffset(frame.numRows()) + : new SimpleAscendingOffset(frame.numRows()); - final SimpleSettableOffset baseOffset = descending - ? new SimpleDescendingOffset(frame.numRows()) - : new SimpleAscendingOffset(frame.numRows()); + final SimpleSettableOffset offset; - final SimpleSettableOffset offset; + final ColumnSelectorFactory columnSelectorFactory = + spec.getVirtualColumns().wrap( + new FrameColumnSelectorFactory( + frame, + frameReader.signature(), + fieldReaders, + new CursorFrameRowPointer(frame, baseOffset) + ) + ); - final ColumnSelectorFactory columnSelectorFactory = - virtualColumns.wrap( - new FrameColumnSelectorFactory( - frame, - frameReader.signature(), - fieldReaders, - new CursorFrameRowPointer(frame, baseOffset) - ) - ); + if (filterToUse == null) { + offset = baseOffset; + } else { + offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse); + } - if (filterToUse == null) { - offset = baseOffset; - } else { - offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse); - } + final FrameCursor cursor = new FrameCursor(offset, columnSelectorFactory); - return new FrameCursor(offset, columnSelectorFactory); + // Note: if anything closeable is ever added to this Sequence, make sure to update FrameProcessors.makeCursor. + // Currently, it assumes that closing the Sequence does nothing. + return Sequences.simple(Collections.singletonList(cursor)); + }; + } + + @Override + public Sequence makeCursors( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + Granularity gran, + boolean descending, + @Nullable QueryMetrics queryMetrics + ) + { + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } } diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index a0b7bda150d0..4edbd9170689 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -24,15 +24,14 @@ import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; import java.util.Collections; @@ -89,8 +88,9 @@ public Sequence getRowsAsSequence() Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); - return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); + return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( + CursorBuildSpec.FULL_SCAN + ).makeCursors(); } ); diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 90b43469dce5..4ce65c90f793 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.granularity.Granularity; @@ -39,13 +41,17 @@ import org.apache.druid.query.timeboundary.TimeBoundaryQuery; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.filter.Filters; import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -271,4 +277,27 @@ default Set getRequiredColumns() { return null; } + + default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + { + final Set columns = getRequiredColumns(); + final List intervals = getIntervals(); + if (intervals.size() > 1) { + throw DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + intervals + ); + } + return new CursorBuildSpec( + Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), + Iterables.getOnlyElement(intervals), + getGranularity(), + columns == null ? Collections.emptyList() : new ArrayList<>(columns), + getVirtualColumns(), + Collections.emptyList(), + context(), + isDescending(), + queryMetrics + ); + } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 61520a04bc28..635ac2e8c48e 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -29,7 +29,6 @@ import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; - import java.math.BigDecimal; import java.util.Arrays; import java.util.HashMap; @@ -138,6 +137,12 @@ public boolean shouldVectorize(final boolean canVectorize) { return false; } + + @Override + public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) + { + return false; + } }, TRUE { @Override @@ -145,6 +150,12 @@ public boolean shouldVectorize(final boolean canVectorize) { return canVectorize; } + + @Override + public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) + { + return canVectorize; + } }, FORCE { @Override @@ -156,10 +167,26 @@ public boolean shouldVectorize(final boolean canVectorize) return true; } + + @Override + public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) + { + if (!canVectorize) { + cleanup.run(); + throw new ISE("Cannot vectorize!"); + } + + return true; + } }; public abstract boolean shouldVectorize(boolean canVectorize); + public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) + { + return shouldVectorize(canVectorize); + } + @JsonCreator public static Vectorize fromString(String str) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java b/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java index 88b12e81c8b8..bb9837bac2d0 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java +++ b/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java @@ -19,50 +19,16 @@ package org.apache.druid.query; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; -import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.Closeable; -import java.util.List; -import java.util.Objects; /** */ public class QueryRunnerHelper { - public static Sequence> makeCursorBasedQuery( - final StorageAdapter adapter, - final List queryIntervals, - final Filter filter, - final VirtualColumns virtualColumns, - final boolean descending, - final Granularity granularity, - final Function> mapFn, - @Nullable final QueryMetrics queryMetrics - ) - { - Preconditions.checkArgument( - queryIntervals.size() == 1, "Can only handle a single interval, got[%s]", queryIntervals - ); - - return Sequences.filter( - Sequences.map( - adapter.makeCursors(filter, queryIntervals.get(0), virtualColumns, granularity, descending, queryMetrics), - mapFn - ), - Objects::nonNull - ); - } public static QueryRunner makeClosingQueryRunner(final QueryRunner runner, final Closeable closeable) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index 99ad72f0ee93..60c175c9f916 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java @@ -25,6 +25,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Cacheable; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.CursorBuildSpec; import javax.annotation.Nullable; import java.util.Set; @@ -76,7 +77,8 @@ public interface DimFilter extends Cacheable * * The Filter returned by this method across multiple calls must be the same object: parts of the query stack * compare Filters, and returning the same object allows these checks to avoid deep comparisons. - * (see {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter#makeCursors for an example} + * (see {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter#asCursorMaker(CursorBuildSpec)} + * for an example} * * @param mayIncludeUnknown whether the optimized filter may need to operate in "includeUnknown" mode. * See {@link NullHandling#useThreeValueLogic()}. diff --git a/processing/src/main/java/org/apache/druid/query/filter/Filter.java b/processing/src/main/java/org/apache/druid/query/filter/Filter.java index 9fae7d51d208..f30681c86691 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/Filter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/Filter.java @@ -106,7 +106,8 @@ default FilterBundle makeFilterBundle( matcherBundle = new FilterBundle.SimpleMatcherBundle( new FilterBundle.MatcherBundleInfo(this::toString, null, null), this::makeMatcher, - this::makeVectorMatcher + this::makeVectorMatcher, + this.canVectorizeMatcher(columnIndexSelector) ); } else { matcherBundle = null; diff --git a/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java b/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java index 695eb26c00b6..e105b0d6163b 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java +++ b/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java @@ -118,6 +118,11 @@ public boolean hasMatcher() return matcherBundle != null; } + public boolean canVectorizeMatcher() + { + return matcherBundle == null || matcherBundle.canVectorize(); + } + public interface IndexBundle { IndexBundleInfo getIndexInfo(); @@ -142,6 +147,8 @@ public interface MatcherBundle ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending); VectorValueMatcher vectorMatcher(VectorColumnSelectorFactory selectorFactory, ReadableVectorOffset baseOffset); + + boolean canVectorize(); } public static class SimpleIndexBundle implements IndexBundle @@ -182,15 +189,19 @@ public static class SimpleMatcherBundle implements MatcherBundle private final Function matcherFn; private final Function vectorMatcherFn; + private final boolean canVectorize; + public SimpleMatcherBundle( MatcherBundleInfo matcherInfo, Function matcherFn, - Function vectorMatcherFn + Function vectorMatcherFn, + boolean canVectorize ) { this.matcherInfo = Preconditions.checkNotNull(matcherInfo); this.matcherFn = Preconditions.checkNotNull(matcherFn); this.vectorMatcherFn = Preconditions.checkNotNull(vectorMatcherFn); + this.canVectorize = canVectorize; } @Override @@ -217,6 +228,12 @@ public VectorValueMatcher vectorMatcher( { return vectorMatcherFn.apply(selectorFactory); } + + @Override + public boolean canVectorize() + { + return canVectorize; + } } public static class BundleInfo 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 cdcf9e3daf40..2dbcee9a4d2d 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 @@ -28,11 +28,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.primitives.Longs; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -47,6 +49,7 @@ import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; @@ -62,12 +65,14 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.filter.Filters; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -838,6 +843,30 @@ public Set getRequiredColumns() ); } + @Override + public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + { + final Set columns = getRequiredColumns(); + final List intervals = getIntervals(); + if (intervals.size() > 1) { + throw DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + intervals + ); + } + return new CursorBuildSpec( + Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), + Iterables.getOnlyElement(intervals), + getGranularity(), + columns == null ? Collections.emptyList() : new ArrayList<>(columns), + getVirtualColumns(), + getAggregatorSpecs(), + context(), + isDescending(), + queryMetrics + ); + } + @Override public GroupByQuery withOverriddenContext(Map contextOverride) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 6451fb9b943d..44f407da283f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -63,7 +63,6 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.groupby.epinephelinae.BufferArrayGrouper; import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner; import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine; @@ -74,6 +73,9 @@ import org.apache.druid.query.groupby.orderby.LimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -81,7 +83,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory; import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; @@ -495,38 +496,39 @@ public Sequence process( ? null : DateTimes.utc(Long.parseLong(fudgeTimestampString)); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); + // group by specific vectorization check: - final boolean doVectorize = query.context().getVectorize().shouldVectorize( - VectorGroupByEngine.canVectorize(query, storageAdapter, filter) - ); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(groupByQueryMetrics); + final CursorMaker cursorMaker = storageAdapter.asCursorMaker(buildSpec); + final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); + final boolean canVectorize = cursorMaker.canVectorize() && + VectorGroupByEngine.canVectorizeDimensions(inspector, query.getDimensions()); + final boolean shouldVectorize = query.context().getVectorize().shouldVectorize( + canVectorize, + cursorMaker::cleanup + ); final Sequence result; - - if (doVectorize) { + if (shouldVectorize) { result = VectorGroupByEngine.process( query, storageAdapter, + cursorMaker, bufferHolder.get(), fudgeTimestamp, - filter, - interval, + buildSpec.getInterval(), querySpecificConfig, - processingConfig, - groupByQueryMetrics + processingConfig ); } else { result = GroupByQueryEngine.process( query, storageAdapter, + cursorMaker, bufferHolder.get(), fudgeTimestamp, querySpecificConfig, - processingConfig, - filter, - interval, - groupByQueryMetrics + processingConfig ); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index 085e6022aabd..162b749c1d6c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -34,7 +34,6 @@ import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryMetrics; @@ -48,6 +47,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; @@ -56,7 +56,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.Closeable; @@ -94,23 +93,14 @@ private GroupByQueryEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, + final CursorMaker cursorMaker, final ByteBuffer processingBuffer, @Nullable final DateTime fudgeTimestamp, final GroupByQueryConfig querySpecificConfig, - final DruidProcessingConfig processingConfig, - @Nullable final Filter filter, - final Interval interval, - @Nullable final GroupByQueryMetrics groupByQueryMetrics + final DruidProcessingConfig processingConfig ) { - final Sequence cursors = storageAdapter.makeCursors( - filter, - interval, - query.getVirtualColumns(), - query.getGranularity(), - false, - groupByQueryMetrics - ); + final Sequence cursors = cursorMaker.makeCursors(); return cursors.flatMap( cursor -> new BaseSequence<>( diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index a2fc9cec8a6e..f589d3892c60 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -22,7 +22,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Suppliers; import org.apache.datasketches.memory.WritableMemory; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; @@ -30,7 +29,6 @@ import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryMetrics; @@ -45,17 +43,15 @@ import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; @@ -83,33 +79,21 @@ private VectorGroupByEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, + final CursorMaker cursorMaker, final ByteBuffer processingBuffer, @Nullable final DateTime fudgeTimestamp, - @Nullable final Filter filter, final Interval interval, final GroupByQueryConfig config, - final DruidProcessingConfig processingConfig, - @Nullable final GroupByQueryMetrics groupByQueryMetrics + final DruidProcessingConfig processingConfig ) { - if (!canVectorize(query, storageAdapter, filter)) { - throw new ISE("Cannot vectorize"); - } - return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override public CloseableIterator make() { - final VectorCursor cursor = storageAdapter.makeVectorCursor( - Filters.toFilter(query.getDimFilter()), - interval, - query.getVirtualColumns(), - false, - query.context().getVectorSize(), - groupByQueryMetrics - ); + final VectorCursor cursor = cursorMaker.makeVectorCursor(); if (cursor == null) { // Return empty iterator. @@ -183,23 +167,7 @@ public void cleanup(CloseableIterator iterFromMake) ); } - public static boolean canVectorize( - final GroupByQuery query, - final StorageAdapter adapter, - @Nullable final Filter filter - ) - { - final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(adapter); - - return adapter.canVectorize(filter, query.getVirtualColumns(), false) - && canVectorizeDimensions(inspector, query.getDimensions()) - && VirtualColumns.shouldVectorize(query, query.getVirtualColumns(), adapter) - && query.getAggregatorSpecs() - .stream() - .allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(inspector)); - } - - private static boolean canVectorizeDimensions( + public static boolean canVectorizeDimensions( final ColumnInspector inspector, final List dimensions ) diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index 81fcf34ed013..d1154aa47da1 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -31,11 +31,11 @@ import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -57,6 +57,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Collections; import java.util.EnumSet; import java.util.LinkedHashMap; import java.util.Map; @@ -276,16 +277,12 @@ private ColumnAnalysis analyzeStringColumn( if (analyzingSize()) { final DateTime start = storageAdapter.getMinTime(); final DateTime end = storageAdapter.getMaxTime(); - - final Sequence cursors = - storageAdapter.makeCursors( - null, - new Interval(start, end), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setColumns(Collections.singletonList(columnName)) + .setInterval(new Interval(start, end)) + .setGranularity(Granularities.ALL) + .build(); + final Sequence cursors = storageAdapter.asCursorMaker(buildSpec).makeCursors(); size = cursors.accumulate( 0L, diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 6c2647aaf768..da07c3c2587c 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -47,6 +47,7 @@ import org.apache.druid.query.rowsandcols.semantic.WireTransferable; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -207,19 +208,11 @@ private void reset(RowsAndColumns rac) @Nullable private Pair materializeStorageAdapter(StorageAdapter as) { - final Sequence cursors = as.makeCursors( - filter, - interval == null ? Intervals.ETERNITY : interval, - virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns, - Granularities.ALL, - false, - null - ); - - final Collection cols; + final List cursorColumns; if (viewableColumns != null) { cols = viewableColumns; + cursorColumns = new ArrayList<>(viewableColumns); } else { if (virtualColumns == null) { cols = base.getColumnNames(); @@ -229,7 +222,17 @@ private Pair materializeStorageAdapter(StorageAdapter as) .addAll(virtualColumns.getColumnNames()) .build(); } + cursorColumns = new ArrayList<>(base.getColumnNames()); } + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(interval == null ? Intervals.ETERNITY : interval) + .setGranularity(Granularities.ALL) + .setColumns(cursorColumns) + .setVirtualColumns(virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns) + .build(); + final Sequence cursors = as.asCursorMaker(buildSpec).makeCursors(); + AtomicReference siggy = new AtomicReference<>(null); FrameWriter writer = cursors.accumulate(null, (accumulated, in) -> { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java index bcc1d075cbe3..d38aae8b6f44 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java @@ -26,16 +26,14 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; import org.apache.druid.segment.CloseableShapeshifter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nonnull; @@ -99,14 +97,7 @@ protected RowsAndColumns getRealRAC() @Nonnull private static RowsAndColumns materialize(StorageAdapter as) { - final Sequence cursors = as.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = as.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); RowSignature rowSignature = as.getRowSignature(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index ac36b106c642..1583c57d8f26 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -27,6 +27,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; @@ -37,9 +38,11 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.Queries; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -47,8 +50,11 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature.Builder; +import org.apache.druid.segment.filter.Filters; +import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -497,6 +503,30 @@ public Set getRequiredColumns() } } + @Override + public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + { + final Set columns = getRequiredColumns(); + final List intervals = getIntervals(); + if (intervals.size() > 1) { + throw DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + intervals + ); + } + return new CursorBuildSpec( + Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), + Iterables.getOnlyElement(intervals), + getGranularity(), + columns == null ? Collections.emptyList() : new ArrayList<>(columns), + getVirtualColumns(), + Collections.emptyList(), + context(), + isDescending() || Order.DESCENDING.equals(timeOrder), + queryMetrics + ); + } + public ScanQuery withOffset(final long newOffset) { return Druids.ScanQueryBuilder.copy(this).offset(newOffset).build(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 2246e349cbb4..6a189f6b4d74 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -27,14 +27,12 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; @@ -42,7 +40,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -125,22 +122,12 @@ public Sequence process( final SegmentId segmentId = segment.getId(); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); return Sequences.concat( - adapter - .makeCursors( - filter, - intervals.get(0), - query.getVirtualColumns(), - Granularities.ALL, - query.getTimeOrder().equals(ScanQuery.Order.DESCENDING) || - (query.getTimeOrder().equals(ScanQuery.Order.NONE) && query.isDescending()), - queryMetrics - ) + adapter.asCursorMaker(query.asCursorBuildSpec(queryMetrics)) + .makeCursors() .map(cursor -> new BaseSequence<>( new BaseSequence.IteratorMaker>() { diff --git a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index 77325c58233c..435cd1baa9f9 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -85,15 +85,7 @@ public CursorBasedExecutor( public Object2IntRBTreeMap execute(final int limit) { final StorageAdapter adapter = segment.asStorageAdapter(); - - final Sequence cursors = adapter.makeCursors( - filter, - interval, - query.getVirtualColumns(), - query.getGranularity(), - query.isDescending(), - null - ); + final Sequence cursors = adapter.asCursorMaker(query.asCursorBuildSpec(null)).makeCursors(); final Object2IntRBTreeMap retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator()); retVal.defaultReturnValue(0); diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 7a695987fb76..cbfe856ab182 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -24,16 +24,15 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.ChainedExecutionQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; -import org.apache.druid.query.QueryRunnerHelper; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; @@ -41,17 +40,17 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.Iterator; import java.util.List; +import java.util.Objects; /** * @@ -116,15 +115,15 @@ public Result apply(Cursor cursor) private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legacyQuery, boolean descending) { - final Sequence> resultSequence = QueryRunnerHelper.makeCursorBasedQuery( - adapter, - legacyQuery.getQuerySegmentSpec().getIntervals(), - Filters.toFilter(legacyQuery.getFilter()), - VirtualColumns.EMPTY, - descending, - Granularities.ALL, - this.skipToFirstMatching, - null + final CursorBuildSpec buildSpec = CursorBuildSpec.builder(legacyQuery.asCursorBuildSpec(null)) + .isDescending(descending) + .build(); + final Sequence> resultSequence = Sequences.filter( + Sequences.map( + adapter.asCursorMaker(buildSpec).makeCursors(), + this.skipToFirstMatching + ), + Objects::nonNull ); final List> resultList = resultSequence.limit(1).toList(); if (resultList.size() > 0) { @@ -201,7 +200,7 @@ public void cleanup(Iterator> toClean) /** * Whether a particular {@link TimeBoundaryQuery} can use {@link StorageAdapter#getMinTime()} and/or - * {@link StorageAdapter#getMaxTime()}. If false, must use {@link StorageAdapter#makeCursors}. + * {@link StorageAdapter#getMaxTime()}. If false, must use {@link StorageAdapter#asCursorMaker(CursorBuildSpec)}. */ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, final StorageAdapter adapter) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 52fddccdf7b5..8d60ea86d341 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -25,6 +25,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; @@ -32,13 +34,17 @@ import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec.LimitJsonIncludeFilter; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.filter.Filters; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; @@ -180,6 +186,31 @@ public Set getRequiredColumns() ); } + + @Override + public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + { + final Set columns = getRequiredColumns(); + final List intervals = getIntervals(); + if (intervals.size() > 1) { + throw DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + intervals + ); + } + return new CursorBuildSpec( + Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), + Iterables.getOnlyElement(intervals), + getGranularity(), + columns == null ? Collections.emptyList() : new ArrayList<>(columns), + getVirtualColumns(), + getAggregatorSpecs(), + context(), + isDescending(), + queryMetrics + ); + } + @Override public TimeseriesQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index c5e83b84e87c..fd387461b32b 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -31,25 +31,20 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.QueryRunnerHelper; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.vector.VectorCursorGranularizer; -import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -93,25 +88,18 @@ public Sequence> process( ); } - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); final Interval interval = Iterables.getOnlyElement(query.getIntervals()); final Granularity gran = query.getGranularity(); - final boolean descending = query.isDescending(); - final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(adapter); - final boolean doVectorize = query.context().getVectorize().shouldVectorize( - adapter.canVectorize(filter, query.getVirtualColumns(), descending) - && VirtualColumns.shouldVectorize(query, query.getVirtualColumns(), adapter) - && query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(inspector)) - ); + final CursorMaker cursorMaker = adapter.asCursorMaker(query.asCursorBuildSpec(timeseriesQueryMetrics)); final Sequence> result; - if (doVectorize) { - result = processVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics); + if (query.context().getVectorize().shouldVectorize(cursorMaker.canVectorize(), cursorMaker::cleanup)) { + result = processVectorized(query, adapter, cursorMaker, interval, gran); } else { - result = processNonVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics); + result = processNonVectorized(query, cursorMaker); } final int limit = query.getLimit(); @@ -125,24 +113,15 @@ public Sequence> process( private Sequence> processVectorized( final TimeseriesQuery query, final StorageAdapter adapter, - @Nullable final Filter filter, + final CursorMaker cursorMaker, final Interval queryInterval, - final Granularity gran, - final boolean descending, - final TimeseriesQueryMetrics timeseriesQueryMetrics + final Granularity gran ) { final boolean skipEmptyBuckets = query.isSkipEmptyBuckets(); final List aggregatorSpecs = query.getAggregatorSpecs(); - final VectorCursor cursor = adapter.makeVectorCursor( - filter, - queryInterval, - query.getVirtualColumns(), - descending, - query.context().getVectorSize(), - timeseriesQueryMetrics - ); + final VectorCursor cursor = cursorMaker.makeVectorCursor(); if (cursor == null) { return Sequences.empty(); @@ -251,61 +230,53 @@ private Sequence> processVectorized( private Sequence> processNonVectorized( final TimeseriesQuery query, - final StorageAdapter adapter, - @Nullable final Filter filter, - final Interval queryInterval, - final Granularity gran, - final boolean descending, - final TimeseriesQueryMetrics timeseriesQueryMetrics + final CursorMaker cursorMaker ) { final boolean skipEmptyBuckets = query.isSkipEmptyBuckets(); final List aggregatorSpecs = query.getAggregatorSpecs(); - return QueryRunnerHelper.makeCursorBasedQuery( - adapter, - Collections.singletonList(queryInterval), - filter, - query.getVirtualColumns(), - descending, - gran, - cursor -> { - if (skipEmptyBuckets && cursor.isDone()) { - return null; - } - - Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()]; - String[] aggregatorNames = new String[aggregatorSpecs.size()]; - - for (int i = 0; i < aggregatorSpecs.size(); i++) { - aggregators[i] = aggregatorSpecs.get(i).factorize(cursor.getColumnSelectorFactory()); - aggregatorNames[i] = aggregatorSpecs.get(i).getName(); - } - - try { - while (!cursor.isDone()) { - for (Aggregator aggregator : aggregators) { - aggregator.aggregate(); + return Sequences.filter( + Sequences.map( + cursorMaker.makeCursors(), + cursor -> { + if (skipEmptyBuckets && cursor.isDone()) { + return null; } - cursor.advance(); - } - TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); + Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()]; + String[] aggregatorNames = new String[aggregatorSpecs.size()]; - for (int i = 0; i < aggregatorSpecs.size(); i++) { - bob.addMetric(aggregatorNames[i], aggregators[i].get()); - } + for (int i = 0; i < aggregatorSpecs.size(); i++) { + aggregators[i] = aggregatorSpecs.get(i).factorize(cursor.getColumnSelectorFactory()); + aggregatorNames[i] = aggregatorSpecs.get(i).getName(); + } - return bob.build(); - } - finally { - // cleanup - for (Aggregator agg : aggregators) { - agg.close(); + try { + while (!cursor.isDone()) { + for (Aggregator aggregator : aggregators) { + aggregator.aggregate(); + } + cursor.advance(); + } + + TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); + + for (int i = 0; i < aggregatorSpecs.size(); i++) { + bob.addMetric(aggregatorNames[i], aggregators[i].get()); + } + + return bob.build(); + } + finally { + // cleanup + for (Aggregator agg : aggregators) { + agg.close(); + } + } } - } - }, - timeseriesQueryMetrics + ), + Objects::nonNull ); } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 349e5a02d163..68cf87335191 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -24,19 +24,25 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.filter.Filters; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; @@ -177,6 +183,31 @@ public Set getRequiredColumns() ); } + + @Override + public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + { + final Set columns = getRequiredColumns(); + final List intervals = getIntervals(); + if (intervals.size() > 1) { + throw DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + intervals + ); + } + return new CursorBuildSpec( + Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), + Iterables.getOnlyElement(intervals), + getGranularity(), + columns == null ? Collections.emptyList() : new ArrayList<>(columns), + getVirtualColumns(), + getAggregatorSpecs(), + context(), + isDescending(), + queryMetrics + ); + } + public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) { if (dimensionSpec.getExtractionFn() != null) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index 50b8a30d1028..632de4461ff2 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -19,28 +19,22 @@ package org.apache.druid.query.topn; -import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import org.apache.druid.collections.NonBlockingPool; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.extraction.ExtractionFn; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.filter.Filters; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.List; /** */ @@ -72,27 +66,11 @@ public Sequence> query( ); } - final List queryIntervals = query.getQuerySegmentSpec().getIntervals(); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter())); - final Granularity granularity = query.getGranularity(); final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); - Preconditions.checkArgument( - queryIntervals.size() == 1, - "Can only handle a single interval, got[%s]", - queryIntervals - ); - return Sequences.filter( Sequences.map( - adapter.makeCursors( - filter, - queryIntervals.get(0), - query.getVirtualColumns(), - granularity, - query.isDescending(), - queryMetrics - ), + adapter.asCursorMaker(query.asCursorBuildSpec(queryMetrics)).makeCursors(), input -> { if (queryMetrics != null) { queryMetrics.cursor(input); diff --git a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java index e7423b5b89d9..fd413f913527 100644 --- a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java @@ -46,8 +46,8 @@ public class ArrayListSegment implements Segment * Create a list-based segment. *

* The provided List must be in time-order according to the provided {@link RowAdapter#timestampFunction()}. - * The cursor returned by {@link RowBasedStorageAdapter#makeCursors} makes no attempt to verify this, and callers - * will expect it. + * The cursor returned by {@link RowBasedStorageAdapter#asCursorMaker(CursorBuildSpec)} makes no attempt to verify + * this, and callers will expect it. *

* The provided "rowSignature" will be used for reporting available columns and their capabilities to users of * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 925c7387e534..3f8d1af1630b 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -23,7 +23,7 @@ /** * Cursor is an interface for iteration over a range of data points, used during query execution. {@link - * QueryableIndexCursorSequenceBuilder.QueryableIndexCursor} is an implementation for historical segments, and {@link + * QueryableIndexCursorMaker.QueryableIndexCursor} is an implementation for historical segments, and {@link * org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter.IncrementalIndexCursor} is an implementation for {@link * org.apache.druid.segment.incremental.IncrementalIndex}. * diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java new file mode 100644 index 000000000000..19265a013d22 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -0,0 +1,240 @@ +/* + * 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.segment; + +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.filter.Filter; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; + +public class CursorBuildSpec +{ + public static CursorBuildSpec FULL_SCAN = CursorBuildSpec.builder().setGranularity(Granularities.ALL).build(); + + public static CursorBuildSpecBuilder builder() + { + return new CursorBuildSpecBuilder(); + } + + public static CursorBuildSpecBuilder builder(CursorBuildSpec spec) + { + return new CursorBuildSpecBuilder(spec); + } + + @Nullable + private final Filter filter; + private final Interval interval; + private final Granularity granularity; + @Nullable + private final List columns; + private final VirtualColumns virtualColumns; + @Nullable + private final List aggregators; + + private final QueryContext queryContext; + + private final boolean descending; + @Nullable + private final QueryMetrics queryMetrics; + + public CursorBuildSpec( + @Nullable Filter filter, + Interval interval, + Granularity granularity, + @Nullable List columns, + VirtualColumns virtualColumns, + @Nullable List aggregators, + QueryContext queryContext, + boolean descending, + @Nullable QueryMetrics queryMetrics + ) + { + this.filter = filter; + this.interval = interval; + this.granularity = granularity; + this.columns = columns; + this.virtualColumns = virtualColumns; + this.aggregators = aggregators; + this.descending = descending; + this.queryContext = queryContext; + this.queryMetrics = queryMetrics; + } + + @Nullable + public Filter getFilter() + { + return filter; + } + + public Interval getInterval() + { + return interval; + } + + public Granularity getGranularity() + { + return granularity; + } + + @Nullable + public List getColumns() + { + return columns; + } + + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + public List getAggregators() + { + return aggregators; + } + + public boolean isDescending() + { + return descending; + } + + public QueryContext getQueryContext() + { + return queryContext; + } + + @Nullable + public QueryMetrics getQueryMetrics() + { + return queryMetrics; + } + + public static class CursorBuildSpecBuilder + { + @Nullable + private Filter filter; + private Interval interval = Intervals.ETERNITY; + private Granularity granularity = Granularities.NONE; + + @Nullable + private List columns = null; + private VirtualColumns virtualColumns = VirtualColumns.EMPTY; + @Nullable + private List aggregators = null; + private boolean descending = false; + + private QueryContext queryContext = QueryContext.empty(); + @Nullable + private QueryMetrics queryMetrics; + + public CursorBuildSpecBuilder() + { + + } + + public CursorBuildSpecBuilder(CursorBuildSpec buildSpec) + { + this.filter = buildSpec.filter; + this.interval = buildSpec.interval; + this.granularity = buildSpec.granularity; + this.columns = buildSpec.columns; + this.virtualColumns = buildSpec.virtualColumns; + this.aggregators = buildSpec.aggregators; + this.descending = buildSpec.descending; + this.queryContext = buildSpec.queryContext; + this.queryMetrics = buildSpec.queryMetrics; + } + + public CursorBuildSpecBuilder setFilter(@Nullable Filter filter) + { + this.filter = filter; + return this; + } + + public CursorBuildSpecBuilder setInterval(Interval interval) + { + this.interval = interval; + return this; + } + + public CursorBuildSpecBuilder setGranularity(Granularity granularity) + { + this.granularity = granularity; + return this; + } + + public CursorBuildSpecBuilder setColumns(@Nullable List columns) + { + this.columns = columns; + return this; + } + + public CursorBuildSpecBuilder setVirtualColumns(VirtualColumns virtualColumns) + { + this.virtualColumns = virtualColumns; + return this; + } + + public CursorBuildSpecBuilder setAggregators(@Nullable List aggregators) + { + this.aggregators = aggregators; + return this; + } + + public CursorBuildSpecBuilder isDescending(boolean descending) + { + this.descending = descending; + return this; + } + + public CursorBuildSpecBuilder setQueryContext(QueryContext queryContext) + { + this.queryContext = queryContext; + return this; + } + + public CursorBuildSpecBuilder setQueryMetrics(@Nullable QueryMetrics queryMetrics) + { + this.queryMetrics = queryMetrics; + return this; + } + + public CursorBuildSpec build() + { + return new CursorBuildSpec( + filter, + interval, + granularity, + columns, + virtualColumns, + aggregators, + queryContext, + descending, + queryMetrics + ); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index 809e7b85b063..5c72d8dbd50e 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -19,8 +19,11 @@ package org.apache.druid.segment; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.vector.VectorCursor; @@ -35,12 +38,55 @@ */ public interface CursorFactory { + default CursorMaker asCursorMaker(CursorBuildSpec spec) + { + + return new CursorMaker() + { + @Override + public boolean canVectorize() + { + return CursorFactory.this.canVectorize(spec.getFilter(), spec.getVirtualColumns(), spec.isDescending()); + } + + @Override + public Sequence makeCursors() + { + return CursorFactory.this.makeCursors( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + spec.getGranularity(), + spec.isDescending(), + spec.getQueryMetrics() + ); + } + + @Override + public VectorCursor makeVectorCursor() + { + return CursorFactory.this.makeVectorCursor( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + spec.isDescending(), + spec.getQueryContext().getVectorSize(), + spec.getQueryMetrics() + ); + } + }; + } + /** * Returns true if the provided combination of parameters can be handled by "makeVectorCursor". * * Query engines should use this before running in vectorized mode, and be prepared to fall back to non-vectorized * mode if this method returns false. + * + * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#canVectorize()}. + * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. */ + @Deprecated default boolean canVectorize( @Nullable Filter filter, VirtualColumns virtualColumns, @@ -52,7 +98,23 @@ default boolean canVectorize( /** * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity). + * + * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#makeCursors()}. + * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. Recommend for implementors to fill + * this method in with: + *

+   *     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
+   *                                                      .setFilter(filter)
+   *                                                      .setInterval(interval)
+   *                                                      .setGranularity(gran)
+   *                                                      .setVirtualColumns(virtualColumns)
+   *                                                      .isDescending(descending)
+   *                                                      .setQueryMetrics(queryMetrics)
+   *                                                      .build();
+   *     return asCursorMaker(buildSpec).makeCursors();
+   * 
*/ + @Deprecated Sequence makeCursors( @Nullable Filter filter, Interval interval, @@ -69,7 +131,12 @@ Sequence makeCursors( * * Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval * of this segment). + * + * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call + * {@link CursorMaker#makeVectorCursor()}. Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} + * instead. */ + @Deprecated @Nullable default VectorCursor makeVectorCursor( @Nullable Filter filter, @@ -82,4 +149,49 @@ default VectorCursor makeVectorCursor( { throw new UnsupportedOperationException("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); } + + default Sequence delegateMakeCursorToMaker( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + Granularity gran, + boolean descending, + @Nullable QueryMetrics queryMetrics + ) + { + return asCursorMaker( + CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(interval) + .setVirtualColumns(virtualColumns) + .setGranularity(gran) + .isDescending(descending) + .setQueryMetrics(queryMetrics) + .build() + ).makeCursors(); + } + + default VectorCursor delegateMakeVectorCursorToMaker( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + boolean descending, + int vectorSize, + @Nullable QueryMetrics queryMetrics + ) + { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(interval) + .setVirtualColumns(virtualColumns) + .isDescending(descending) + .setQueryContext( + QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, vectorSize) + ) + ) + .setQueryMetrics(queryMetrics) + .build(); + return asCursorMaker(buildSpec).makeVectorCursor(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java b/processing/src/main/java/org/apache/druid/segment/CursorMaker.java new file mode 100644 index 000000000000..eef7bc410e53 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CursorMaker.java @@ -0,0 +1,74 @@ +/* + * 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.segment; + +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.vector.VectorCursor; + +import javax.annotation.Nullable; + +public interface CursorMaker +{ + default boolean canVectorize() + { + return false; + } + + Sequence makeCursors(); + + @Nullable + default VectorCursor makeVectorCursor() + { + throw new UOE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor' on %s.", this.getClass().getName()); + } + + /** + * Release any resources acquired if cursors are not consumed. Typically consuming a cursor or vector cursor releases + * the resources upon completion, but if for some reason this will not happen, this method must be called. + */ + default void cleanup() + { + // nothing to cleanup + } + + CursorMaker EMPTY = new CursorMaker() + { + @Override + public boolean canVectorize() + { + return true; + } + + @Override + public Sequence makeCursors() + { + return Sequences.empty(); + } + + @Nullable + @Override + public VectorCursor makeVectorCursor() + { + return null; + } + }; +} diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index 2b32860698fa..e2f8ecadb681 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -44,6 +44,24 @@ public FilteredStorageAdapter(final StorageAdapter adapter, final DimFilter filt this.filterOnDataSource = filter; } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + final CursorBuildSpec.CursorBuildSpecBuilder buildSpecBuilder = CursorBuildSpec.builder(spec); + final Filter newFilter; + if (spec.getFilter() == null) { + if (filterOnDataSource != null) { + newFilter = filterOnDataSource.toFilter(); + } else { + newFilter = null; + } + } else { + newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter())); + } + buildSpecBuilder.setFilter(newFilter); + return baseStorageAdapter.asCursorMaker(buildSpecBuilder.build()); + } + @Override public Sequence makeCursors( @Nullable Filter filter, @@ -54,17 +72,7 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - final Filter andFilter; - if (filter == null) { - if (filterOnDataSource != null) { - andFilter = filterOnDataSource.toFilter(); - } else { - andFilter = null; - } - } else { - andFilter = new AndFilter(ImmutableList.of(filter, filterOnDataSource.toFilter())); - } - return baseStorageAdapter.makeCursors(andFilter, interval, virtualColumns, gran, descending, queryMetrics); + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java similarity index 65% rename from processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java rename to processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 86069068eaf9..b0ce54d66d08 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -21,10 +21,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -34,7 +37,9 @@ import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterBundle; import org.apache.druid.query.filter.ValueMatcher; @@ -57,61 +62,98 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; +import java.util.List; +import java.util.Objects; import java.util.concurrent.TimeUnit; -public class QueryableIndexCursorSequenceBuilder +public class QueryableIndexCursorMaker implements CursorMaker { - private static final Logger log = new Logger(QueryableIndexCursorSequenceBuilder.class); + private static final Logger log = new Logger(QueryableIndexCursorMaker.class); private final QueryableIndex index; private final Interval interval; + private final Granularity gran; private final VirtualColumns virtualColumns; + + @Nullable + private final List aggregatorFactories; @Nullable private final Filter filter; @Nullable private final QueryMetrics metrics; - private final long minDataTimestamp; - private final long maxDataTimestamp; private final boolean descending; + private final QueryContext queryContext; + private final int vectorSize; + private final Supplier resourcesSupplier; - public QueryableIndexCursorSequenceBuilder( + public QueryableIndexCursorMaker( QueryableIndex index, - Interval interval, - VirtualColumns virtualColumns, - @Nullable Filter filter, - @Nullable QueryMetrics metrics, - long minDataTimestamp, - long maxDataTimestamp, - boolean descending + CursorBuildSpec cursorBuildSpec ) { this.index = index; - this.interval = interval; - this.virtualColumns = virtualColumns; - this.filter = filter; - this.metrics = metrics; - this.minDataTimestamp = minDataTimestamp; - this.maxDataTimestamp = maxDataTimestamp; - this.descending = descending; + this.interval = cursorBuildSpec.getInterval(); + this.gran = cursorBuildSpec.getGranularity(); + this.virtualColumns = cursorBuildSpec.getVirtualColumns(); + this.aggregatorFactories = cursorBuildSpec.getAggregators(); + this.filter = cursorBuildSpec.getFilter(); + this.descending = cursorBuildSpec.isDescending(); + this.queryContext = cursorBuildSpec.getQueryContext(); + this.vectorSize = cursorBuildSpec.getQueryContext().getVectorSize(); + this.metrics = cursorBuildSpec.getQueryMetrics(); + this.resourcesSupplier = Suppliers.memoize(CursorResources::new); } - public Sequence build(final Granularity gran) + @Override + public boolean canVectorize() { - final Closer closer = Closer.create(); + final ColumnInspector inspector = virtualColumns.wrapInspector(index); + if (!virtualColumns.isEmpty()) { + if (!queryContext.getVectorizeVirtualColumns().shouldVectorize(virtualColumns.canVectorize(inspector))) { + return false; + } + } + if (aggregatorFactories != null) { + for (AggregatorFactory factory : aggregatorFactories) { + if (!factory.canVectorize(inspector)) { + return false; + } + } + } - // Column caches shared amongst all cursors in this sequence. - final ColumnCache columnCache = new ColumnCache(index, closer); + final CursorResources resources = resourcesSupplier.get(); + try { + final FilterBundle filterBundle = resources.filterBundle; + if (filterBundle != null) { + if (!filterBundle.canVectorizeMatcher()) { + return false; + } + } - final Offset baseOffset; + // vector cursors can't iterate backwards yet + return !descending; + } + catch (ISE rethrow) { + throw CloseableUtils.closeInCatch(rethrow, resources); + } + } - final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector( - index.getBitmapFactoryForDimensions(), - virtualColumns, - columnCache - ); + @Override + public Sequence makeCursors() + { + if (metrics != null) { + metrics.vectorized(false); + } + final Offset baseOffset; - final int numRows = index.getNumRows(); - final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); + final CursorResources resources = resourcesSupplier.get(); + final FilterBundle filterBundle = resources.filterBundle; + final int numRows = resources.numRows; + final long minDataTimestamp = resources.minDataTimestamp; + final long maxDataTimestamp = resources.maxDataTimestamp; + final NumericColumn timestamps = resources.timestamps; + final ColumnCache columnCache = resources.columnCache; // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter // can use an index @@ -121,123 +163,121 @@ public Sequence build(final Granularity gran) baseOffset = BitmapOffset.of(filterBundle.getIndex().getBitmap(), descending, index.getNumRows()); } - final NumericColumn timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); - Iterable iterable = gran.getIterable(interval); if (descending) { iterable = Lists.reverse(ImmutableList.copyOf(iterable)); } - return Sequences.withBaggage( - Sequences.map( - Sequences.simple(iterable), - new Function() - { - @Override - public Cursor apply(final Interval inputInterval) - { - final long timeStart = Math.max(interval.getStartMillis(), inputInterval.getStartMillis()); - final long timeEnd = Math.min( - interval.getEndMillis(), - gran.increment(inputInterval.getStartMillis()) - ); - - if (descending) { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { - break; + return Sequences.filter( + Sequences.withBaggage( + Sequences.map( + Sequences.simple(iterable), + new Function() + { + @Override + public Cursor apply(final Interval inputInterval) + { + final long timeStart = Math.max(interval.getStartMillis(), inputInterval.getStartMillis()); + final long timeEnd = Math.min( + interval.getEndMillis(), + gran.increment(inputInterval.getStartMillis()) + ); + + if (descending) { + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { + break; + } + } + } else { + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { + break; + } + } } - } - } else { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { - break; + + final Offset offset = descending ? + new DescendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeStart, + minDataTimestamp >= timeStart + ) : + new AscendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeEnd, + maxDataTimestamp < timeEnd + ); + + + final Offset baseCursorOffset = offset.clone(); + final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( + virtualColumns, + descending, + baseCursorOffset.getBaseReadableOffset(), + columnCache + ); + final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis()); + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter + // needs to use a value matcher + if (filterBundle != null && filterBundle.getMatcherBundle() != null) { + final ValueMatcher matcher = filterBundle.getMatcherBundle() + .valueMatcher( + columnSelectorFactory, + baseCursorOffset, + descending + ); + final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); + return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, myBucket); + } else { + return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket); } } } - - final Offset offset = descending ? - new DescendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeStart, - minDataTimestamp >= timeStart - ) : - new AscendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeEnd, - maxDataTimestamp < timeEnd - ); - - - final Offset baseCursorOffset = offset.clone(); - final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( - virtualColumns, - descending, - baseCursorOffset.getBaseReadableOffset(), - columnCache - ); - final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis()); - // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter - // needs to use a value matcher - if (filterBundle != null && filterBundle.getMatcherBundle() != null) { - final ValueMatcher matcher = filterBundle.getMatcherBundle() - .valueMatcher( - columnSelectorFactory, - baseCursorOffset, - descending - ); - final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); - return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, myBucket); - } else { - return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket); - } - } - } + ), + resources ), - closer + Objects::nonNull ); } - public VectorCursor buildVectorized(final int vectorSize) + @Nullable + @Override + public VectorCursor makeVectorCursor() { - // Sanity check - matches QueryableIndexStorageAdapter.canVectorize - Preconditions.checkState(!descending, "!descending"); - - final Closer closer = Closer.create(); - final ColumnCache columnCache = new ColumnCache(index, closer); - + final CursorResources resources = resourcesSupplier.get(); + final FilterBundle filterBundle = resources.filterBundle; + final long minDataTimestamp = resources.minDataTimestamp; + final long maxDataTimestamp = resources.maxDataTimestamp; + final NumericColumn timestamps = resources.timestamps; + final ColumnCache columnCache = resources.columnCache; // Wrap the remainder of cursor setup in a try, so if an error is encountered while setting it up, we don't // leak columns in the ColumnCache. + try { - final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector( - index.getBitmapFactoryForDimensions(), - virtualColumns, - columnCache - ); - final int numRows = index.getNumRows(); - final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); + // sanity check + if (!canVectorize()) { + cleanup(); + throw new IllegalStateException("canVectorize()"); + } + if (metrics != null) { + metrics.vectorized(true); + } - NumericColumn timestamps = null; final int startOffset; final int endOffset; if (interval.getStartMillis() > minDataTimestamp) { - timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); - startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows()); } else { startOffset = 0; } if (interval.getEndMillis() <= maxDataTimestamp) { - if (timestamps == null) { - timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); - } - endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows()); } else { endOffset = index.getNumRows(); @@ -271,16 +311,32 @@ public VectorCursor buildVectorized(final int vectorSize) columnCache, filteredOffset ); - return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize, closer); + return new QueryableIndexVectorCursor( + filteredColumnSelectorFactory, + filteredOffset, + vectorSize, + resources + ); } else { - return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer); + return new QueryableIndexVectorCursor( + baseColumnSelectorFactory, + baseOffset, + vectorSize, + resources + ); } } catch (Throwable t) { - throw CloseableUtils.closeAndWrapInCatch(t, closer); + throw CloseableUtils.closeAndWrapInCatch(t, resources); } } + @Override + public void cleanup() + { + CloseableUtils.closeAndWrapExceptions(resourcesSupplier.get()); + } + @Nullable private FilterBundle makeFilterBundle( ColumnSelectorColumnIndexSelector bitmapIndexSelector, @@ -391,7 +447,7 @@ static int timeSearch( private static class QueryableIndexVectorCursor implements VectorCursor { - private final Closer closer; + private final Closeable closer; private final int vectorSize; private final VectorOffset offset; private final VectorColumnSelectorFactory columnSelectorFactory; @@ -400,7 +456,7 @@ public QueryableIndexVectorCursor( final VectorColumnSelectorFactory vectorColumnSelectorFactory, final VectorOffset offset, final int vectorSize, - final Closer closer + final Closeable closer ) { this.columnSelectorFactory = vectorColumnSelectorFactory; @@ -495,7 +551,7 @@ public void advance() cursorOffset.increment(); // Must call BaseQuery.checkInterrupted() after cursorOffset.increment(), not before, because // FilteredOffset.increment() is a potentially long, not an "instant" operation (unlike to all other subclasses - // of Offset) and it returns early on interruption, leaving itself in an illegal state. We should not let + // of Offset) and it returns early on interruption, leaving itself in an illegal We should not let // aggregators, etc. access this illegal state and throw a QueryInterruptedException by calling // BaseQuery.checkInterrupted(). BaseQuery.checkInterrupted(); @@ -668,4 +724,43 @@ public Offset clone() return new DescendingTimestampCheckingOffset(baseOffset.clone(), timestamps, timeLimit, allWithinThreshold); } } + + private final class CursorResources implements Closeable + { + private final Closer closer; + private final long minDataTimestamp; + private final long maxDataTimestamp; + private final int numRows; + @Nullable + private final FilterBundle filterBundle; + private final NumericColumn timestamps; + private final ColumnCache columnCache; + + private CursorResources() + { + this.closer = Closer.create(); + this.columnCache = new ColumnCache(index, closer); + final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector( + index.getBitmapFactoryForDimensions(), + virtualColumns, + columnCache + ); + try { + this.numRows = index.getNumRows(); + this.filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); + this.timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); + this.minDataTimestamp = DateTimes.utc(timestamps.getLongSingleValueRow(0)).getMillis(); + this.maxDataTimestamp = DateTimes.utc(timestamps.getLongSingleValueRow(timestamps.length() - 1)).getMillis(); + } + catch (Throwable t) { + throw CloseableUtils.closeAndWrapInCatch(t, closer); + } + } + + @Override + public void close() throws IOException + { + closer.close(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 57c7da953605..0cbf4e7ac840 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -20,11 +20,8 @@ package org.apache.druid.segment; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.BaseColumn; @@ -43,7 +40,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.LinkedHashSet; -import java.util.Objects; import java.util.Set; /** @@ -178,6 +174,21 @@ public DateTime getMaxIngestedEventTime() return getMaxTime(); } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + final Interval actualInterval = computeCursorInterval(spec.getGranularity(), spec.getInterval()); + + if (actualInterval == null) { + return CursorMaker.EMPTY; + } + + return new QueryableIndexCursorMaker( + index, + CursorBuildSpec.builder(spec).setInterval(actualInterval).build() + ); + } + @Override public boolean canVectorize( @Nullable final Filter filter, @@ -185,12 +196,11 @@ public boolean canVectorize( final boolean descending ) { + // todo (clint): + // this uses the old-school canVectorize implementation instead of delegating to the CursorMaker, because + // the cursor maker expects to make cursors one way or another, and so opens stuff, so need to fix some tests if (filter != null) { - // ideally we would allow stuff to vectorize if we can build indexes even if the value matcher cannot be - // vectorized, this used to be true in fact, but changes to filter partitioning (FilterBundle) have caused - // the only way to know this to be building the bitmaps since BitmapColumnIndex can return null. - // this will be changed in a future refactor of cursor building, at which point this method can just return - // true if !descending... + final boolean filterCanVectorize = filter.canVectorizeMatcher(this); if (!filterCanVectorize) { @@ -213,29 +223,7 @@ public VectorCursor makeVectorCursor( @Nullable final QueryMetrics queryMetrics ) { - if (!canVectorize(filter, virtualColumns, descending)) { - throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); - } - - if (queryMetrics != null) { - queryMetrics.vectorized(true); - } - - final Interval actualInterval = computeCursorInterval(Granularities.ALL, interval); - - if (actualInterval == null) { - return null; - } - return new QueryableIndexCursorSequenceBuilder( - index, - actualInterval, - virtualColumns, - filter, - queryMetrics, - getMinTime().getMillis(), - getMaxTime().getMillis(), - descending - ).buildVectorized(vectorSize > 0 ? vectorSize : DEFAULT_VECTOR_SIZE); + return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); } @Override @@ -248,29 +236,7 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - if (queryMetrics != null) { - queryMetrics.vectorized(false); - } - - final Interval actualInterval = computeCursorInterval(gran, interval); - - if (actualInterval == null) { - return Sequences.empty(); - } - - return Sequences.filter( - new QueryableIndexCursorSequenceBuilder( - index, - actualInterval, - virtualColumns, - filter, - queryMetrics, - getMinTime().getMillis(), - getMaxTime().getMillis(), - descending - ).build(gran), - Objects::nonNull - ); + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java index d9b2cfbd3247..0d273adc7573 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java @@ -40,8 +40,8 @@ public class RowBasedSegment implements Segment * Create a row-based segment. * * The provided "rowIterable" must be in time-order according to the provided {@link RowAdapter#timestampFunction()}. - * The cursor returned by {@link RowBasedStorageAdapter#makeCursors} makes no attempt to verify this, and callers - * will expect it. + * The cursor returned by {@link RowBasedStorageAdapter#asCursorMaker(CursorBuildSpec)} makes no attempt to verify + * this, and callers will expect it. * * The provided "rowSignature" will be used for reporting available columns and their capabilities to users of * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java index ec08fd865273..7e756ffb616a 100644 --- a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java +++ b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java @@ -39,7 +39,8 @@ public interface SegmentWrangler * * @return Segments that, collectively, contain data for dataSource. May be empty if dataSource does not exist or * has no data in the provided intervals. May contain data outside the provided intervals, so callers should - * filter it down further, e.g. through the "interval" parameter of {@link StorageAdapter#makeCursors} + * filter it down further, e.g. through the "interval" parameter of {@link CursorBuildSpec} for + * {@link StorageAdapter#asCursorMaker(CursorBuildSpec)} */ Iterable getSegmentsForIntervals(DataSource dataSource, Iterable intervals); } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index e9839a37818c..b3b2f93b7441 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -89,6 +89,67 @@ public UnnestStorageAdapter( this.unnestFilter = unnestFilter; } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + final String input = getUnnestInputIfDirectAccess(unnestColumn); + final Pair filterPair = computeBaseAndPostUnnestFilters( + spec.getFilter(), + unnestFilter != null ? unnestFilter.toFilter() : null, + spec.getVirtualColumns(), + input, + input == null ? null : spec.getVirtualColumns().getColumnCapabilitiesWithFallback(baseAdapter, input) + ); + final CursorBuildSpec unnestBuildSpec = CursorBuildSpec.builder(spec) + .setFilter(filterPair.lhs) + .setVirtualColumns( + VirtualColumns.create( + Collections.singletonList(unnestColumn) + ) + ) + .build(); + return new CursorMaker() + { + @Override + public Sequence makeCursors() + { + final Sequence baseCursorSequence = baseAdapter.asCursorMaker(unnestBuildSpec).makeCursors(); + return Sequences.map( + baseCursorSequence, + cursor -> { + Objects.requireNonNull(cursor); + final ColumnCapabilities capabilities = unnestColumn.capabilities( + cursor.getColumnSelectorFactory(), + unnestColumn.getOutputName() + ); + final Cursor unnestCursor; + + if (useDimensionCursor(capabilities)) { + unnestCursor = new UnnestDimensionCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); + } else { + unnestCursor = new UnnestColumnValueSelectorCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); + } + return PostJoinCursor.wrap( + unnestCursor, + spec.getVirtualColumns(), + filterPair.rhs + ); + } + ); + } + }; + } + @Override public Sequence makeCursors( @Nullable Filter filter, @@ -99,56 +160,15 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - final String inputColumn = getUnnestInputIfDirectAccess(unnestColumn); - final Pair filterPair = computeBaseAndPostUnnestFilters( - filter, - unnestFilter != null ? unnestFilter.toFilter() : null, - virtualColumns, - inputColumn, - inputColumn == null ? null : virtualColumns.getColumnCapabilitiesWithFallback(baseAdapter, inputColumn) - ); - - final Sequence baseCursorSequence = baseAdapter.makeCursors( - filterPair.lhs, - interval, - VirtualColumns.create(Collections.singletonList(unnestColumn)), - gran, - descending, - queryMetrics - ); - - return Sequences.map( - baseCursorSequence, - cursor -> { - Objects.requireNonNull(cursor); - final ColumnCapabilities capabilities = unnestColumn.capabilities( - cursor.getColumnSelectorFactory(), - unnestColumn.getOutputName() - ); - final Cursor unnestCursor; - - if (useDimensionCursor(capabilities)) { - unnestCursor = new UnnestDimensionCursor( - cursor, - cursor.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } else { - unnestCursor = new UnnestColumnValueSelectorCursor( - cursor, - cursor.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } - return PostJoinCursor.wrap( - unnestCursor, - virtualColumns, - filterPair.rhs - ); - } - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(interval) + .setGranularity(gran) + .setVirtualColumns(virtualColumns) + .isDescending(descending) + .setQueryMetrics(queryMetrics) + .build(); + return asCursorMaker(buildSpec).makeCursors(); } @Override 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 583f0425c2a6..c60bf162861e 100644 --- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java +++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.Cacheable; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.query.Query; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.ColumnIndexSelector; @@ -48,7 +47,6 @@ import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory; import javax.annotation.Nullable; - import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -125,15 +123,6 @@ public static VirtualColumns nullToEmpty(@Nullable VirtualColumns virtualColumns return virtualColumns == null ? EMPTY : virtualColumns; } - public static boolean shouldVectorize(Query query, VirtualColumns virtualColumns, ColumnInspector inspector) - { - if (virtualColumns.getVirtualColumns().length > 0) { - return query.context().getVectorizeVirtualColumns().shouldVectorize(virtualColumns.canVectorize(inspector)); - } else { - return true; - } - } - private VirtualColumns( List virtualColumns, Map withDotSupport, diff --git a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java index 036c6a8250a1..dfc618acad89 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java @@ -188,6 +188,17 @@ public VectorValueMatcher vectorMatcher(VectorColumnSelectorFactory selectorFact } return makeVectorMatcher(vectorMatchers); } + + @Override + public boolean canVectorize() + { + for (FilterBundle.MatcherBundle bundle : matcherBundles) { + if (!bundle.canVectorize()) { + return false; + } + } + return true; + } }; } else { matcherBundle = null; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java index 0665aabf1959..700b2fbfa168 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java @@ -243,6 +243,17 @@ public VectorValueMatcher vectorMatcher( } return makeVectorMatcher(matchers); } + + @Override + public boolean canVectorize() + { + for (FilterBundle.MatcherBundle bundle : allMatcherBundles) { + if (!bundle.canVectorize()) { + return false; + } + } + return true; + } } ); } @@ -527,6 +538,12 @@ public VectorValueMatcher vectorMatcher( } ); } + + @Override + public boolean canVectorize() + { + return bundle.getMatcherBundle() == null || bundle.getMatcherBundle().canVectorize(); + } }; } @@ -585,6 +602,12 @@ public VectorValueMatcher vectorMatcher( { return convertIndexToVectorValueMatcher(baseOffset, partialIndex); } + + @Override + public boolean canVectorize() + { + return true; + } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java new file mode 100644 index 000000000000..22f58693b7c0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java @@ -0,0 +1,264 @@ +/* + * 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.segment.incremental; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.filter.ValueMatchers; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Iterator; + +public class IncrementalIndexCursorMaker implements CursorMaker +{ + private final IncrementalIndexStorageAdapter storageAdapter; + private final IncrementalIndex index; + private final CursorBuildSpec builder; + + public IncrementalIndexCursorMaker( + IncrementalIndexStorageAdapter storageAdapter, + IncrementalIndex index, + CursorBuildSpec builder + ) + { + this.storageAdapter = storageAdapter; + this.index = index; + this.builder = builder; + } + + @Override + public Sequence makeCursors() + { + if (index.isEmpty()) { + return Sequences.empty(); + } + + if (builder.getQueryMetrics() != null) { + builder.getQueryMetrics().vectorized(false); + } + + final Interval dataInterval = new Interval( + index.getMinTime(), + builder.getGranularity().bucketEnd(index.getMaxTime()) + ); + + if (!builder.getInterval().overlaps(dataInterval)) { + return Sequences.empty(); + } + final Interval actualInterval = builder.getInterval().overlap(dataInterval); + Iterable intervals = builder.getGranularity().getIterable(actualInterval); + if (builder.isDescending()) { + intervals = Lists.reverse(ImmutableList.copyOf(intervals)); + } + + return Sequences + .simple(intervals) + .map(i -> new IncrementalIndexCursor( + storageAdapter, + index, + builder.getVirtualColumns(), + builder.isDescending(), + builder.getFilter(), + i, + actualInterval, + builder.getGranularity() + )); + } + + static class IncrementalIndexCursor implements Cursor + { + private IncrementalIndexRowHolder currEntry; + private final ColumnSelectorFactory columnSelectorFactory; + private final ValueMatcher filterMatcher; + private final int maxRowIndex; + private Iterator baseIter; + private Iterable cursorIterable; + private boolean emptyRange; + private final DateTime time; + private int numAdvanced; + private boolean done; + + IncrementalIndexCursor( + IncrementalIndexStorageAdapter storageAdapter, + IncrementalIndex index, + VirtualColumns virtualColumns, + boolean descending, + @Nullable Filter filter, + Interval interval, + Interval actualInterval, + Granularity gran + ) + { + currEntry = new IncrementalIndexRowHolder(); + columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( + storageAdapter, + virtualColumns, + descending, + currEntry + ); + // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 + maxRowIndex = index.getLastRowIndex(); + filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); + numAdvanced = -1; + final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); + cursorIterable = index.getFacts().timeRangeIterable( + descending, + timeStart, + Math.min(actualInterval.getEndMillis(), gran.increment(interval.getStartMillis())) + ); + emptyRange = !cursorIterable.iterator().hasNext(); + time = gran.toDateTime(interval.getStartMillis()); + + reset(); + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return columnSelectorFactory; + } + + @Override + public DateTime getTime() + { + return time; + } + + @Override + public void advance() + { + if (!baseIter.hasNext()) { + done = true; + return; + } + + while (baseIter.hasNext()) { + BaseQuery.checkInterrupted(); + + IncrementalIndexRow entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { + continue; + } + + currEntry.set(entry); + + if (filterMatcher.matches(false)) { + return; + } + } + + done = true; + } + + @Override + public void advanceUninterruptibly() + { + if (!baseIter.hasNext()) { + done = true; + return; + } + + while (baseIter.hasNext()) { + if (Thread.currentThread().isInterrupted()) { + return; + } + + IncrementalIndexRow entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { + continue; + } + + currEntry.set(entry); + + if (filterMatcher.matches(false)) { + return; + } + } + + done = true; + } + + @Override + public boolean isDone() + { + return done; + } + + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } + + @Override + public void reset() + { + baseIter = cursorIterable.iterator(); + + if (numAdvanced == -1) { + numAdvanced = 0; + } else { + Iterators.advance(baseIter, numAdvanced); + } + + BaseQuery.checkInterrupted(); + + boolean foundMatched = false; + while (baseIter.hasNext()) { + IncrementalIndexRow entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { + numAdvanced++; + continue; + } + currEntry.set(entry); + if (filterMatcher.matches(false)) { + foundMatched = true; + break; + } + + numAdvanced++; + } + + done = !foundMatched && (emptyRange || !baseIter.hasNext()); + } + + private boolean beyondMaxRowIndex(int rowIndex) + { + // ignore rows whose rowIndex is beyond the maxRowIndex + // rows are order by timestamp, not rowIndex, + // so we still need to go through all rows to skip rows added after cursor created + return rowIndex > maxRowIndex; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 8d72133a205c..5fb9adba1d57 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -19,18 +19,13 @@ package org.apache.druid.segment.incremental; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.BaseQuery; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.Metadata; @@ -43,12 +38,10 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; -import org.apache.druid.segment.filter.ValueMatchers; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.Iterator; /** */ @@ -255,6 +248,12 @@ public DateTime getMaxIngestedEventTime() return index.getMaxIngestedEventTime(); } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + return new IncrementalIndexCursorMaker(this, index, spec); + } + @Override public Sequence makeCursors( @Nullable final Filter filter, @@ -265,28 +264,7 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - if (index.isEmpty()) { - return Sequences.empty(); - } - - if (queryMetrics != null) { - queryMetrics.vectorized(false); - } - - final Interval dataInterval = new Interval(getMinTime(), gran.bucketEnd(getMaxTime())); - - if (!interval.overlaps(dataInterval)) { - return Sequences.empty(); - } - final Interval actualInterval = interval.overlap(dataInterval); - Iterable intervals = gran.getIterable(actualInterval); - if (descending) { - intervals = Lists.reverse(ImmutableList.copyOf(intervals)); - } - - return Sequences - .simple(intervals) - .map(i -> new IncrementalIndexCursor(virtualColumns, descending, filter, i, actualInterval, gran)); + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } @Override @@ -294,168 +272,4 @@ public Metadata getMetadata() { return index.getMetadata(); } - - private class IncrementalIndexCursor implements Cursor - { - private IncrementalIndexRowHolder currEntry; - private final ColumnSelectorFactory columnSelectorFactory; - private final ValueMatcher filterMatcher; - private final int maxRowIndex; - private Iterator baseIter; - private Iterable cursorIterable; - private boolean emptyRange; - private final DateTime time; - private int numAdvanced; - private boolean done; - - IncrementalIndexCursor( - VirtualColumns virtualColumns, - boolean descending, - Filter filter, - Interval interval, - Interval actualInterval, - Granularity gran - ) - { - currEntry = new IncrementalIndexRowHolder(); - columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( - IncrementalIndexStorageAdapter.this, - virtualColumns, - descending, - currEntry - ); - // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 - maxRowIndex = index.getLastRowIndex(); - filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); - numAdvanced = -1; - final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); - cursorIterable = index.getFacts().timeRangeIterable( - descending, - timeStart, - Math.min(actualInterval.getEndMillis(), gran.increment(interval.getStartMillis())) - ); - emptyRange = !cursorIterable.iterator().hasNext(); - time = gran.toDateTime(interval.getStartMillis()); - - reset(); - } - - @Override - public ColumnSelectorFactory getColumnSelectorFactory() - { - return columnSelectorFactory; - } - - @Override - public DateTime getTime() - { - return time; - } - - @Override - public void advance() - { - if (!baseIter.hasNext()) { - done = true; - return; - } - - while (baseIter.hasNext()) { - BaseQuery.checkInterrupted(); - - IncrementalIndexRow entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getRowIndex())) { - continue; - } - - currEntry.set(entry); - - if (filterMatcher.matches(false)) { - return; - } - } - - done = true; - } - - @Override - public void advanceUninterruptibly() - { - if (!baseIter.hasNext()) { - done = true; - return; - } - - while (baseIter.hasNext()) { - if (Thread.currentThread().isInterrupted()) { - return; - } - - IncrementalIndexRow entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getRowIndex())) { - continue; - } - - currEntry.set(entry); - - if (filterMatcher.matches(false)) { - return; - } - } - - done = true; - } - - @Override - public boolean isDone() - { - return done; - } - - @Override - public boolean isDoneOrInterrupted() - { - return isDone() || Thread.currentThread().isInterrupted(); - } - - @Override - public void reset() - { - baseIter = cursorIterable.iterator(); - - if (numAdvanced == -1) { - numAdvanced = 0; - } else { - Iterators.advance(baseIter, numAdvanced); - } - - BaseQuery.checkInterrupted(); - - boolean foundMatched = false; - while (baseIter.hasNext()) { - IncrementalIndexRow entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getRowIndex())) { - numAdvanced++; - continue; - } - currEntry.set(entry); - if (filterMatcher.matches(false)) { - foundMatched = true; - break; - } - - numAdvanced++; - } - - done = !foundMatched && (emptyRange || !baseIter.hasNext()); - } - - private boolean beyondMaxRowIndex(int rowIndex) - { - // ignore rows whose rowIndex is beyond the maxRowIndex - // rows are order by timestamp, not rowIndex, - // so we still need to go through all rows to skip rows added after cursor created - return rowIndex > maxRowIndex; - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index c71264481121..f752d7062c1c 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.join; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -28,6 +27,8 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; @@ -219,6 +220,105 @@ public boolean hasBuiltInFilters() ); } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + final CursorBuildSpec.CursorBuildSpecBuilder cursorBuildSpecBuilder = + CursorBuildSpec.builder() + .setInterval(spec.getInterval()) + .setGranularity(spec.getGranularity()) + .isDescending(spec.isDescending()) + .setColumns(baseAdapter.getRowSignature().getColumnNames()) + .setQueryMetrics(spec.getQueryMetrics()); + + final Filter combinedFilter = baseFilterAnd(spec.getFilter()); + + if (clauses.isEmpty()) { + // HashJoinEngine isn't vectorized yet. + // However, we can still vectorize if there are no clauses, since that means all we need to do is apply + // a base filter. That's easy enough! + final CursorBuildSpec newSpec = cursorBuildSpecBuilder.setFilter(combinedFilter) + .setVirtualColumns(spec.getVirtualColumns()) + .build(); + return baseAdapter.asCursorMaker(newSpec); + } + return new CursorMaker() + { + + @Override + public Sequence makeCursors() + { + // Filter pre-analysis key implied by the call to "makeCursors". We need to sanity-check that it matches + // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursors" call (it + // requires access to the query context) so we'll need to skip sanity-checking it, by re-using the one present + // in the cached key.) + final JoinFilterPreAnalysisKey keyIn = + new JoinFilterPreAnalysisKey( + joinFilterPreAnalysis.getKey().getRewriteConfig(), + clauses, + spec.getVirtualColumns(), + combinedFilter + ); + + final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey(); + final JoinFilterSplit joinFilterSplit; + + if (keyIn.equals(keyCached)) { + // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursors call (keyIn). + joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis, baseFilter); + } else { + // Less common case: key differs. Re-analyze the filter. This case can happen when an unnest datasource is + // layered on top of a join datasource. + joinFilterSplit = JoinFilterAnalyzer.splitFilter( + JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn), + baseFilter + ); + } + + final List preJoinVirtualColumns = new ArrayList<>(); + final List postJoinVirtualColumns = new ArrayList<>(); + + determineBaseColumnsWithPreAndPostJoinVirtualColumns( + spec.getVirtualColumns(), + preJoinVirtualColumns, + postJoinVirtualColumns + ); + + // We merge the filter on base table specified by the user and filter on the base table that is pushed from + // the join + preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns()); + + if (joinFilterSplit.getBaseTableFilter().isPresent()) { + cursorBuildSpecBuilder.setFilter(joinFilterSplit.getBaseTableFilter().get()); + } + cursorBuildSpecBuilder.setVirtualColumns(VirtualColumns.create(preJoinVirtualColumns)); + + final Sequence baseCursorSequence = baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build()) + .makeCursors(); + + Closer joinablesCloser = Closer.create(); + + return Sequences.map( + baseCursorSequence, + cursor -> { + assert cursor != null; + Cursor retVal = cursor; + + for (JoinableClause clause : clauses) { + retVal = HashJoinEngine.makeJoinCursor(retVal, clause, spec.isDescending(), joinablesCloser); + } + + return PostJoinCursor.wrap( + retVal, + VirtualColumns.create(postJoinVirtualColumns), + joinFilterSplit.getJoinTableFilter().orElse(null) + ); + } + ).withBaggage(joinablesCloser); + } + }; + } + @Override public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) { @@ -239,21 +339,14 @@ public VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - if (!canVectorize(filter, virtualColumns, descending)) { - throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); - } - - // Should have been checked by canVectorize. - assert clauses.isEmpty(); - - return baseAdapter.makeVectorCursor( - baseFilterAnd(filter), - interval, - virtualColumns, - descending, - vectorSize, - queryMetrics - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(interval) + .setVirtualColumns(virtualColumns) + .isDescending(descending) + .setQueryMetrics(queryMetrics) + .build(); + return asCursorMaker(buildSpec).makeVectorCursor(); } @Override @@ -266,86 +359,16 @@ public Sequence makeCursors( @Nullable final QueryMetrics queryMetrics ) { - final Filter combinedFilter = baseFilterAnd(filter); - - if (clauses.isEmpty()) { - return baseAdapter.makeCursors( - combinedFilter, - interval, - virtualColumns, - gran, - descending, - queryMetrics - ); - } - - // Filter pre-analysis key implied by the call to "makeCursors". We need to sanity-check that it matches - // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursors" call (it - // requires access to the query context) so we'll need to skip sanity-checking it, by re-using the one present - // in the cached key.) - final JoinFilterPreAnalysisKey keyIn = - new JoinFilterPreAnalysisKey( - joinFilterPreAnalysis.getKey().getRewriteConfig(), - clauses, - virtualColumns, - combinedFilter - ); - - final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey(); - final JoinFilterSplit joinFilterSplit; - - if (keyIn.equals(keyCached)) { - // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursors call (keyIn). - joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis, baseFilter); - } else { - // Less common case: key differs. Re-analyze the filter. This case can happen when an unnest datasource is - // layered on top of a join datasource. - joinFilterSplit = JoinFilterAnalyzer.splitFilter( - JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn), - baseFilter - ); - } - - final List preJoinVirtualColumns = new ArrayList<>(); - final List postJoinVirtualColumns = new ArrayList<>(); - - determineBaseColumnsWithPreAndPostJoinVirtualColumns( - virtualColumns, - preJoinVirtualColumns, - postJoinVirtualColumns - ); - // We merge the filter on base table specified by the user and filter on the base table that is pushed from - // the join - preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns()); - - final Sequence baseCursorSequence = baseAdapter.makeCursors( - joinFilterSplit.getBaseTableFilter().isPresent() ? joinFilterSplit.getBaseTableFilter().get() : null, - interval, - VirtualColumns.create(preJoinVirtualColumns), - gran, - descending, - queryMetrics - ); - - Closer joinablesCloser = Closer.create(); - return Sequences.map( - baseCursorSequence, - cursor -> { - assert cursor != null; - Cursor retVal = cursor; - - for (JoinableClause clause : clauses) { - retVal = HashJoinEngine.makeJoinCursor(retVal, clause, descending, joinablesCloser); - } - - return PostJoinCursor.wrap( - retVal, - VirtualColumns.create(postJoinVirtualColumns), - joinFilterSplit.getJoinTableFilter().orElse(null) - ); - } - ).withBaggage(joinablesCloser); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(interval) + .setGranularity(gran) + .setVirtualColumns(virtualColumns) + .isDescending(descending) + .setQueryMetrics(queryMetrics) + .build(); + return asCursorMaker(buildSpec).makeCursors(); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 26c119bd34f3..38413b57a4fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -24,14 +24,15 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; import org.joda.time.DateTime; import javax.annotation.Nullable; /** - * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#makeCursors} to add post-join virtual columns - * and filters. + * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#asCursorMaker(CursorBuildSpec)} to add post-join + * virtual columns and filters. */ public class PostJoinCursor implements Cursor { diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index de306209f908..72373af40fdc 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -31,6 +31,7 @@ import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; @@ -113,15 +114,15 @@ public BroadcastSegmentIndexedTable( indexBuilders.add(m); } - // sort of like the dump segment tool, but build key column indexes when reading the segment - final Sequence cursors = adapter.makeCursors( - null, - queryableIndex.getDataInterval().withChronology(ISOChronology.getInstanceUTC()), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval( + queryableIndex.getDataInterval() + .withChronology(ISOChronology.getInstanceUTC()) + ) + .setGranularity(Granularities.ALL) + .setColumns(keyColumnNames) + .build(); + final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); final Sequence sequence = Sequences.map( cursors, diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index 3e2b46d5f681..7f63fbbe0f83 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.logger.Logger; @@ -39,10 +38,10 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SimpleAscendingOffset; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -123,14 +122,7 @@ public FrameBasedIndexedTable( RowSignature rowSignature = frameSignaturePair.getRowSignature(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); - return frameStorageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); }) .collect(Collectors.toList()) ); diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index fde58855a53e..86b3e305f0d9 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java @@ -19,12 +19,15 @@ package org.apache.druid.segment.vector; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.QueryableIndexCursorMaker; + import java.io.Closeable; /** * Vectorized cursor used during query execution. VectorCursors are returned by - * {@link org.apache.druid.segment.StorageAdapter#makeVectorCursor} and are created by - * {@link org.apache.druid.segment.QueryableIndexCursorSequenceBuilder#buildVectorized}. + * {@link org.apache.druid.segment.StorageAdapter#asCursorMaker(CursorBuildSpec)} and are created by + * {@link QueryableIndexCursorMaker#makeVectorCursor()}. * * Unlike the non-vectorized version, VectorCursor does not have a getTime() method. This is because we are trying to * avoid creating needlessly-small vectors when the time granularity is very fine. See diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index a8190e29f3af..da9a7eeeb3c5 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -20,21 +20,18 @@ package org.apache.druid.frame; import com.google.common.collect.Iterables; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; @@ -42,7 +39,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; @@ -57,28 +53,12 @@ public TestArrayStorageAdapter(QueryableIndex index) } @Override - public boolean canVectorize( - @Nullable Filter filter, - VirtualColumns virtualColumns, - boolean descending - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return false; + final CursorMaker delegate = super.asCursorMaker(spec); + return () -> delegate.makeCursors().map(DecoratedCursor::new); } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable final QueryMetrics queryMetrics - ) - { - return super.makeCursors(filter, interval, virtualColumns, gran, descending, queryMetrics) - .map(DecoratedCursor::new); - } @Override public RowSignature getRowSignature() diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java index 16d155f3df36..e84d46852605 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java @@ -31,10 +31,9 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -129,28 +128,22 @@ protected void verifySingleFrameReadableChannel( readableFrameChannel.close(); // build list of rows from written and read data to verify - List> writtenData = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ).flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())).toList(); + List> writtenData = + adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors() + .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())) + .toList(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( readbackFrame, FrameReader.create(adapter.getRowSignature()), Intervals.ETERNITY ); - List> readData = frameStorageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ).flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())).toList(); + List> readData = + adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors() + .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())) + .toList(); Assert.assertEquals("Read rows count is different from written rows count", writtenData.size(), readData.size()); Assert.assertEquals("Read data is different from written data", writtenData, readData); diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index 481851a0f75a..854fb13b60c8 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -20,6 +20,7 @@ package org.apache.druid.frame.segment; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.Intervals; @@ -28,12 +29,16 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.extraction.TimeFormatExtractionFn; import org.apache.druid.query.extraction.UpperExtractionFn; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; @@ -244,6 +249,11 @@ public static class CursorTests extends InitializedNullHandlingTest private StorageAdapter queryableAdapter; private FrameSegment frameSegment; private StorageAdapter frameAdapter; + private final QueryContext queryContext = QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, VECTOR_SIZE) + ); + + private CursorBuildSpec buildSpec; public CursorTests( FrameType frameType, @@ -258,6 +268,15 @@ public CursorTests( this.interval = interval; this.virtualColumns = virtualColumns; this.descending = descending; + this.buildSpec = CursorBuildSpec.builder() + .setFilter(this.filter) + .setInterval(this.interval) + // Frames only support Granularities.ALL: no point testing the others. + .setGranularity(Granularities.ALL) + .setVirtualColumns(this.virtualColumns) + .isDescending(this.descending) + .setQueryContext(queryContext) + .build(); } @Parameterized.Parameters(name = "frameType = {0}, " @@ -347,17 +366,7 @@ public void tearDown() @Test public void test_makeCursors() { - assertCursorsMatch( - adapter -> - adapter.makeCursors( - filter, - interval, - virtualColumns, - Granularities.ALL, // Frames only support Granularities.ALL: no point testing the others. - descending, - null - ) - ); + assertCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec).makeCursors()); } @Test @@ -365,17 +374,7 @@ public void test_makeVectorCursor() { Assume.assumeTrue(frameAdapter.canVectorize(filter, virtualColumns, descending)); - assertVectorCursorsMatch( - adapter -> - adapter.makeVectorCursor( - filter, - interval, - virtualColumns, - descending, - VECTOR_SIZE, - null - ) - ); + assertVectorCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec)); } private void assertCursorsMatch(final Function> call) @@ -389,13 +388,15 @@ private void assertCursorsMatch(final Function> FrameTestUtil.assertRowsEqual(queryableRows, frameRows); } - private void assertVectorCursorsMatch(final Function call) + private void assertVectorCursorsMatch(final Function call) { + final CursorMaker maker = call.apply(queryableAdapter); final RowSignature signature = frameAdapter.getRowSignature(); final Sequence> queryableRows = - FrameTestUtil.readRowsFromVectorCursor(call.apply(queryableAdapter), signature); + FrameTestUtil.readRowsFromVectorCursor(maker.makeVectorCursor(), signature); + final CursorMaker frameMaker = call.apply(frameAdapter); final Sequence> frameRows = - FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(call.apply(frameAdapter)), signature); + FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameMaker.makeVectorCursor()), signature); FrameTestUtil.assertRowsEqual(queryableRows, frameRows); } 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 7402e6ef5920..8e15babadc72 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 @@ -41,6 +41,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -158,7 +159,12 @@ public static void assertRowEqual(final String message, final List expec final Object expectedValue = expected.get(i); final Object actualValue = actual.get(i); - if (!Objects.deepEquals(expectedValue, actualValue)) { + if (expectedValue instanceof List && actualValue instanceof Object[]) { + if (!Objects.deepEquals(expectedValue, Arrays.asList((Object[]) actualValue))) { + ok = false; + break; + } + } else if (!Objects.deepEquals(expectedValue, actualValue)) { ok = false; break; } @@ -218,7 +224,7 @@ public static FrameSegment adapterToFrameSegment( /** * Reads a sequence of rows from a frame channel using a non-vectorized cursor from - * {@link FrameStorageAdapter#makeCursors}. + * {@link FrameStorageAdapter#asCursorMaker(CursorBuildSpec)}. * * @param channel the channel * @param frameReader reader for this channel @@ -232,7 +238,8 @@ public static Sequence> readRowsFromFrameChannel( .flatMap( frame -> new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null) + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors() .flatMap(cursor -> readRowsFromCursor(cursor, frameReader.signature())) ); } @@ -283,7 +290,12 @@ public static Sequence makeCursorsForAdapter( virtualColumns = VirtualColumns.EMPTY; } - return adapter.makeCursors(null, Intervals.ETERNITY, virtualColumns, Granularities.ALL, false, null) + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setVirtualColumns(virtualColumns) + .build(); + return adapter.asCursorMaker(buildSpec) + .makeCursors() .map(cursor -> { if (populateRowNumber) { return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn); diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index 770d79beb76f..e4aa0d50f765 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -42,18 +42,17 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.RowIdSupplier; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; @@ -577,7 +576,8 @@ private static Pair writeFrame( } return inputSegment.asStorageAdapter() - .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null) + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursors() .accumulate( null, (retVal, cursor) -> { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index 77f86bdc4c50..ea4af2ffeced 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -34,9 +34,10 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -654,18 +655,17 @@ private void runResults( List segments = segmentsGenerator.apply(tempFolder, closer); Supplier> runner = () -> helper.runQueryOnSegmentsObjs(segments, groupQuery).toList(); - Filter filter = groupQuery.getFilter() == null ? null : groupQuery.getFilter().toFilter(); + final CursorBuildSpec spec = groupQuery.asCursorBuildSpec(null); boolean allCanVectorize = segments.stream() .allMatch( - s -> s.asStorageAdapter() - .canVectorize( - filter, - groupQuery.getVirtualColumns(), - groupQuery.isDescending() - ) - ); - - Assert.assertEquals(NestedDataTestUtils.expectSegmentGeneratorCanVectorize(segmentsName), allCanVectorize); + s -> { + final CursorMaker maker = s.asStorageAdapter() + .asCursorMaker(spec); + final boolean canVectorize = maker.canVectorize(); + maker.cleanup(); + return canVectorize; + }); + if (!allCanVectorize) { if (vectorize == QueryContexts.Vectorize.FORCE) { Throwable t = Assert.assertThrows(RuntimeException.class, runner::get); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java index 8bc83277c65b..91a455ff83f5 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java @@ -30,10 +30,10 @@ import org.apache.druid.query.groupby.epinephelinae.VectorGrouper; import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine.VectorGroupByEngineIterator; import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -61,15 +61,9 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException .setAggregatorSpecs(factory) .build(); final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); + final CursorMaker maker = storageAdapter.asCursorMaker(query.asCursorBuildSpec(null)); final ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[4096]); - final VectorCursor cursor = storageAdapter.makeVectorCursor( - Filters.toFilter(query.getDimFilter()), - interval, - query.getVirtualColumns(), - false, - query.context().getVectorSize(), - null - ); + final VectorCursor cursor = maker.makeVectorCursor(); final List dimensions = query.getDimensions().stream().map( dimensionSpec -> ColumnProcessors.makeVectorProcessor( diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java index 2d36ffa1e632..aa369dc63777 100644 --- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java @@ -29,9 +29,9 @@ import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.RowBasedStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; import org.apache.druid.timeline.SegmentId; @@ -192,14 +192,12 @@ public void test_asStorageAdapter_getDimensionCardinalityV() @Test public void test_asStorageAdapter_makeCursors() { - final Sequence cursors = LOOKUP_SEGMENT.asStorageAdapter().makeCursors( - null, - Intervals.of("1970/PT1H"), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = LOOKUP_SEGMENT.asStorageAdapter().asCursorMaker( + CursorBuildSpec.builder() + .setInterval(Intervals.of("1970/PT1H")) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final List> kvs = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java index 919f9fcff82d..367c29653473 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java @@ -37,7 +37,7 @@ import org.apache.druid.segment.ArrayListSegment; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.TypeStrategy; @@ -250,16 +250,16 @@ private void validateDecorated( }, siggy ); + final CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() + .setFilter(filter) + .setGranularity(Granularities.ALL); + if (interval != null) { + builder.setInterval(interval); + } final Sequence cursors = seggy .asStorageAdapter() - .makeCursors( - filter, - interval == null ? Intervals.ETERNITY : interval, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + .asCursorMaker(builder.build()) + .makeCursors(); vals = cursors.accumulate( new ArrayList<>(), diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index fee5b4b9de72..6a524e5d0c91 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.junit.Assert; @@ -65,7 +66,7 @@ * Ensures that we have run-to-run stability of result order, which is important for offset-based pagination. */ @RunWith(Parameterized.class) -public class ScanQueryResultOrderingTest +public class ScanQueryResultOrderingTest extends InitializedNullHandlingTest { private static final String DATASOURCE = "datasource"; private static final String ID_COLUMN = "id"; diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index 815a6ed9951f..b1b16103a77b 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -37,10 +37,11 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.EqualityFilter; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; @@ -661,18 +662,17 @@ private void runResults( List segments = segmentsGenerator.apply(tempFolder, closer); Supplier>> runner = () -> helper.runQueryOnSegmentsObjs(segments, timeseriesQuery).toList(); - Filter filter = timeseriesQuery.getFilter() == null ? null : timeseriesQuery.getFilter().toFilter(); + final CursorBuildSpec spec = timeseriesQuery.asCursorBuildSpec(null); boolean allCanVectorize = segments.stream() .allMatch( - s -> s.asStorageAdapter() - .canVectorize( - filter, - timeseriesQuery.getVirtualColumns(), - timeseriesQuery.isDescending() - ) - ); + s -> { + final CursorMaker maker = s.asStorageAdapter() + .asCursorMaker(spec); + final boolean canVectorize = maker.canVectorize(); + maker.cleanup(); + return canVectorize; + }); - Assert.assertEquals(NestedDataTestUtils.expectSegmentGeneratorCanVectorize(segmentsName), allCanVectorize); if (!allCanVectorize) { if (vectorize == QueryContexts.Vectorize.FORCE) { Throwable t = Assert.assertThrows(RuntimeException.class, runner::get); diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index 3b0e4ec74feb..745757768597 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -149,14 +148,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -216,14 +208,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -313,14 +298,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -409,14 +387,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -475,14 +446,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -534,14 +498,7 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -619,14 +576,7 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index ec691867a317..70514703030c 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -148,14 +147,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -213,14 +206,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -308,14 +294,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -403,14 +382,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -468,14 +440,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); @@ -525,14 +490,7 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.NONE, - false, - null - ); + Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); List cursorList = cursorSequence.toList(); ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilderTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorMakerTest.java similarity index 72% rename from processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilderTest.java rename to processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorMakerTest.java index ea93e1b75c2e..2a819636a418 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorMakerTest.java @@ -25,7 +25,7 @@ import org.junit.Assert; import org.junit.Test; -public class QueryableIndexCursorSequenceBuilderTest +public class QueryableIndexCursorMakerTest { @Test public void testTimeSearch() @@ -66,52 +66,52 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) Assert.assertEquals( 0, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 0, 0, values.length) + QueryableIndexCursorMaker.timeSearch(column, 0, 0, values.length) ); Assert.assertEquals( 2, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 0, 2, values.length) + QueryableIndexCursorMaker.timeSearch(column, 0, 2, values.length) ); Assert.assertEquals( 0, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 0, 0, values.length / 2) + QueryableIndexCursorMaker.timeSearch(column, 0, 0, values.length / 2) ); Assert.assertEquals( 1, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 0, values.length) + QueryableIndexCursorMaker.timeSearch(column, 1, 0, values.length) ); Assert.assertEquals( 2, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 2, values.length) + QueryableIndexCursorMaker.timeSearch(column, 1, 2, values.length) ); Assert.assertEquals( 1, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 0, values.length / 2) + QueryableIndexCursorMaker.timeSearch(column, 1, 0, values.length / 2) ); Assert.assertEquals( 1, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 1, 8) + QueryableIndexCursorMaker.timeSearch(column, 1, 1, 8) ); Assert.assertEquals( 8, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 2, 0, values.length) + QueryableIndexCursorMaker.timeSearch(column, 2, 0, values.length) ); Assert.assertEquals( 10, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 10, 0, values.length) + QueryableIndexCursorMaker.timeSearch(column, 10, 0, values.length) ); Assert.assertEquals( 11, - QueryableIndexCursorSequenceBuilder.timeSearch(column, 15, 0, values.length) + QueryableIndexCursorMaker.timeSearch(column, 15, 0, values.length) ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java index ddb1cc419f33..1f5c18fdd056 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java @@ -20,9 +20,7 @@ package org.apache.druid.segment; import org.apache.druid.hll.HyperLogLogCollector; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; @@ -74,17 +72,9 @@ public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - if (vectorize) { final VectorCursor cursor = closer.register( - adapter.makeVectorCursor( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - false, - QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE, - null - ) + adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeVectorCursor() ); final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -96,15 +86,7 @@ public void setUp() partialNullSelector = columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column")); } else { - final Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - + final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); final Yielder yielder = closer.register(Yielders.each(cursors)); final Cursor cursor = yielder.get(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -250,14 +232,7 @@ public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); final Yielder cursorYielder = Yielders.each(cursors); cursor = cursorYielder.get(); columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 9822f6e5f332..2382b6d0e071 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -42,6 +42,7 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -173,7 +174,7 @@ public static void setUpClass() /** * A RowAdapter for Integers where: - * + *

* 1) timestampFunction returns a timestamp where the millis instant is equal to that integer as a number of hours * since the epoch (1970). * 2) columnFunction provides columns named after value types where each one equal to the cast to that type. All @@ -455,14 +456,12 @@ public void test_makeCursors_filterOnLong() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.makeCursors( - new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -479,14 +478,12 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.makeCursors( - new SelectorDimFilter("nonexistent", null, null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -504,23 +501,24 @@ public void test_makeCursors_filterOnVirtualColumn() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.makeCursors( - new SelectorDimFilter("vc", "2", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "vc", - "\"LONG\" + 1", - ColumnType.LONG, - ExprMacroTable.nil() - ) - ) - ), - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) + .setVirtualColumns( + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "vc", + "\"LONG\" + 1", + ColumnType.LONG, + ExprMacroTable.nil() + ) + ) + ) + ) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -537,14 +535,9 @@ public void test_makeCursors_descending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - true, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -563,14 +556,12 @@ public void test_makeCursors_intervalDoesNotMatch() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.of("2000/P1D"), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(Intervals.of("2000/P1D")) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of(), @@ -585,14 +576,12 @@ public void test_makeCursors_intervalPartiallyMatches() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.of("1970-01-01T01/PT1H"), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT1H")) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -609,14 +598,13 @@ public void test_makeCursors_hourGranularity() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.of("1970/1971"), - VirtualColumns.EMPTY, - Granularities.HOUR, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(Intervals.of("1970/1971")) + .setGranularity(Granularities.HOUR) + .build() + ).makeCursors(); + Assert.assertEquals( ImmutableList.of( @@ -637,14 +625,12 @@ public void test_makeCursors_hourGranularityWithInterval() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.of("1970-01-01T01/PT2H"), - VirtualColumns.EMPTY, - Granularities.HOUR, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setGranularity(Granularities.HOUR) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -663,14 +649,13 @@ public void test_makeCursors_hourGranularityWithIntervalDescending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.of("1970-01-01T01/PT2H"), - VirtualColumns.EMPTY, - Granularities.HOUR, - true, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setGranularity(Granularities.HOUR) + .isDescending(true) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -689,14 +674,7 @@ public void test_makeCursors_allProcessors() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); Assert.assertEquals( ImmutableList.of( @@ -802,14 +780,12 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNonnull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.makeCursors( - new SelectorDimFilter("nonexistent", "abc", null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursors = adapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); Assert.assertEquals( ImmutableList.of(), @@ -824,14 +800,11 @@ public void test_makeCursors_eternityIntervalWithMonthGranularity() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); Assert.assertThrows(IAE.class, () -> { - adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.MONTH, - false, - null - ); + adapter.asCursorMaker( + CursorBuildSpec.builder() + .setGranularity(Granularities.MONTH) + .build() + ).makeCursors(); }); } diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index 6df1766bf984..2025c0117705 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -25,12 +25,10 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.NestedDataTestUtils; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.Filter; @@ -57,7 +55,6 @@ import org.apache.druid.utils.CloseableUtils; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; -import org.joda.time.Interval; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -65,7 +62,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; import java.util.List; @@ -226,15 +222,12 @@ public void test_group_of_unnest_adapters_column_capabilities() @Test public void test_unnest_adapters_basic() { - - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER.makeCursors( - null, - UNNEST_STORAGE_ADAPTER.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + Sequence cursorSequence = UNNEST_STORAGE_ADAPTER.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(UNNEST_STORAGE_ADAPTER.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -262,15 +255,12 @@ public void test_unnest_adapters_basic() @Test public void test_two_levels_of_unnest_adapters() { - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER1.makeCursors( - null, - UNNEST_STORAGE_ADAPTER1.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - + Sequence cursorSequence = UNNEST_STORAGE_ADAPTER1.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -324,14 +314,13 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest selector(inputColumn, "2") )); - final Sequence cursorSequence = unnestStorageAdapter.makeCursors( - baseFilter, - unnestStorageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(baseFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); final Filter pushDownFilter = base.getPushDownFilter(); @@ -375,14 +364,13 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap )) )); - final Sequence cursorSequence = unnestStorageAdapter.makeCursors( - baseFilter, - unnestStorageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(baseFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); final Filter pushDownFilter = base.getPushDownFilter(); @@ -396,6 +384,7 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap return null; }); } + @Test public void test_nested_filters_unnested_and_topLevel1And3filtersInOR() { @@ -413,6 +402,7 @@ public void test_nested_filters_unnested_and_topLevel1And3filtersInOR() "(unnested-multi-string1 = 3 && (newcol = 2 || multi-string1 = 2 || unnested-multi-string1 = 1))" ); } + @Test public void test_nested_multiLevel_filters_unnested() { @@ -436,6 +426,7 @@ public void test_nested_multiLevel_filters_unnested() "(unnested-multi-string1 = 3 && (newcol = 2 || multi-string1 = 2 || (newcol = 3 && multi-string1 = 7) || unnested-multi-string1 = 1))" ); } + @Test public void test_nested_multiLevel_filters_unnested5Level() { @@ -462,6 +453,7 @@ public void test_nested_multiLevel_filters_unnested5Level() "(unnested-multi-string1 = 3 || newcol = 2 || multi-string1 = 2 || (newcol = 3 && multi-string1 = 7 && newcol_1 = 10) || unnested-multi-string1 = 1)" ); } + @Test public void test_nested_filters_unnested_and_topLevelORAnd3filtersInOR() { @@ -479,6 +471,7 @@ public void test_nested_filters_unnested_and_topLevelORAnd3filtersInOR() "(unnested-multi-string1 = 3 || (newcol = 2 && multi-string1 = 2 && unnested-multi-string1 = 1))" ); } + @Test public void test_nested_filters_unnested_and_topLevelAND3filtersInORWithNestedOrs() { @@ -644,7 +637,7 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( new TestStorageAdapter(INCREMENTAL_INDEX), new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), - new SelectorDimFilter(OUTPUT_COLUMN_NAME, "1", null) + new SelectorDimFilter(OUTPUT_COLUMN_NAME, "1", null) ); final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); @@ -655,14 +648,12 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() selector(inputColumn, "1"); - final Sequence cursorSequence = unnestStorageAdapter.makeCursors( - null, - unnestStorageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); final Filter pushDownFilter = base.getPushDownFilter(); @@ -701,14 +692,13 @@ public void test_pushdown_filters_unnested_dimension_outside() selector(inputColumn, "1"); final Filter queryFilter = new SelectorFilter(OUTPUT_COLUMN_NAME, "1", null); - final Sequence cursorSequence = unnestStorageAdapter.makeCursors( - queryFilter, - unnestStorageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(queryFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); final Filter pushDownFilter = base.getPushDownFilter(); @@ -753,14 +743,12 @@ public void testUnnestValueMatcherValueDoesntExist() new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), null ); - Sequence cursorSequence = withNullsStorageAdapter.makeCursors( - null, - withNullsStorageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + Sequence cursorSequence = withNullsStorageAdapter.asCursorMaker( + CursorBuildSpec.builder() + .setInterval(withNullsStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(); cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -858,16 +846,9 @@ public Filter getPushDownFilter() } @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - this.pushDownFilter = filter; - return super.makeCursors(filter, interval, virtualColumns, gran, descending, queryMetrics); + this.pushDownFilter = spec.getFilter(); + return super.asCursorMaker(spec); } } 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 d9eb693ab76d..082c6cfcf451 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 @@ -44,7 +44,6 @@ import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -54,6 +53,8 @@ import org.apache.druid.math.expr.ExprType; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Parser; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; @@ -70,6 +71,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; @@ -416,11 +418,13 @@ public static InputRow makeSchemaRow( protected StorageAdapter adapter; + protected VirtualColumns virtualColumns; + // JUnit creates a new test instance for every test method call. // For filter tests, the test setup creates a segment. // Creating a new segment for every test method call is pretty slow, so cache the StorageAdapters. // Each thread gets its own map. - private static ThreadLocal>>> adapterCache = + private static ThreadLocal>> adapterCache = ThreadLocal.withInitial(HashMap::new); public BaseFilterTest( @@ -447,32 +451,40 @@ public void setUp() throws Exception { NestedDataModule.registerHandlersAndSerde(); String className = getClass().getName(); - Map> adaptersForClass = adapterCache.get().get(className); + Map adaptersForClass = adapterCache.get().get(className); if (adaptersForClass == null) { adaptersForClass = new HashMap<>(); adapterCache.get().put(className, adaptersForClass); } - Pair pair = adaptersForClass.get(testName); - if (pair == null) { - pair = finisher.apply( + AdapterStuff adapterStuff = adaptersForClass.get(testName); + if (adapterStuff == null) { + Pair pair = finisher.apply( indexBuilder.tmpDir(temporaryFolder.newFolder()).rows(rows) ); - adaptersForClass.put(testName, pair); + adapterStuff = new AdapterStuff( + pair.lhs, + VirtualColumns.create( + Arrays.stream(VIRTUAL_COLUMNS.getVirtualColumns()) + .filter(x -> x.canVectorize(VIRTUAL_COLUMNS.wrapInspector(pair.lhs))) + .collect(Collectors.toList()) + ), + pair.rhs + ); + adaptersForClass.put(testName, adapterStuff); } - this.adapter = pair.lhs; - + this.adapter = adapterStuff.adapter; + this.virtualColumns = adapterStuff.virtualColumns; } public static void tearDown(String className) throws Exception { - Map> adaptersForClass = adapterCache.get().get(className); + Map adaptersForClass = adapterCache.get().get(className); if (adaptersForClass != null) { - for (Map.Entry> entry : adaptersForClass.entrySet()) { - Closeable closeable = entry.getValue().rhs; - closeable.close(); + for (Map.Entry entry : adaptersForClass.entrySet()) { + entry.getValue().closeable.close(); } adapterCache.get().put(className, null); } @@ -780,29 +792,30 @@ private DimFilter maybeOptimize(final DimFilter dimFilter) return optimize ? dimFilter.optimize(false) : dimFilter; } + private Sequence makeCursorSequence(final Filter filter) { - return adapter.makeCursors( - filter, - Intervals.ETERNITY, - VIRTUAL_COLUMNS, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(VIRTUAL_COLUMNS) + .setGranularity(Granularities.ALL) + .build(); + return adapter.asCursorMaker(buildSpec).makeCursors(); } private VectorCursor makeVectorCursor(final Filter filter) { - - return adapter.makeVectorCursor( - filter, - Intervals.ETERNITY, - VIRTUAL_COLUMNS, - false, - 3, // Vector size smaller than the number of rows, to ensure we use more than one. - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(virtualColumns) + .setGranularity(Granularities.ALL) + .setQueryContext( + QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) + ) + ) + .build(); + return adapter.asCursorMaker(buildSpec).makeVectorCursor(); } /** @@ -1240,4 +1253,22 @@ private void assertFilterMatches( ); } } + + private static class AdapterStuff + { + private final StorageAdapter adapter; + private final VirtualColumns virtualColumns; + private final Closeable closeable; + + private AdapterStuff( + StorageAdapter adapter, + VirtualColumns virtualColumns, + Closeable closeable + ) + { + this.adapter = adapter; + this.virtualColumns = virtualColumns; + this.closeable = closeable; + } + } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java index f61ed0f14c03..c9717f186284 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java @@ -22,10 +22,6 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; @@ -37,11 +33,9 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.filter.BoundDimFilter; -import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.filter.RegexDimFilter; @@ -55,7 +49,6 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -66,7 +59,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; @RunWith(Parameterized.class) public class FloatAndDoubleFilteringTest extends BaseFilterTest @@ -74,8 +66,6 @@ public class FloatAndDoubleFilteringTest extends BaseFilterTest private static final String FLOAT_COLUMN = "flt"; private static final String DOUBLE_COLUMN = "dbl"; private static final String TIMESTAMP_COLUMN = "ts"; - private static int EXECUTOR_NUM_THREADS = 16; - private static int EXECUTOR_NUM_TASKS = 2000; private static final int NUM_FILTER_VALUES = 32; private static final InputRowParser> PARSER = new MapInputRowParser( @@ -149,13 +139,6 @@ public void testFloatFilterWithExtractionFn() doTestFloatFilterWithExtractionFn(DOUBLE_COLUMN); } - @Test - public void testMultithreaded() - { - doTestMultithreaded(FLOAT_COLUMN); - doTestMultithreaded(DOUBLE_COLUMN); - } - private void doTestFloatColumnFiltering(final String columnName) { assertFilterMatches( @@ -352,90 +335,4 @@ private void doTestFloatFilterWithExtractionFn(final String columnName) ImmutableList.of("2", "3", "4") ); } - - private void doTestMultithreaded(final String columnName) - { - assertFilterMatchesMultithreaded( - new SelectorDimFilter(columnName, "3", null), - ImmutableList.of("3") - ); - - assertFilterMatchesMultithreaded( - new SelectorDimFilter(columnName, "3.0", null), - ImmutableList.of("3") - ); - - assertFilterMatchesMultithreaded( - new InDimFilter(columnName, Arrays.asList("2", "4", "8"), null), - ImmutableList.of("2", "4") - ); - - assertFilterMatchesMultithreaded( - new InDimFilter(columnName, Arrays.asList("2.0", "4.0", "8.0"), null), - ImmutableList.of("2", "4") - ); - - // cross the hashing threshold to test hashset implementation, filter on even values - List infilterValues = new ArrayList<>(NUM_FILTER_VALUES); - for (int i = 0; i < NUM_FILTER_VALUES; i++) { - infilterValues.add(String.valueOf(i * 2)); - } - assertFilterMatchesMultithreaded( - new InDimFilter(columnName, infilterValues, null), - ImmutableList.of("2", "4", "6") - ); - - assertFilterMatches( - new BoundDimFilter(columnName, "2", "5", false, false, null, null, StringComparators.NUMERIC), - ImmutableList.of("2", "3", "4", "5") - ); - - assertFilterMatches( - new BoundDimFilter(columnName, "2.0", "5.0", false, false, null, null, StringComparators.NUMERIC), - ImmutableList.of("2", "3", "4", "5") - ); - } - - private void assertFilterMatchesMultithreaded( - final DimFilter filter, - final List expectedRows - ) - { - testWithExecutor(filter, expectedRows); - } - - private Runnable makeFilterRunner( - final DimFilter filter, - final List expectedRows - ) - { - return () -> assertFilterMatches(filter, expectedRows); - } - - private void testWithExecutor( - final DimFilter filter, - final List expectedRows - ) - { - ListeningExecutorService executor = MoreExecutors.listeningDecorator( - Execs.multiThreaded(EXECUTOR_NUM_THREADS, "FloatAndDoubleFilteringTest-%d") - ); - - List> futures = new ArrayList<>(); - - for (int i = 0; i < EXECUTOR_NUM_TASKS; i++) { - Runnable runnable = makeFilterRunner(filter, expectedRows); - ListenableFuture fut = executor.submit(runnable); - futures.add(fut); - } - - try { - Futures.allAsList(futures).get(60, TimeUnit.SECONDS); - } - catch (Exception ex) { - Assert.fail(ex.getMessage()); - } - - executor.shutdown(); - } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java index d24c599eceb8..3fd1c55ebab8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java @@ -22,10 +22,6 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; @@ -34,12 +30,10 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.filter.BoundDimFilter; -import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.filter.RegexDimFilter; @@ -53,7 +47,6 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -64,15 +57,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; @RunWith(Parameterized.class) public class LongFilteringTest extends BaseFilterTest { private static final String LONG_COLUMN = "lng"; private static final String TIMESTAMP_COLUMN = "ts"; - private static int EXECUTOR_NUM_THREADS = 16; - private static int EXECUTOR_NUM_TASKS = 2000; private static final int NUM_FILTER_VALUES = 32; private static final InputRowParser> PARSER = new MapInputRowParser( @@ -375,81 +365,4 @@ public void testLongFilterWithExtractionFn() ImmutableList.of("2", "3", "4") ); } - - @Test - public void testMultithreaded() - { - assertFilterMatchesMultithreaded( - new SelectorDimFilter(LONG_COLUMN, "3", null), - ImmutableList.of("3") - ); - - assertFilterMatchesMultithreaded( - new InDimFilter(LONG_COLUMN, Arrays.asList("2", "4", "8"), null), - ImmutableList.of("2", "4") - ); - - // cross the hashing threshold to test hashset implementation, filter on even values - List infilterValues = new ArrayList<>(NUM_FILTER_VALUES); - for (int i = 0; i < NUM_FILTER_VALUES; i++) { - infilterValues.add(String.valueOf(i * 2)); - } - assertFilterMatchesMultithreaded( - new InDimFilter(LONG_COLUMN, infilterValues, null), - ImmutableList.of("2", "4", "6") - ); - - assertFilterMatches( - new BoundDimFilter(LONG_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC), - ImmutableList.of("2", "3", "4", "5") - ); - } - - private void assertFilterMatchesMultithreaded( - final DimFilter filter, - final List expectedRows - ) - { - testWithExecutor(filter, expectedRows); - } - - private Runnable makeFilterRunner( - final DimFilter filter, - final List expectedRows - ) - { - return new Runnable() - { - @Override - public void run() - { - assertFilterMatches(filter, expectedRows); - } - }; - } - - private void testWithExecutor( - final DimFilter filter, - final List expectedRows - ) - { - ListeningExecutorService executor = MoreExecutors.listeningDecorator(Execs.multiThreaded(EXECUTOR_NUM_THREADS, "LongFilteringTest-%d")); - - List> futures = new ArrayList<>(); - - for (int i = 0; i < EXECUTOR_NUM_TASKS; i++) { - Runnable runnable = makeFilterRunner(filter, expectedRows); - ListenableFuture fut = executor.submit(runnable); - futures.add(fut); - } - - try { - Futures.allAsList(futures).get(60, TimeUnit.SECONDS); - } - catch (Exception ex) { - Assert.fail(ex.getMessage()); - } - - executor.shutdown(); - } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 0d0f979f715b..c620917eb01e 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -58,9 +58,10 @@ import org.apache.druid.segment.CloserRule; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; @@ -147,18 +148,16 @@ public void testSanity() throws Exception .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .addOrderByColumn("billy") .build(); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); final Sequence rows = GroupByQueryEngine.process( query, - new IncrementalIndexStorageAdapter(index), + adapter, + maker, processingBuffer.get(), null, new GroupByQueryConfig(), - new DruidProcessingConfig(), - filter, - interval, - null + new DruidProcessingConfig() ); final List results = rows.toList(); @@ -223,18 +222,16 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception ) .addOrderByColumn("billy") .build(); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); final Sequence rows = GroupByQueryEngine.process( query, - new IncrementalIndexStorageAdapter(index), + adapter, + maker, processingBuffer.get(), null, new GroupByQueryConfig(), - new DruidProcessingConfig(), - filter, - interval, - null + new DruidProcessingConfig() ); final List results = rows.toList(); @@ -278,14 +275,12 @@ public void testResetSanity() throws IOException IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); for (boolean descending : Arrays.asList(false, true)) { - Sequence cursorSequence = adapter.makeCursors( - new SelectorFilter("sally", "bo"), - interval, - VirtualColumns.EMPTY, - Granularities.NONE, - descending, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorFilter("sally", "bo")) + .setInterval(interval) + .isDescending(descending) + .build(); + Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); Cursor cursor = cursorSequence.limit(1).toList().get(0); DimensionSelector dimSelector; @@ -391,19 +386,16 @@ public void testFilterByNull() throws Exception .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); - + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); final Sequence rows = GroupByQueryEngine.process( query, - new IncrementalIndexStorageAdapter(index), + adapter, + maker, processingBuffer.get(), null, new GroupByQueryConfig(), - new DruidProcessingConfig(), - filter, - interval, - null + new DruidProcessingConfig() ); final List results = rows.toList(); @@ -433,14 +425,11 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); - Sequence cursors = sa.makeCursors( - null, - Intervals.utc(timestamp - 60_000, timestamp + 60_000), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); cursors @@ -497,14 +486,12 @@ public void testCursorDictionaryRaceConditionFix() throws Exception final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); - Sequence cursors = sa.makeCursors( - new DictionaryRaceTestFilter(index, timestamp), - Intervals.utc(timestamp - 60_000, timestamp + 60_000), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new DictionaryRaceTestFilter(index, timestamp)) + .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); cursors @@ -548,14 +535,11 @@ public void testCursoringAndSnapshot() throws Exception final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); - Sequence cursors = sa.makeCursors( - null, - Intervals.utc(timestamp - 60_000, timestamp + 60_000), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); cursors diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java index e5ab7d1be8aa..286d03b2ada9 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java @@ -26,6 +26,7 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; @@ -222,7 +223,7 @@ protected JoinableClause regionToCountry(final JoinType joinType) /** * Creates a fact-to-country join segment without a {@link JoinFilterPreAnalysis}. This means it cannot - * have {@link org.apache.druid.segment.StorageAdapter#makeCursors} called on it. + * have {@link org.apache.druid.segment.StorageAdapter#asCursorMaker(CursorBuildSpec)} called on it. */ protected HashJoinSegmentStorageAdapter makeFactToCountrySegment() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index 20d032aba381..25c5cfaddb1f 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -33,6 +33,7 @@ import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -312,14 +313,7 @@ public void test_makeCursors_factToCountryLeft() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -376,14 +370,7 @@ public void test_makeCursors_factToCountryLeftUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -438,14 +425,7 @@ public void test_makeCursors_factToCountryInner() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -495,14 +475,7 @@ public void test_makeCursors_factToCountryInnerUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -554,14 +527,9 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -619,14 +587,9 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -680,14 +643,9 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -716,14 +674,9 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -751,14 +704,9 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -789,14 +737,9 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -826,14 +769,9 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -864,14 +802,9 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -906,14 +839,9 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -948,14 +876,9 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -990,14 +913,9 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1032,14 +950,9 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1087,14 +1000,9 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1163,14 +1071,9 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1222,14 +1125,7 @@ public void test_makeCursors_factToRegionToCountryLeft() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1286,14 +1182,7 @@ public void test_makeCursors_factToRegionToCountryInnerIncludeNull() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1359,14 +1248,9 @@ public void test_makeCursors_factToCountryAlwaysTrue() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1423,14 +1307,9 @@ public void test_makeCursors_factToCountryAlwaysFalse() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1469,14 +1348,9 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1533,14 +1407,9 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1583,14 +1452,9 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1639,14 +1503,9 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1692,14 +1551,7 @@ public void test_makeCursors_factToCountryUsingExpression() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1744,14 +1596,7 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1798,14 +1643,9 @@ public void test_makeCursors_factToRegionTheWrongWay() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "regionIsoCode", @@ -1854,14 +1694,7 @@ public void test_makeCursors_errorOnNonEquiJoin() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of() ); } @@ -1896,14 +1729,7 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of() ); } @@ -1938,14 +1764,7 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of() ); } @@ -1980,14 +1799,7 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of() ); } @@ -2008,14 +1820,9 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -2042,14 +1849,9 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -2075,14 +1877,9 @@ public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowIS factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + ).asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors().toList(); } @Test @@ -2106,14 +1903,7 @@ public void test_makeCursors_factToCountryLeftWithBaseFilter() baseFilter, joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -2150,14 +1940,7 @@ public void test_makeCursors_factToCountryInnerWithBaseFilter() baseFilter, joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -2193,14 +1976,7 @@ public void test_makeCursors_factToCountryRightWithBaseFilter() baseFilter, joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -2253,14 +2029,7 @@ public void test_makeCursors_factToCountryFullWithBaseFilter() baseFilter, joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), ImmutableList.of( "page", "countryIsoCode", 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 1b7f250f8479..79d43ac4d433 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 @@ -24,7 +24,6 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; @@ -34,6 +33,7 @@ import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -83,14 +83,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -160,14 +155,9 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -216,14 +206,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -283,14 +268,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -354,14 +334,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -424,14 +399,13 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(originalFilter) + .setVirtualColumns(virtualColumns) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -498,14 +472,13 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder() + .setFilter(originalFilter) + .setVirtualColumns(virtualColumns) + .setGranularity(Granularities.ALL) + .build() + ).makeCursors(), ImmutableList.of( "page", "v0" @@ -602,14 +575,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -729,14 +697,9 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -823,14 +786,9 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan joinFilterPreAnalysis ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -873,14 +831,9 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -975,14 +928,9 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1065,14 +1013,9 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1139,14 +1082,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1197,14 +1135,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1254,14 +1187,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1310,14 +1238,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1367,14 +1290,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1424,14 +1342,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1488,14 +1401,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1549,14 +1457,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1604,14 +1507,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1658,14 +1556,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1716,14 +1609,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1773,14 +1661,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1829,14 +1712,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", "countryIsoCode", @@ -1896,14 +1774,9 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1967,14 +1840,9 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -2048,14 +1916,9 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -2109,14 +1972,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2189,14 +2047,9 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2392,14 +2245,9 @@ public boolean supportsRequiredColumnRewrite() } JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2526,14 +2374,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2606,14 +2449,9 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName ); JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + adapter.asCursorMaker( + CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() + ).makeCursors(), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index 6813c04bb10f..3abfb020575b 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -21,17 +21,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.QueryInterruptedException; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -40,10 +36,8 @@ import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; -import org.joda.time.Interval; import org.junit.Test; -import javax.annotation.Nullable; import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -76,17 +70,10 @@ public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return super.makeCursors(filter, interval, virtualColumns, gran, descending, queryMetrics) - .map(cursor -> new CursorNoAdvance(cursor, countDownLatch)); + final CursorMaker delegate = super.asCursorMaker(spec); + return () -> delegate.makeCursors().map(cursor -> new CursorNoAdvance(cursor, countDownLatch)); } private static class CursorNoAdvance implements Cursor @@ -235,14 +222,9 @@ public void makeCursorAndAdvance() joinFilterPreAnalysis ); - Cursor cursor = Iterables.getOnlyElement(hashJoinSegmentStorageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ).toList()); + Cursor cursor = Iterables.getOnlyElement( + hashJoinSegmentStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors().toList() + ); ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() { 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 86455f12c6e8..6b3d4698ab98 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 @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.error.DruidException; import org.apache.druid.guice.NestedDataModule; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; @@ -35,6 +34,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; @@ -347,14 +347,12 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir ); Assert.assertEquals(1, segments.size()); StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); - Sequence cursorSequence = storageAdapter.makeCursors( - null, - Intervals.ETERNITY, - virtualColumns, - Granularities.DAY, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(storageAdapter.getInterval()) + .setVirtualColumns(virtualColumns) + .setGranularity(Granularities.DAY) + .build(); + Sequence cursorSequence = storageAdapter.asCursorMaker(buildSpec).makeCursors(); final Yielder yielder = Yielders.each(cursorSequence); closer.register(yielder); final Cursor cursor = yielder.get(); @@ -378,14 +376,9 @@ private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumn ); Assert.assertEquals(1, segments.size()); StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); - VectorCursor cursor = storageAdapter.makeVectorCursor( - null, - Intervals.ETERNITY, - virtualColumns, - false, - 512, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build(); + VectorCursor cursor = storageAdapter.asCursorMaker(buildSpec).makeVectorCursor(); + closer.register(cursor); return cursor.getColumnSelectorFactory(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 8224f24e8954..5cd8cc2778e0 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -28,7 +28,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; @@ -45,6 +44,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -138,15 +138,11 @@ public void test_single_value_string_bindings() { final String columnName = "string3"; for (StorageAdapter adapter : ADAPTERS) { - Sequence cursorSequence = adapter.makeCursors( - null, - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(adapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -213,14 +209,11 @@ public void test_multi_value_string_bindings() { final String columnName = "multi-string3"; for (StorageAdapter adapter : ADAPTERS) { - Sequence cursorSequence = adapter.makeCursors( - null, - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(adapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); cursorSequence.accumulate(null, (ignored, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -299,14 +292,11 @@ public void test_long_bindings() { final String columnName = "long3"; for (StorageAdapter adapter : ADAPTERS) { - Sequence cursorSequence = adapter.makeCursors( - null, - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(adapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -354,14 +344,11 @@ public void test_double_bindings() { final String columnName = "double3"; for (StorageAdapter adapter : ADAPTERS) { - Sequence cursorSequence = adapter.makeCursors( - null, - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(adapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); cursorSequence.accumulate(null, (accumulated, cursor) -> { @@ -670,14 +657,7 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); int rowsProcessed = cursors.map(cursor -> { DimensionSelector xExprSelector = ExpressionSelectors.makeDimensionSelector( cursor.getColumnSelectorFactory(), 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 6f733ee0b91c..0caf9b0eab59 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 @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -116,15 +117,11 @@ public static void testCast( ) ); final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); - VectorCursor cursor = storageAdapter.makeVectorCursor( - null, - index.getDataInterval(), - virtualColumns, - false, - 512, - null - ); - closer.register(cursor); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(virtualColumns) + .setInterval(index.getDataInterval()) + .build(); + VectorCursor cursor = closer.register(storageAdapter.asCursorMaker(buildSpec).makeVectorCursor()); ColumnCapabilities capabilities = INDEX.getColumnCapabilities(column); 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 cd8b78ed1f97..f88bfd322ddb 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 @@ -31,6 +31,7 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; @@ -229,14 +230,12 @@ public static void sanityTestVectorizedExpressionSelectors( ) ); final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); - VectorCursor cursor = storageAdapter.makeVectorCursor( - null, - index.getDataInterval(), - virtualColumns, - false, - 512, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns(virtualColumns) + .build(); + final VectorCursor cursor = storageAdapter.asCursorMaker(buildSpec).makeVectorCursor(); ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); @@ -299,14 +298,7 @@ public static void sanityTestVectorizedExpressionSelectors( } closer.register(cursor); - Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - virtualColumns, - Granularities.ALL, - false, - null - ); + Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); int rowCountCursor = cursors .map(nonVectorized -> { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java index c0064a25f64c..8e5f0dfbaa11 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java @@ -37,8 +37,8 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; @@ -79,14 +79,16 @@ public Sequence apply(WindowedStorageAdapter adapter) { return Sequences.concat( Sequences.map( - adapter.getAdapter().makeCursors( - Filters.toFilter(dimFilter), - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), new Function>() + adapter.getAdapter() + .asCursorMaker( + CursorBuildSpec.builder() + .setFilter(Filters.toFilter(dimFilter)) + .setInterval(adapter.getInterval()) + .setGranularity(Granularities.ALL) + .build() + ) + .makeCursors() + , new Function>() { @Nullable @Override diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 3c663e22488f..99170d298270 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -71,13 +71,13 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SimpleAscendingOffset; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; @@ -281,14 +281,16 @@ private void runDump(final Injector injector, final QueryableIndex index) throws final List columnNames = getColumnsToInclude(index); final DimFilter filter = filterJson != null ? objectMapper.readValue(filterJson, DimFilter.class) : null; - final Sequence cursors = adapter.makeCursors( - Filters.toFilter(filter), - index.getDataInterval().withChronology(ISOChronology.getInstanceUTC()), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(Filters.toFilter(filter)) + .setInterval( + index.getDataInterval() + .withChronology(ISOChronology.getInstanceUTC()) + ) + .setGranularity(Granularities.ALL) + .build(); + + final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); withOutputStream( new Function() From ff3999c959ae9bbb05db2748b8b6213a3e9801dd Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 3 Jun 2024 15:10:54 -0700 Subject: [PATCH 02/74] add dump segment test, fixes --- .../benchmark/JoinAndLookupBenchmark.java | 16 +++++++- .../java/org/apache/druid/query/Query.java | 22 +++++------ .../druid/query/groupby/GroupByQuery.java | 22 +++++------ .../apache/druid/query/scan/ScanQuery.java | 21 +++++----- .../query/search/CursorOnlyStrategy.java | 14 +------ .../query/search/UseIndexesStrategy.java | 4 +- .../query/timeseries/TimeseriesQuery.java | 22 +++++------ .../apache/druid/query/topn/TopNQuery.java | 22 +++++------ .../java/org/apache/druid/segment/Cursor.java | 4 +- .../firehose/IngestSegmentFirehose.java | 4 +- .../org/apache/druid/cli/DumpSegment.java | 37 ++++++++++++------ .../org/apache/druid/cli/DumpSegmentTest.java | 38 +++++++++++++++++++ 12 files changed, 138 insertions(+), 88 deletions(-) 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 c803cfb786e3..291e75c004fe 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -519,7 +519,12 @@ public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) public void lookupVirtualColumnStringKey(Blackhole blackhole) { final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .asCursorMaker( + CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .setGranularity(Granularities.ALL) + .build() + ) .makeCursors(); blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); @@ -536,6 +541,7 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) CursorBuildSpec.builder() .setGranularity(Granularities.ALL) .setFilter(filter) + .setVirtualColumns(lookupVirtualColumns) .build() ) .makeCursors(); @@ -549,7 +555,12 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) public void lookupVirtualColumnLongKey(Blackhole blackhole) { final Sequence cursors = baseSegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .asCursorMaker( + CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .setGranularity(Granularities.ALL) + .build() + ) .makeCursors(); blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); @@ -565,6 +576,7 @@ public void lookupVirtualColumnLongKeyWithFilter(Blackhole blackhole) .asCursorMaker( CursorBuildSpec.builder() .setGranularity(Granularities.ALL) + .setVirtualColumns(lookupVirtualColumns) .setFilter(filter) .build() ) diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 4ce65c90f793..293e26a027c9 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -51,7 +51,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -288,16 +287,15 @@ default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics intervals ); } - return new CursorBuildSpec( - Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), - Iterables.getOnlyElement(intervals), - getGranularity(), - columns == null ? Collections.emptyList() : new ArrayList<>(columns), - getVirtualColumns(), - Collections.emptyList(), - context(), - isDescending(), - queryMetrics - ); + return CursorBuildSpec.builder() + .setInterval(Iterables.getOnlyElement(intervals)) + .setGranularity(getGranularity()) + .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) + .setColumns(columns == null ? null : new ArrayList<>(columns)) + .setVirtualColumns(getVirtualColumns()) + .setQueryContext(context()) + .isDescending(isDescending()) + .setQueryMetrics(queryMetrics) + .build(); } } 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 2dbcee9a4d2d..26c24ccde1fc 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 @@ -854,17 +854,17 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) intervals ); } - return new CursorBuildSpec( - Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), - Iterables.getOnlyElement(intervals), - getGranularity(), - columns == null ? Collections.emptyList() : new ArrayList<>(columns), - getVirtualColumns(), - getAggregatorSpecs(), - context(), - isDescending(), - queryMetrics - ); + return CursorBuildSpec.builder() + .setInterval(Iterables.getOnlyElement(intervals)) + .setGranularity(getGranularity()) + .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) + .setColumns(columns == null ? Collections.emptyList() : new ArrayList<>(columns)) + .setVirtualColumns(getVirtualColumns()) + .setAggregators(getAggregatorSpecs()) + .setQueryContext(context()) + .isDescending(isDescending()) + .setQueryMetrics(queryMetrics) + .build(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 1583c57d8f26..f61524efe17c 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -514,17 +514,16 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) intervals ); } - return new CursorBuildSpec( - Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), - Iterables.getOnlyElement(intervals), - getGranularity(), - columns == null ? Collections.emptyList() : new ArrayList<>(columns), - getVirtualColumns(), - Collections.emptyList(), - context(), - isDescending() || Order.DESCENDING.equals(timeOrder), - queryMetrics - ); + return CursorBuildSpec.builder() + .setInterval(Iterables.getOnlyElement(intervals)) + .setGranularity(getGranularity()) + .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) + .setColumns(columns == null ? null : new ArrayList<>(columns)) + .setVirtualColumns(getVirtualColumns()) + .setQueryContext(context()) + .isDescending(isDescending() || Order.DESCENDING.equals(timeOrder)) + .setQueryMetrics(queryMetrics) + .build(); } public ScanQuery withOffset(final long newOffset) diff --git a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index 435cd1baa9f9..ffcdfdb0930b 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -24,13 +24,11 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.search.SearchQueryRunner.SearchColumnSelectorStrategy; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; -import org.joda.time.Interval; import java.util.List; @@ -56,29 +54,19 @@ public List getExecutionPlan(SearchQuery query, Segment seg return ImmutableList.of(new CursorBasedExecutor( query, segment, - filter, - interval, dimensionSpecs )); } public static class CursorBasedExecutor extends SearchQueryExecutor { - - protected Filter filter; - protected Interval interval; - public CursorBasedExecutor( SearchQuery query, Segment segment, - Filter filter, - Interval interval, List dimensionSpecs + List dimensionSpecs ) { super(query, segment, dimensionSpecs); - - this.filter = filter; - this.interval = interval; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java index 8fc431a756b5..c522e70c90b1 100644 --- a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java @@ -114,10 +114,10 @@ public List getExecutionPlan(SearchQuery query, Segment seg } if (nonBitmapSuppDims.size() > 0) { - builder.add(new CursorBasedExecutor(query, segment, filter, interval, nonBitmapSuppDims)); + builder.add(new CursorBasedExecutor(query, segment, nonBitmapSuppDims)); } } else { - builder.add(new CursorBasedExecutor(query, segment, filter, interval, searchDims)); + builder.add(new CursorBasedExecutor(query, segment, searchDims)); } return builder.build(); diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 8d60ea86d341..3cac7870e32a 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -198,17 +198,17 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) intervals ); } - return new CursorBuildSpec( - Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), - Iterables.getOnlyElement(intervals), - getGranularity(), - columns == null ? Collections.emptyList() : new ArrayList<>(columns), - getVirtualColumns(), - getAggregatorSpecs(), - context(), - isDescending(), - queryMetrics - ); + return CursorBuildSpec.builder() + .setInterval(Iterables.getOnlyElement(intervals)) + .setGranularity(getGranularity()) + .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) + .setColumns(columns == null ? Collections.emptyList() : new ArrayList<>(columns)) + .setVirtualColumns(getVirtualColumns()) + .setAggregators(getAggregatorSpecs()) + .setQueryContext(context()) + .isDescending(isDescending()) + .setQueryMetrics(queryMetrics) + .build(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 68cf87335191..26294bb02f93 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -195,17 +195,17 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) intervals ); } - return new CursorBuildSpec( - Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())), - Iterables.getOnlyElement(intervals), - getGranularity(), - columns == null ? Collections.emptyList() : new ArrayList<>(columns), - getVirtualColumns(), - getAggregatorSpecs(), - context(), - isDescending(), - queryMetrics - ); + return CursorBuildSpec.builder() + .setInterval(Iterables.getOnlyElement(intervals)) + .setGranularity(getGranularity()) + .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) + .setColumns(columns == null ? Collections.emptyList() : new ArrayList<>(columns)) + .setVirtualColumns(getVirtualColumns()) + .setAggregators(getAggregatorSpecs()) + .setQueryContext(context()) + .isDescending(isDescending()) + .setQueryMetrics(queryMetrics) + .build(); } public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 3f8d1af1630b..dce076f3a91e 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -24,8 +24,8 @@ /** * Cursor is an interface for iteration over a range of data points, used during query execution. {@link * QueryableIndexCursorMaker.QueryableIndexCursor} is an implementation for historical segments, and {@link - * org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter.IncrementalIndexCursor} is an implementation for {@link - * org.apache.druid.segment.incremental.IncrementalIndex}. + * org.apache.druid.segment.incremental.IncrementalIndexCursorMaker.IncrementalIndexCursor} is an implementation for + * {@link org.apache.druid.segment.incremental.IncrementalIndex}. * * Cursor is conceptually similar to {@link TimeAndDimsPointer}, but the latter is used for historical segment creation * rather than query execution (as Cursor). If those abstractions could be collapsed (and if it is worthwhile) is yet to diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java index 8e5f0dfbaa11..036c9fa637fb 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java @@ -87,8 +87,8 @@ public Sequence apply(WindowedStorageAdapter adapter) .setGranularity(Granularities.ALL) .build() ) - .makeCursors() - , new Function>() + .makeCursors(), + new Function>() { @Nullable @Override diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 99170d298270..940ef3ac1d68 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -197,7 +197,7 @@ public void run() try (final QueryableIndex index = indexIO.loadIndex(new File(directory))) { switch (dumpType) { case ROWS: - runDump(injector, index); + runDump(injector, outputFileName, index, getColumnsToInclude(index), filterJson, timeISO8601); break; case METADATA: runMetadata(injector, index); @@ -226,6 +226,16 @@ public void run() } } + private List getColumnsToInclude(final QueryableIndex index) + { + return getColumnsToInclude(index, columnNamesFromCli); + } + + private T withOutputStream(Function f) throws IOException + { + return withOutputStream(f, outputFileName); + } + private void runMetadata(final Injector injector, final QueryableIndex index) throws IOException { final ObjectMapper objectMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)) @@ -274,11 +284,19 @@ public Object apply(SegmentAnalysis analysis) ); } - private void runDump(final Injector injector, final QueryableIndex index) throws IOException + @VisibleForTesting + public static void runDump( + final Injector injector, + final String outputFileName, + final QueryableIndex index, + final List columnNames, + final String filterJson, + final boolean timeISO8601 + ) + throws IOException { final ObjectMapper objectMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final List columnNames = getColumnsToInclude(index); final DimFilter filter = filterJson != null ? objectMapper.readValue(filterJson, DimFilter.class) : null; final CursorBuildSpec buildSpec = CursorBuildSpec.builder() @@ -345,7 +363,8 @@ public Object apply(Cursor cursor) return null; } - } + }, + outputFileName ); } @@ -689,9 +708,10 @@ public static void runDumpNestedColumnPath( ); } - private List getColumnsToInclude(final QueryableIndex index) + @VisibleForTesting + public static List getColumnsToInclude(final QueryableIndex index, List columns) { - final Set columnNames = Sets.newLinkedHashSet(columnNamesFromCli); + final Set columnNames = Sets.newLinkedHashSet(columns); // Empty columnNames => include all columns. if (columnNames.isEmpty()) { @@ -709,11 +729,6 @@ private List getColumnsToInclude(final QueryableIndex index) return ImmutableList.copyOf(columnNames); } - private T withOutputStream(Function f) throws IOException - { - return withOutputStream(f, outputFileName); - } - @SuppressForbidden(reason = "System#out") private static T withOutputStream(Function f, String outputFileName) throws IOException { diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 9369717bf1f6..619c5a4ebfb1 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -115,6 +115,44 @@ public void testExecuteQuery() Assert.assertSame(expected, actual); } + @Test + public void testDumpRows() throws Exception + { + Injector injector = Mockito.mock(Injector.class); + ObjectMapper mapper = TestHelper.makeJsonMapper(); + mapper.registerModules(NestedDataModule.getJacksonModulesList()); + mapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), mapper) + .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null)) + ); + Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper); + Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null)); + + List segments = createSegments(tempFolder, closer); + QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + + File outputFile = tempFolder.newFile(); + + DumpSegment.runDump( + injector, + outputFile.getPath(), + queryableIndex, + DumpSegment.getColumnsToInclude(queryableIndex, Collections.emptyList()), + null, + false + ); + final byte[] fileBytes = Files.readAllBytes(outputFile.toPath()); + final String output = StringUtils.fromUtf8(fileBytes); + final String expected = "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":200,\"y\":2.2}}\n" + + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":400,\"y\":1.1,\"z\":\"a\"}}\n" + + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":200,\"z\":\"b\"}}\n" + + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":100,\"y\":1.1,\"z\":\"a\"}}\n" + + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"y\":3.3,\"z\":\"b\"}}\n"; + Assert.assertEquals(expected, output); + } + @Test public void testDumpBitmap() throws IOException { From dc4a6b4c76db9e1b299e80e8539b89f533d6de6e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 3 Jun 2024 16:08:17 -0700 Subject: [PATCH 03/74] final --- .../src/main/java/org/apache/druid/segment/CursorBuildSpec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index 19265a013d22..6ba6b44c15b5 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -33,7 +33,7 @@ public class CursorBuildSpec { - public static CursorBuildSpec FULL_SCAN = CursorBuildSpec.builder().setGranularity(Granularities.ALL).build(); + public static final CursorBuildSpec FULL_SCAN = CursorBuildSpec.builder().setGranularity(Granularities.ALL).build(); public static CursorBuildSpecBuilder builder() { From dca26501f0090078e06279b487e3cb1458bffbcf Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 4 Jun 2024 20:50:38 -0700 Subject: [PATCH 04/74] adjustments --- .../java/org/apache/druid/query/Queries.java | 24 +++------ .../java/org/apache/druid/query/Query.java | 3 -- .../druid/query/groupby/GroupByQuery.java | 13 ++--- .../druid/query/metadata/SegmentAnalyzer.java | 2 - .../LazilyDecoratedRowsAndColumns.java | 22 ++++---- .../apache/druid/query/scan/ScanQuery.java | 8 +-- .../query/timeseries/TimeseriesQuery.java | 5 +- .../apache/druid/query/topn/TopNQuery.java | 8 ++- .../apache/druid/segment/CursorBuildSpec.java | 27 +++++----- .../join/HashJoinSegmentStorageAdapter.java | 1 - .../table/BroadcastSegmentIndexedTable.java | 1 - .../segment/virtual/ExpressionSelectors.java | 2 +- .../druid/query/groupby/GroupByQueryTest.java | 50 +++++++++++++++++- .../druid/query/scan/ScanQueryTest.java | 33 +++++++++++- .../query/timeseries/TimeseriesQueryTest.java | 52 ++++++++++++++++++- .../druid/query/topn/TopNQueryTest.java | 34 +++++++++++- 16 files changed, 210 insertions(+), 75 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index c08f63938bb5..e3262872d7a3 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; @@ -36,7 +35,6 @@ import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nullable; - import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -206,17 +204,15 @@ public static Query withBaseDataSource(final Query query, final DataSo * * @param virtualColumns virtual columns whose inputs should be included. * @param filter optional filter whose inputs should be included. - * @param dimensions dimension specs whose inputs should be included. - * @param aggregators aggregators whose inputs should be included. - * @param additionalColumns additional columns to include. Each of these will be added to the returned set, unless it + * @param columns additional columns to include. Each of these will be added to the returned set, unless it * refers to a virtual column, in which case the virtual column inputs will be added instead. + * @param aggregators aggregators whose inputs should be included. */ public static Set computeRequiredColumns( final VirtualColumns virtualColumns, @Nullable final DimFilter filter, - final List dimensions, - final List aggregators, - final List additionalColumns + final List columns, + final List aggregators ) { final Set requiredColumns = new HashSet<>(); @@ -240,9 +236,9 @@ public static Set computeRequiredColumns( } } - for (DimensionSpec dimensionSpec : dimensions) { - if (!virtualColumns.exists(dimensionSpec.getDimension())) { - requiredColumns.add(dimensionSpec.getDimension()); + for (String column : columns) { + if (!virtualColumns.exists(column)) { + requiredColumns.add(column); } } @@ -254,12 +250,6 @@ public static Set computeRequiredColumns( } } - for (String column : additionalColumns) { - if (!virtualColumns.exists(column)) { - requiredColumns.add(column); - } - } - return requiredColumns; } diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 293e26a027c9..dd3b0463dadb 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -50,7 +50,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -279,7 +278,6 @@ default Set getRequiredColumns() default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final Set columns = getRequiredColumns(); final List intervals = getIntervals(); if (intervals.size() > 1) { throw DruidException.defensive( @@ -291,7 +289,6 @@ default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics .setInterval(Iterables.getOnlyElement(intervals)) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setColumns(columns == null ? null : new ArrayList<>(columns)) .setVirtualColumns(getVirtualColumns()) .setQueryContext(context()) .isDescending(isDescending()) 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 26c24ccde1fc..f78ee48fed0b 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 @@ -79,7 +79,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.List; @@ -114,6 +113,8 @@ public static Builder builder() @Nullable private final DimFilter dimFilter; private final List dimensions; + private final List groupingColumns; + private final List aggregatorSpecs; private final List postAggregatorSpecs; @Nullable @@ -211,8 +212,10 @@ private GroupByQuery( this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns); this.dimFilter = dimFilter; this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; + this.groupingColumns = new ArrayList<>(); for (DimensionSpec spec : this.dimensions) { Preconditions.checkArgument(spec != null, "dimensions has null DimensionSpec"); + groupingColumns.add(spec.getDimension()); } this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; @@ -837,16 +840,14 @@ public Set getRequiredColumns() return Queries.computeRequiredColumns( virtualColumns, dimFilter, - dimensions, - aggregatorSpecs, - Collections.emptyList() + groupingColumns, + aggregatorSpecs ); } @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final Set columns = getRequiredColumns(); final List intervals = getIntervals(); if (intervals.size() > 1) { throw DruidException.defensive( @@ -858,7 +859,7 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setInterval(Iterables.getOnlyElement(intervals)) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setColumns(columns == null ? Collections.emptyList() : new ArrayList<>(columns)) + .setGroupingColumns(groupingColumns) .setVirtualColumns(getVirtualColumns()) .setAggregators(getAggregatorSpecs()) .setQueryContext(context()) diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index d1154aa47da1..d0f46a47e712 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -57,7 +57,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.Collections; import java.util.EnumSet; import java.util.LinkedHashMap; import java.util.Map; @@ -278,7 +277,6 @@ private ColumnAnalysis analyzeStringColumn( final DateTime start = storageAdapter.getMinTime(); final DateTime end = storageAdapter.getMaxTime(); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setColumns(Collections.singletonList(columnName)) .setInterval(new Interval(start, end)) .setGranularity(Granularities.ALL) .build(); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index da07c3c2587c..429ab42c58d2 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -29,7 +29,6 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; @@ -209,10 +208,8 @@ private void reset(RowsAndColumns rac) private Pair materializeStorageAdapter(StorageAdapter as) { final Collection cols; - final List cursorColumns; if (viewableColumns != null) { cols = viewableColumns; - cursorColumns = new ArrayList<>(viewableColumns); } else { if (virtualColumns == null) { cols = base.getColumnNames(); @@ -222,16 +219,17 @@ private Pair materializeStorageAdapter(StorageAdapter as) .addAll(virtualColumns.getColumnNames()) .build(); } - cursorColumns = new ArrayList<>(base.getColumnNames()); } - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setInterval(interval == null ? Intervals.ETERNITY : interval) - .setGranularity(Granularities.ALL) - .setColumns(cursorColumns) - .setVirtualColumns(virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns) - .build(); - final Sequence cursors = as.asCursorMaker(buildSpec).makeCursors(); + final CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() + .setFilter(filter) + .setGranularity(Granularities.ALL); + if (interval != null) { + builder.setInterval(interval); + } + if (virtualColumns != null) { + builder.setVirtualColumns(virtualColumns); + } + final Sequence cursors = as.asCursorMaker(builder.build()).makeCursors(); AtomicReference siggy = new AtomicReference<>(null); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index f61524efe17c..b8a61c3084df 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -54,7 +54,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -496,9 +495,8 @@ public Set getRequiredColumns() return Queries.computeRequiredColumns( virtualColumns, dimFilter, - Collections.emptyList(), - Collections.emptyList(), - columns + columns, + Collections.emptyList() ); } } @@ -506,7 +504,6 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final Set columns = getRequiredColumns(); final List intervals = getIntervals(); if (intervals.size() > 1) { throw DruidException.defensive( @@ -518,7 +515,6 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setInterval(Iterables.getOnlyElement(intervals)) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setColumns(columns == null ? null : new ArrayList<>(columns)) .setVirtualColumns(getVirtualColumns()) .setQueryContext(context()) .isDescending(isDescending() || Order.DESCENDING.equals(timeOrder)) diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 3cac7870e32a..b02454a9f575 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -181,8 +181,7 @@ public Set getRequiredColumns() virtualColumns, dimFilter, Collections.emptyList(), - aggregatorSpecs, - Collections.emptyList() + aggregatorSpecs ); } @@ -190,7 +189,6 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final Set columns = getRequiredColumns(); final List intervals = getIntervals(); if (intervals.size() > 1) { throw DruidException.defensive( @@ -202,7 +200,6 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setInterval(Iterables.getOnlyElement(intervals)) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setColumns(columns == null ? Collections.emptyList() : new ArrayList<>(columns)) .setVirtualColumns(getVirtualColumns()) .setAggregators(getAggregatorSpecs()) .setQueryContext(context()) diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 26294bb02f93..1dc60e48d9cc 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -177,9 +177,8 @@ public Set getRequiredColumns() return Queries.computeRequiredColumns( virtualColumns, dimFilter, - Collections.singletonList(dimensionSpec), - aggregatorSpecs, - Collections.emptyList() + Collections.singletonList(dimensionSpec.getDimension()), + aggregatorSpecs ); } @@ -187,7 +186,6 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final Set columns = getRequiredColumns(); final List intervals = getIntervals(); if (intervals.size() > 1) { throw DruidException.defensive( @@ -199,7 +197,7 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setInterval(Iterables.getOnlyElement(intervals)) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setColumns(columns == null ? Collections.emptyList() : new ArrayList<>(columns)) + .setGroupingColumns(Collections.singletonList(dimensionSpec.getDimension())) .setVirtualColumns(getVirtualColumns()) .setAggregators(getAggregatorSpecs()) .setQueryContext(context()) diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index 6ba6b44c15b5..0383f43df6aa 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -50,7 +50,7 @@ public static CursorBuildSpecBuilder builder(CursorBuildSpec spec) private final Interval interval; private final Granularity granularity; @Nullable - private final List columns; + private final List groupingColumns; private final VirtualColumns virtualColumns; @Nullable private final List aggregators; @@ -65,7 +65,7 @@ public CursorBuildSpec( @Nullable Filter filter, Interval interval, Granularity granularity, - @Nullable List columns, + @Nullable List groupingColumns, VirtualColumns virtualColumns, @Nullable List aggregators, QueryContext queryContext, @@ -76,7 +76,7 @@ public CursorBuildSpec( this.filter = filter; this.interval = interval; this.granularity = granularity; - this.columns = columns; + this.groupingColumns = groupingColumns; this.virtualColumns = virtualColumns; this.aggregators = aggregators; this.descending = descending; @@ -101,9 +101,9 @@ public Granularity getGranularity() } @Nullable - public List getColumns() + public List getGroupingColumns() { - return columns; + return groupingColumns; } public VirtualColumns getVirtualColumns() @@ -111,6 +111,7 @@ public VirtualColumns getVirtualColumns() return virtualColumns; } + @Nullable public List getAggregators() { return aggregators; @@ -140,7 +141,7 @@ public static class CursorBuildSpecBuilder private Granularity granularity = Granularities.NONE; @Nullable - private List columns = null; + private List groupingColumns = null; private VirtualColumns virtualColumns = VirtualColumns.EMPTY; @Nullable private List aggregators = null; @@ -150,17 +151,17 @@ public static class CursorBuildSpecBuilder @Nullable private QueryMetrics queryMetrics; - public CursorBuildSpecBuilder() + private CursorBuildSpecBuilder() { - + // } - public CursorBuildSpecBuilder(CursorBuildSpec buildSpec) + private CursorBuildSpecBuilder(CursorBuildSpec buildSpec) { this.filter = buildSpec.filter; this.interval = buildSpec.interval; this.granularity = buildSpec.granularity; - this.columns = buildSpec.columns; + this.groupingColumns = buildSpec.groupingColumns; this.virtualColumns = buildSpec.virtualColumns; this.aggregators = buildSpec.aggregators; this.descending = buildSpec.descending; @@ -186,9 +187,9 @@ public CursorBuildSpecBuilder setGranularity(Granularity granularity) return this; } - public CursorBuildSpecBuilder setColumns(@Nullable List columns) + public CursorBuildSpecBuilder setGroupingColumns(@Nullable List columns) { - this.columns = columns; + this.groupingColumns = columns; return this; } @@ -228,7 +229,7 @@ public CursorBuildSpec build() filter, interval, granularity, - columns, + groupingColumns, virtualColumns, aggregators, queryContext, diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index f752d7062c1c..a7986ae1666c 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -228,7 +228,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) .setInterval(spec.getInterval()) .setGranularity(spec.getGranularity()) .isDescending(spec.isDescending()) - .setColumns(baseAdapter.getRowSignature().getColumnNames()) .setQueryMetrics(spec.getQueryMetrics()); final Filter combinedFilter = baseFilterAnd(spec.getFilter()); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index 72373af40fdc..ee8d70cab305 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -120,7 +120,6 @@ public BroadcastSegmentIndexedTable( .withChronology(ISOChronology.getInstanceUTC()) ) .setGranularity(Granularities.ALL) - .setColumns(keyColumnNames) .build(); final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index ca2cda3e0e14..0c91ae5d3946 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -211,7 +211,7 @@ public static ColumnValueSelector makeExprEvalSelector( !ColumnHolder.TIME_COLUMN_NAME.equals(column), // __time doesn't need an LRU cache since it is sorted. rowIdSupplier ); - } else if (inputType.is(ValueType.STRING)) { + } else if (inputType.is(ValueType.STRING) && plan.is(ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE)) { return new SingleStringInputCachingExpressionColumnValueSelector( columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ColumnType.STRING)), plan.getExpression(), diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java index 2784d6b89181..57a546550058 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java @@ -40,7 +40,9 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -162,6 +164,51 @@ public void testSegmentLookUpForNestedQueries() Assert.assertEquals(innerQuerySegmentSpec, BaseQuery.getQuerySegmentSpecForLookUp(query)); } + @Test + public void testAsCursorBuildSpec() + { + final VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v0", "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil()) + ); + final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index"); + Query query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) + .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) + .setDimensions( + new DefaultDimensionSpec(QueryRunnerTestHelper.QUALITY_DIMENSION, "alias"), + new DefaultDimensionSpec( + QueryRunnerTestHelper.MARKET_DIMENSION, + "market", + ColumnType.STRING_ARRAY + ), + new DefaultDimensionSpec("v0", "v0", ColumnType.STRING) + ) + .setVirtualColumns(virtualColumns) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, longSum) + .setGranularity(QueryRunnerTestHelper.DAY_GRAN) + .setPostAggregatorSpecs(ImmutableList.of(new FieldAccessPostAggregator("x", "idx"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of(new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.LEXICOGRAPHIC + )), + 100 + ) + ) + .build(); + + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); + Assert.assertEquals(Granularities.DAY, buildSpec.getGranularity()); + Assert.assertEquals(ImmutableList.of("quality", "market", "v0"), buildSpec.getGroupingColumns()); + Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); + Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); + Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); + } + @Test public void testEquals() { @@ -176,7 +223,8 @@ public void testEquals() "forceLimitPushDown", "postProcessingFn", "resultRowSignature", - "universalTimestamp" + "universalTimestamp", + "groupingColumns" ) .verify(); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index a79936b8818b..1804125c0ce6 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -23,14 +23,21 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; import org.junit.Assert; import org.junit.BeforeClass; @@ -43,7 +50,7 @@ import java.util.List; import java.util.Set; -public class ScanQueryTest +public class ScanQueryTest extends InitializedNullHandlingTest { private static QuerySegmentSpec intervalSpec; private static ScanResultValue s1; @@ -400,4 +407,28 @@ public void testGetRowSignature() Assert.assertEquals(sig, query.getRowSignature()); } + + @Test + public void testAsCursorBuildSpec() + { + final VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v0", "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil()) + ); + final ScanQuery query = + Druids.newScanQueryBuilder() + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) + .dataSource("some src") + .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD) + .virtualColumns(virtualColumns) + .columns("foo", "bar") + .build(); + + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); + Assert.assertEquals(Granularities.ALL, buildSpec.getGranularity()); + Assert.assertNull(buildSpec.getGroupingColumns()); + Assert.assertNull(buildSpec.getAggregators()); + Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); + Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); + } } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java index e8f10da8bf3d..444863135d84 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java @@ -20,15 +20,20 @@ package org.apache.druid.query.timeseries; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -38,7 +43,7 @@ import java.util.Arrays; @RunWith(Parameterized.class) -public class TimeseriesQueryTest +public class TimeseriesQueryTest extends InitializedNullHandlingTest { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); @@ -101,4 +106,49 @@ public void testGetRequiredColumns() Assert.assertEquals(ImmutableSet.of("__time", "fieldFromVirtualColumn", "aField"), query.getRequiredColumns()); } + + @Test + public void testAsCursorBuildSpec() + { + final VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn( + "index", + "\"fieldFromVirtualColumn\"", + ColumnType.LONG, + ExprMacroTable.nil() + ) + ); + final LongSumAggregatorFactory beep = new LongSumAggregatorFactory("beep", "aField"); + final TimeseriesQuery query = + Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .granularity(QueryRunnerTestHelper.DAY_GRAN) + .virtualColumns(virtualColumns) + .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .aggregators( + QueryRunnerTestHelper.ROWS_COUNT, + QueryRunnerTestHelper.INDEX_DOUBLE_SUM, + QueryRunnerTestHelper.INDEX_LONG_MAX, + beep + ) + .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) + .descending(descending) + .build(); + + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + Assert.assertEquals(QueryRunnerTestHelper.FULL_ON_INTERVAL, buildSpec.getInterval()); + Assert.assertEquals(Granularities.DAY, buildSpec.getGranularity()); + Assert.assertNull(buildSpec.getGroupingColumns()); + Assert.assertEquals( + ImmutableList.of( + QueryRunnerTestHelper.ROWS_COUNT, + QueryRunnerTestHelper.INDEX_DOUBLE_SUM, + QueryRunnerTestHelper.INDEX_LONG_MAX, + beep + ), + buildSpec.getAggregators() + ); + Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); + Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); + } } diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java index 6d7f1a03be9a..028f958ca5b7 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerTestHelper; @@ -38,9 +39,12 @@ import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -48,7 +52,7 @@ import java.io.IOException; -public class TopNQueryTest +public class TopNQueryTest extends InitializedNullHandlingTest { @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -266,4 +270,32 @@ public void testGetRequiredColumns() Assert.assertEquals(ImmutableSet.of("__time", "other", "index"), query.getRequiredColumns()); } + + @Test + public void testAsCursorBuildSpec() + { + final VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn("v", "\"other\"", ColumnType.STRING, ExprMacroTable.nil()) + ); + final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index"); + final TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD) + .virtualColumns(virtualColumns) + .dimension(DefaultDimensionSpec.of("v")) + .aggregators(QueryRunnerTestHelper.ROWS_COUNT, longSum) + .granularity(QueryRunnerTestHelper.DAY_GRAN) + .postAggregators(ImmutableList.of(new FieldAccessPostAggregator("x", "idx"))) + .metric(new NumericTopNMetricSpec("idx")) + .threshold(100) + .build(); + + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); + Assert.assertEquals(Granularities.DAY, buildSpec.getGranularity()); + Assert.assertEquals(ImmutableList.of("v"), buildSpec.getGroupingColumns()); + Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); + Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); + Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); + } } From 5f60f65cf47fac072ef24b6d897eb53e8f801d4e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 4 Jun 2024 22:47:23 -0700 Subject: [PATCH 05/74] oops, missing unstaged from merge --- .../org/apache/druid/indexing/common/task/IndexTaskTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 53c0d9ffbe89..cb52bea4fc67 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -75,11 +75,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SegmentSchemaMapping; -<<<<<<< HEAD -======= import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.VirtualColumns; ->>>>>>> upstream/master import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.CompressionStrategy; From 3858f361d551d544ed4c5cdbdfd005c25a59bfbe Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 11 Jun 2024 17:52:09 -0700 Subject: [PATCH 06/74] adjust --- .../org/apache/druid/segment/CursorMaker.java | 29 +++++++++++++++---- .../segment/QueryableIndexStorageAdapter.java | 6 ++-- .../druid/segment/UnnestStorageAdapter.java | 10 +------ .../join/HashJoinSegmentStorageAdapter.java | 19 ++---------- .../segment/virtual/ExpressionSelectors.java | 2 +- 5 files changed, 31 insertions(+), 35 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java b/processing/src/main/java/org/apache/druid/segment/CursorMaker.java index eef7bc410e53..6922814c5bb5 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorMaker.java @@ -28,19 +28,38 @@ public interface CursorMaker { - default boolean canVectorize() - { - return false; - } - + /** + * Create a {@link Sequence} of {@link Cursor} for use with non-vectorized query engines. Each {@link Cursor} of the + * sequence corresponds to a {@link org.apache.druid.java.util.common.granularity.Granularity} bucket determined by + * {@link CursorBuildSpec#getGranularity()}. + *

+ * Consuming this {@link Sequence} will automatically close all resources associated with this {@link CursorMaker} + * so calling {@link #cleanup()} is not needed. + */ Sequence makeCursors(); + /** + * Create a {@link VectorCursor} for use with vectorized query engines. + *

+ * Advancing this {@link VectorCursor} to the end or explicitly calling {@link VectorCursor#close()} will + * automatically close all resources associated with this {@link CursorMaker} so calling {@link #cleanup()} is not + * needed. + */ @Nullable default VectorCursor makeVectorCursor() { throw new UOE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor' on %s.", this.getClass().getName()); } + /** + * Returns true if this {@link CursorMaker} supports creating vectorized selectors. This operation may acquire + * underlying resources, so calling {@link #cleanup()} is necessary if no cursors are created and consumed. + */ + default boolean canVectorize() + { + return false; + } + /** * Release any resources acquired if cursors are not consumed. Typically consuming a cursor or vector cursor releases * the resources upon completion, but if for some reason this will not happen, this method must be called. diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 0cbf4e7ac840..810f59cb4dbe 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -196,9 +196,9 @@ public boolean canVectorize( final boolean descending ) { - // todo (clint): - // this uses the old-school canVectorize implementation instead of delegating to the CursorMaker, because - // the cursor maker expects to make cursors one way or another, and so opens stuff, so need to fix some tests + // For safety, this uses the old-school canVectorize implementation instead of delegating to the CursorMaker, + // because QueryableIndexCursorMaker expects to make cursors one way or another and so opens stuff that must be + // cleaned if for some reason a cursor or vector cursor is not constructed if (filter != null) { final boolean filterCanVectorize = filter.canVectorizeMatcher(this); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index b3b2f93b7441..6b345dde5cae 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -160,15 +160,7 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setInterval(interval) - .setGranularity(gran) - .setVirtualColumns(virtualColumns) - .isDescending(descending) - .setQueryMetrics(queryMetrics) - .build(); - return asCursorMaker(buildSpec).makeCursors(); + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index a7986ae1666c..6e4ef7e84d70 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -338,14 +338,7 @@ public VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setInterval(interval) - .setVirtualColumns(virtualColumns) - .isDescending(descending) - .setQueryMetrics(queryMetrics) - .build(); - return asCursorMaker(buildSpec).makeVectorCursor(); + return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); } @Override @@ -359,15 +352,7 @@ public Sequence makeCursors( ) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setInterval(interval) - .setGranularity(gran) - .setVirtualColumns(virtualColumns) - .isDescending(descending) - .setQueryMetrics(queryMetrics) - .build(); - return asCursorMaker(buildSpec).makeCursors(); + return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index 0c91ae5d3946..ca2cda3e0e14 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -211,7 +211,7 @@ public static ColumnValueSelector makeExprEvalSelector( !ColumnHolder.TIME_COLUMN_NAME.equals(column), // __time doesn't need an LRU cache since it is sorted. rowIdSupplier ); - } else if (inputType.is(ValueType.STRING) && plan.is(ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE)) { + } else if (inputType.is(ValueType.STRING)) { return new SingleStringInputCachingExpressionColumnValueSelector( columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ColumnType.STRING)), plan.getExpression(), From 6718f2b155c4977110e02aca26845b30784c9dda Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 11 Jun 2024 19:12:06 -0700 Subject: [PATCH 07/74] adjust --- .../segment/FrameStorageAdapterTest.java | 20 ++++++++++--------- .../druid/frame/testutil/FrameTestUtil.java | 7 +------ 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index 854fb13b60c8..9b0d46f2fa80 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -56,7 +56,6 @@ import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; -import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -372,8 +371,6 @@ public void test_makeCursors() @Test public void test_makeVectorCursor() { - Assume.assumeTrue(frameAdapter.canVectorize(filter, virtualColumns, descending)); - assertVectorCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec)); } @@ -391,13 +388,18 @@ private void assertCursorsMatch(final Function> private void assertVectorCursorsMatch(final Function call) { final CursorMaker maker = call.apply(queryableAdapter); - final RowSignature signature = frameAdapter.getRowSignature(); - final Sequence> queryableRows = - FrameTestUtil.readRowsFromVectorCursor(maker.makeVectorCursor(), signature); final CursorMaker frameMaker = call.apply(frameAdapter); - final Sequence> frameRows = - FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameMaker.makeVectorCursor()), signature); - FrameTestUtil.assertRowsEqual(queryableRows, frameRows); + if (frameMaker.canVectorize()) { + final RowSignature signature = frameAdapter.getRowSignature(); + final Sequence> queryableRows = + FrameTestUtil.readRowsFromVectorCursor(maker.makeVectorCursor(), signature); + final Sequence> frameRows = + FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameMaker.makeVectorCursor()), signature); + FrameTestUtil.assertRowsEqual(queryableRows, frameRows); + } else { + maker.cleanup(); + frameMaker.cleanup(); + } } /** 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 8e15babadc72..7cd94b4f3972 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 @@ -159,12 +159,7 @@ public static void assertRowEqual(final String message, final List expec final Object expectedValue = expected.get(i); final Object actualValue = actual.get(i); - if (expectedValue instanceof List && actualValue instanceof Object[]) { - if (!Objects.deepEquals(expectedValue, Arrays.asList((Object[]) actualValue))) { - ok = false; - break; - } - } else if (!Objects.deepEquals(expectedValue, actualValue)) { + if (!Objects.deepEquals(expectedValue, actualValue)) { ok = false; break; } From 0b5d399bd4ac5f2756788ffbfffef4048af3f55d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 12 Jun 2024 04:40:23 -0700 Subject: [PATCH 08/74] missed a spot --- .../druid/segment/RowBasedStorageAdapter.java | 89 +++++++++++-------- 1 file changed, 52 insertions(+), 37 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index efd3b96829d6..d8b16cf900af 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -167,6 +167,57 @@ public Metadata getMetadata() throw new UnsupportedOperationException("Cannot retrieve metadata"); } + @Override + public CursorMaker asCursorMaker(CursorBuildSpec spec) + { + return new CursorMaker() + { + @Override + public Sequence makeCursors() + { + final Granularity gran = spec.getGranularity(); + final Interval actualInterval = spec.getInterval() + .overlap(new Interval(getMinTime(), gran.bucketEnd(getMaxTime()))); + + if (actualInterval == null) { + return Sequences.empty(); + } + + if (!isQueryGranularityAllowed(actualInterval, gran)) { + throw new IAE( + "Cannot support interval [%s] with granularity [%s]", + Intervals.ETERNITY.equals(actualInterval) ? "ETERNITY" : actualInterval, + gran + ); + } + + final RowWalker rowWalker = new RowWalker<>( + spec.isDescending() ? reverse(rowSequence) : rowSequence, + rowAdapter + ); + + final Iterable bucketIntervals = gran.getIterable(actualInterval); + + return Sequences.simple( + Iterables.transform( + spec.isDescending() ? reverse(bucketIntervals) : bucketIntervals, + bucketInterval -> + (Cursor) new RowBasedCursor<>( + rowWalker, + rowAdapter, + spec.getFilter(), + bucketInterval, + spec.getVirtualColumns(), + gran, + spec.isDescending(), + rowSignature + ) + ) + ).withBaggage(rowWalker::close); + } + }; + } + @Override public Sequence makeCursors( @Nullable final Filter filter, @@ -177,43 +228,7 @@ public Sequence makeCursors( @Nullable final QueryMetrics queryMetrics ) { - final Interval actualInterval = queryInterval.overlap(new Interval(getMinTime(), gran.bucketEnd(getMaxTime()))); - - if (actualInterval == null) { - return Sequences.empty(); - } - - if (!isQueryGranularityAllowed(actualInterval, gran)) { - throw new IAE( - "Cannot support interval [%s] with granularity [%s]", - Intervals.ETERNITY.equals(actualInterval) ? "ETERNITY" : actualInterval, - gran - ); - } - - final RowWalker rowWalker = new RowWalker<>( - descending ? reverse(rowSequence) : rowSequence, - rowAdapter - ); - - final Iterable bucketIntervals = gran.getIterable(actualInterval); - - return Sequences.simple( - Iterables.transform( - descending ? reverse(bucketIntervals) : bucketIntervals, - bucketInterval -> - (Cursor) new RowBasedCursor<>( - rowWalker, - rowAdapter, - filter, - bucketInterval, - virtualColumns, - gran, - descending, - rowSignature - ) - ) - ).withBaggage(rowWalker::close); + return delegateMakeCursorToMaker(filter, queryInterval, virtualColumns, gran, descending, queryMetrics); } /** From 8184213eb1e0351df0e04f81afff4b3d1584ddde Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 26 Jul 2024 04:12:59 -0700 Subject: [PATCH 09/74] changes: * CursorMaker no longer handles query granularity directly * Sequence of makeCursors is now just a Cursor from makeCursor * added CursorGranularizer to bucket queries by granularity instead, updated all engines that support query granularity to use this instead (topN still needs some more work) * remove Cursor.getTime * remove implementations of CursorFactory methods that are not asCursorMaker, replacing with defaults that throw exceptions --- .../ExpressionAggregationBenchmark.java | 31 +- .../benchmark/ExpressionFilterBenchmark.java | 44 +- .../ExpressionSelectorBenchmark.java | 734 ++++++++---------- .../ExpressionVectorSelectorBenchmark.java | 41 +- .../benchmark/FilterPartitionBenchmark.java | 149 ++-- .../IndexedTableJoinCursorBenchmark.java | 180 +++-- .../benchmark/JoinAndLookupBenchmark.java | 270 +++---- .../IncrementalIndexReadBenchmark.java | 68 +- .../SegmentGeneratorFrameProcessor.java | 70 +- .../results/ExportResultsFrameProcessor.java | 97 ++- .../scan/ScanQueryFrameProcessor.java | 63 +- .../indexing/input/DruidSegmentReader.java | 11 +- .../common/task/CompactionTaskRunTest.java | 42 +- .../indexing/common/task/IndexTaskTest.java | 157 ++-- .../frame/processor/FrameProcessors.java | 5 +- .../druid/frame/segment/FrameCursor.java | 8 - .../frame/segment/FrameStorageAdapter.java | 40 - .../segment/columnar/FrameCursorFactory.java | 54 +- .../frame/segment/row/FrameCursorFactory.java | 24 +- .../druid/query/CursorGranularizer.java | 190 +++++ .../query/FrameBasedInlineDataSource.java | 5 +- .../druid/query/groupby/GroupingEngine.java | 15 +- .../epinephelinae/GroupByQueryEngine.java | 216 ++++-- .../druid/query/metadata/SegmentAnalyzer.java | 47 +- .../LazilyDecoratedRowsAndColumns.java | 48 +- .../StorageAdapterRowsAndColumns.java | 40 +- .../druid/query/scan/ScanQueryEngine.java | 216 +++--- .../query/search/CursorOnlyStrategy.java | 71 +- .../TimeBoundaryQueryRunnerFactory.java | 25 +- .../timeseries/TimeseriesQueryEngine.java | 86 +- .../AggregateTopNMetricFirstAlgorithm.java | 9 +- .../druid/query/topn/BaseTopNAlgorithm.java | 6 +- .../topn/Generic1AggPooledTopNScanner.java | 2 + ...Generic1AggPooledTopNScannerPrototype.java | 6 +- .../topn/Generic2AggPooledTopNScanner.java | 2 + ...Generic2AggPooledTopNScannerPrototype.java | 6 +- .../query/topn/HeapBasedTopNAlgorithm.java | 6 +- .../druid/query/topn/PooledTopNAlgorithm.java | 36 +- .../topn/TimeExtractionTopNAlgorithm.java | 10 +- .../druid/query/topn/TopNAlgorithm.java | 3 +- .../apache/druid/query/topn/TopNMapFn.java | 5 +- .../apache/druid/query/topn/TopNParams.java | 9 + .../druid/query/topn/TopNQueryEngine.java | 42 +- ...eNumericTopNColumnAggregatesProcessor.java | 6 +- .../StringTopNColumnAggregatesProcessor.java | 16 +- .../types/TopNColumnAggregatesProcessor.java | 2 + .../vector/VectorCursorGranularizer.java | 62 +- .../java/org/apache/druid/segment/Cursor.java | 6 +- .../apache/druid/segment/CursorFactory.java | 95 +-- .../org/apache/druid/segment/CursorMaker.java | 34 +- .../druid/segment/FilteredStorageAdapter.java | 16 - .../segment/QueryableIndexCursorMaker.java | 145 ++-- .../segment/QueryableIndexStorageAdapter.java | 54 -- .../apache/druid/segment/RowBasedCursor.java | 6 - .../druid/segment/RowBasedStorageAdapter.java | 58 +- .../UnnestColumnValueSelectorCursor.java | 7 - .../druid/segment/UnnestDimensionCursor.java | 7 - .../druid/segment/UnnestStorageAdapter.java | 102 ++- .../IncrementalIndexCursorMaker.java | 49 +- .../IncrementalIndexStorageAdapter.java | 19 - .../druid/segment/join/HashJoinEngine.java | 8 - .../join/HashJoinSegmentStorageAdapter.java | 91 +-- .../druid/segment/join/PostJoinCursor.java | 7 - .../table/BroadcastSegmentIndexedTable.java | 84 +- .../join/table/FrameBasedIndexedTable.java | 4 +- .../druid/frame/TestArrayStorageAdapter.java | 23 +- .../processor/OutputChannelFactoryTest.java | 46 +- .../segment/FrameStorageAdapterTest.java | 22 +- .../frame/testutil/FrameSequenceBuilder.java | 94 +-- .../druid/frame/testutil/FrameTestUtil.java | 39 +- .../testutil/RowNumberUpdatingCursor.java | 7 - .../druid/frame/write/FrameWriterTest.java | 83 +- .../druid/query/CursorGranularizerTest.java | 255 ++++++ .../groupby/NestedDataGroupByQueryTest.java | 2 +- .../druid/query/lookup/LookupSegmentTest.java | 58 +- .../semantic/RowsAndColumnsDecoratorTest.java | 28 +- .../NestedDataTimeseriesQueryTest.java | 2 +- .../topn/TopNMetricSpecOptimizationsTest.java | 26 +- .../segment/AutoTypeColumnIndexerTest.java | 603 +++++++------- .../org/apache/druid/segment/ListCursor.java | 7 - .../NestedDataColumnIndexerV4Test.java | 516 ++++++------ .../QueryableIndexStorageAdapterTest.java | 14 +- .../segment/RowBasedStorageAdapterTest.java | 590 +++++++------- .../TombstoneSegmentStorageAdapterTest.java | 22 +- .../segment/UnnestStorageAdapterTest.java | 160 ++-- .../druid/segment/filter/BaseFilterTest.java | 153 ++-- .../IncrementalIndexStorageAdapterTest.java | 324 ++++---- .../HashJoinSegmentStorageAdapterTest.java | 158 ++-- .../segment/join/JoinFilterAnalyzerTest.java | 136 ++-- .../druid/segment/join/JoinTestHelper.java | 69 +- .../segment/join/PostJoinCursorTest.java | 55 +- .../NestedFieldColumnSelectorsTest.java | 10 +- .../selector/TestColumnValueSelector.java | 6 - .../virtual/ExpressionSelectorsTest.java | 47 +- .../ExpressionVectorSelectorsTest.java | 162 ++-- .../apache/druid/server/TestSegmentUtils.java | 26 +- .../org/apache/druid/cli/DumpSegment.java | 87 +-- 97 files changed, 3927 insertions(+), 3970 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/CursorGranularizer.java create mode 100644 processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java index 7c1d9994d0df..0c873bb4246c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -20,11 +20,9 @@ package org.apache.druid.benchmark; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.aggregation.BufferAggregator; @@ -36,6 +34,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.ValueType; @@ -60,7 +59,6 @@ import org.openjdk.jmh.infra.Blackhole; import java.nio.ByteBuffer; -import java.util.List; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -169,25 +167,22 @@ private double compute(final Function a .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - final List results = cursors - .map(cursor -> { - final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory()); - bufferAggregator.init(aggregationBuffer, 0); + final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory()); + bufferAggregator.init(aggregationBuffer, 0); - while (!cursor.isDone()) { - bufferAggregator.aggregate(aggregationBuffer, 0); - cursor.advance(); - } + while (!cursor.isDone()) { + bufferAggregator.aggregate(aggregationBuffer, 0); + cursor.advance(); + } - final Double dbl = (Double) bufferAggregator.get(aggregationBuffer, 0); - bufferAggregator.close(); - return dbl; - }) - .toList(); + final Double dbl = (Double) bufferAggregator.get(aggregationBuffer, 0); + bufferAggregator.close(); - return Iterables.getOnlyElement(results); + return dbl; + } } private static class NativeBufferAggregator implements BufferAggregator diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java index c9eaf06f8414..bd0ed02b24d9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -23,7 +23,6 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.expression.TestExprMacroTable; @@ -34,6 +33,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.ValueType; @@ -58,7 +58,6 @@ import org.openjdk.jmh.infra.Blackhole; import java.util.Arrays; -import java.util.List; import java.util.concurrent.TimeUnit; @State(Scope.Benchmark) @@ -151,17 +150,16 @@ public void expressionFilter(Blackhole blackhole) .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); - consumeString(cursor, selector, blackhole); - return null; - }) - .toList(); - blackhole.consume(results); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); + while (!cursor.isDone()) { + blackhole.consume(selector.getObject()); + cursor.advance(); + } + } } @Benchmark @@ -172,23 +170,13 @@ public void nativeFilter(Blackhole blackhole) .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); - consumeString(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); - } - - private void consumeString(final Cursor cursor, final ColumnValueSelector selector, final Blackhole blackhole) - { - while (!cursor.isDone()) { - blackhole.consume(selector.getLong()); - cursor.advance(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); + while (!cursor.isDone()) { + blackhole.consume(selector.getObject()); + cursor.advance(); + } } } } 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 b087b3fedd22..ea11639addfd 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -25,8 +25,10 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExpressionProcessing; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; @@ -36,9 +38,11 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -63,8 +67,8 @@ import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; + import java.util.BitSet; -import java.util.List; import java.util.concurrent.TimeUnit; @State(Scope.Benchmark) @@ -142,372 +146,314 @@ public void tearDown() throws Exception @Benchmark public void timeFloorUsingExpression(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "timestamp_floor(__time, 'PT1H')", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "timestamp_floor(__time, 'PT1H')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void timeFloorUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector( - new ExtractionDimensionSpec( - ColumnHolder.TIME_COLUMN_NAME, - "v", - new TimeFormatExtractionFn(null, null, null, Granularities.HOUR, true) - ) - ); - consumeDimension(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector( + new ExtractionDimensionSpec( + ColumnHolder.TIME_COLUMN_NAME, + "v", + new TimeFormatExtractionFn(null, null, null, Granularities.HOUR, true) + ) + ); + consumeDimension(cursor, selector, blackhole); + } } @Benchmark public void timeFloorUsingCursor(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.HOUR) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - long count = 0L; - while (!cursor.isDone()) { - count++; - cursor.advance(); - } - return count; - }) - .toList(); - - long count = 0L; - for (Long result : results) { - count += result; - } + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.HOUR) + .build(); + final StorageAdapter adapter = new QueryableIndexStorageAdapter(index); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + buildSpec.getGranularity(), + buildSpec.getInterval(), + buildSpec.isDescending() + ); + final Sequence results = + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + if (!granularizer.advanceToBucket(bucketInterval)) { + return 0L; + } + long count = 0L; + while (!cursor.isDone()) { + count++; + if (!granularizer.advanceCursorWithinBucket()) { + break; + } + } + return count; + }); + + long count = 0L; + for (Long result : results.toList()) { + count += result; + } - blackhole.consume(count); + blackhole.consume(count); + } } @Benchmark public void timeFormatUsingExpression(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "timestamp_format(__time, 'yyyy-MM-dd')", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector( - DefaultDimensionSpec.of("v") - ); - consumeDimension(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "timestamp_format(__time, 'yyyy-MM-dd')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector( + DefaultDimensionSpec.of("v") + ); + consumeDimension(cursor, selector, blackhole); + } } @Benchmark public void timeFormatUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector( - new ExtractionDimensionSpec( - ColumnHolder.TIME_COLUMN_NAME, - "v", - new TimeFormatExtractionFn("yyyy-MM-dd", null, null, null, false) - ) - ); - consumeDimension(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector( + new ExtractionDimensionSpec( + ColumnHolder.TIME_COLUMN_NAME, + "v", + new TimeFormatExtractionFn("yyyy-MM-dd", null, null, null, false) + ) + ); + consumeDimension(cursor, selector, blackhole); + } } @Benchmark public void strlenUsingExpressionAsLong(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "strlen(s)", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "strlen(s)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void strlenUsingExpressionAsString(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "strlen(s)", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("v", "v", ColumnType.STRING)); - - consumeDimension(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "strlen(s)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("v", "v", ColumnType.STRING)); + + consumeDimension(cursor, selector, blackhole); + } } @Benchmark public void strlenUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new ExtractionDimensionSpec("x", "v", StrlenExtractionFn.instance())); - - consumeDimension(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new ExtractionDimensionSpec("x", "v", StrlenExtractionFn.instance())); + + consumeDimension(cursor, selector, blackhole); + } } @Benchmark public void arithmeticOnLong(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "n + 1", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "n + 1", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void stringConcatAndCompareOnLong(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "concat(n, ' is my favorite number') == '3 is my favorite number'", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "concat(n, ' is my favorite number') == '3 is my favorite number'", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void caseSearched1(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "case_searched(s == 'asd' || isnull(s) || s == 'xxx', 1, s == 'foo' || s == 'bar', 2, 3)", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || s == 'xxx', 1, s == 'foo' || s == 'bar', 2, 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void caseSearched2(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "case_searched(s == 'asd' || isnull(s) || n == 1, 1, n == 2, 2, 3)", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || n == 1, 1, n == 2, 2, 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void caseSearched100(Blackhole blackhole) { - StringBuilder caseBranches = new StringBuilder(); for (int i = 0; i < 100; i++) { caseBranches.append( @@ -519,114 +465,96 @@ public void caseSearched100(Blackhole blackhole) ); } - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "case_searched(s == 'asd' || isnull(s) || n == 1, 1, " + caseBranches + " 3)", - ColumnType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || n == 1, 1, " + caseBranches + " 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void caseSearchedWithLookup(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - VirtualColumns.create( - new ExpressionVirtualColumn( - "v", - "case_searched(n == 1001, -1, " - + "lookup(s, 'lookyloo') == 'asd1', 1, " - + "lookup(s, 'lookyloo') == 'asd2', 2, " - + "lookup(s, 'lookyloo') == 'asd3', 3, " - + "lookup(s, 'lookyloo') == 'asd4', 4, " - + "lookup(s, 'lookyloo') == 'asd5', 5, " - + "-2)", - ColumnType.LONG, - LookupEnabledTestExprMacroTable.INSTANCE - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(n == 1001, -1, " + + "lookup(s, 'lookyloo') == 'asd1', 1, " + + "lookup(s, 'lookyloo') == 'asd2', 2, " + + "lookup(s, 'lookyloo') == 'asd3', 3, " + + "lookup(s, 'lookyloo') == 'asd4', 4, " + + "lookup(s, 'lookyloo') == 'asd5', 5, " + + "-2)", + ColumnType.LONG, + LookupEnabledTestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } @Benchmark public void caseSearchedWithLookup2(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker( - CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .setGranularity(Granularities.ALL) - .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 - ) - ) - ) - ) - .build() - ).makeCursors(); - - final List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - consumeLong(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(index.getDataInterval()) + .setGranularity(Granularities.ALL) + .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 + ) + ) + ) + ) + .build(); + + try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, blackhole); + } } 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 908f1c8f4cad..074caf85b861 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; @@ -157,7 +156,7 @@ public void scan(Blackhole blackhole) .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - final CursorMaker cursorMaker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec); + final CursorMaker cursorMaker = closer.register(new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)); if (vectorize) { VectorCursor cursor = cursorMaker.makeVectorCursor(); if (outputType.isNumeric()) { @@ -178,18 +177,32 @@ public void scan(Blackhole blackhole) closer.register(cursor); } } else { - final Sequence cursors = cursorMaker.makeCursors(); - int rowCount = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - int rows = 0; - while (!cursor.isDone()) { - blackhole.consume(selector.getObject()); - rows++; - cursor.advance(); - } - return rows; - }).accumulate(0, (acc, in) -> acc + in); + final Cursor cursor = cursorMaker.makeCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + int rowCount = 0; + if (outputType.isNumeric()) { + if (outputType.is(ExprType.DOUBLE)) { + while (!cursor.isDone()) { + blackhole.consume(selector.isNull()); + blackhole.consume(selector.getDouble()); + rowCount++; + cursor.advance(); + } + } else { + while (!cursor.isDone()) { + blackhole.consume(selector.isNull()); + blackhole.consume(selector.getLong()); + rowCount++; + cursor.advance(); + } + } + } else { + while (!cursor.isDone()) { + blackhole.consume(selector.getObject()); + rowCount++; + cursor.advance(); + } + } blackhole.consume(rowCount); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 041f71697f0e..3e4cb6bef234 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -26,8 +26,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; @@ -49,6 +47,7 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -234,8 +233,10 @@ private IncrementalIndex makeIncIndex() public void stringRead(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, null); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, null)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -244,9 +245,10 @@ public void stringRead(Blackhole blackhole) public void longRead(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, null); - - readCursorsLong(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, null)) { + final Cursor cursor = maker.makeCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -255,9 +257,10 @@ public void longRead(Blackhole blackhole) public void timeFilterNone(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, timeFilterNone); - - readCursorsLong(cursors, blackhole); + try (CursorMaker maker = makeCursorMaker(sa, timeFilterNone)) { + final Cursor cursor = maker.makeCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -266,9 +269,10 @@ public void timeFilterNone(Blackhole blackhole) public void timeFilterHalf(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, timeFilterHalf); - - readCursorsLong(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, timeFilterHalf)) { + final Cursor cursor = maker.makeCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -277,9 +281,10 @@ public void timeFilterHalf(Blackhole blackhole) public void timeFilterAll(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, timeFilterAll); - - readCursorsLong(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, timeFilterAll)) { + final Cursor cursor = maker.makeCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -290,8 +295,10 @@ public void readWithPreFilter(Blackhole blackhole) Filter filter = new SelectorFilter("dimSequential", "199"); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, filter); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, filter)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -302,8 +309,10 @@ public void readWithPostFilter(Blackhole blackhole) Filter filter = new NoBitmapSelectorFilter("dimSequential", "199"); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, filter); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, filter)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -314,8 +323,10 @@ public void readWithExFnPreFilter(Blackhole blackhole) Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, filter); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, filter)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -326,8 +337,10 @@ public void readWithExFnPostFilter(Blackhole blackhole) Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, filter); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, filter)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -343,8 +356,10 @@ public void readAndFilter(Blackhole blackhole) ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, andFilter); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, andFilter)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -357,8 +372,10 @@ public void readOrFilter(Blackhole blackhole) Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, orFilter); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, orFilter)) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -371,8 +388,10 @@ public void readOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosionExcept Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, Filters.toCnf(orFilter)); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, Filters.toCnf(orFilter))) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -408,8 +427,10 @@ public void readComplexOrFilter(Blackhole blackhole) ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, dimFilter3.toFilter()); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, dimFilter3.toFilter())) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -445,11 +466,13 @@ public void readComplexOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosio ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, Filters.toCnf(dimFilter3.toFilter())); - readCursors(cursors, blackhole); + try (final CursorMaker maker = makeCursorMaker(sa, Filters.toCnf(dimFilter3.toFilter()))) { + final Cursor cursor = maker.makeCursor(); + readCursor(cursor, blackhole); + } } - private Sequence makeCursors(StorageAdapter sa, Filter filter) + private CursorMaker makeCursorMaker(StorageAdapter sa, Filter filter) { return sa.asCursorMaker( CursorBuildSpec.builder() @@ -457,49 +480,33 @@ private Sequence makeCursors(StorageAdapter sa, Filter filter) .setInterval(schemaInfo.getDataInterval()) .setGranularity(Granularities.ALL) .build() - ).makeCursors(); + ); } - private void readCursors(Sequence cursors, Blackhole blackhole) + private void readCursor(Cursor cursor, Blackhole blackhole) { - final Sequence voids = Sequences.map( - cursors, - input -> { - List selectors = new ArrayList<>(); - selectors.add( - input.getColumnSelectorFactory().makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null)) - ); - while (!input.isDone()) { - for (DimensionSelector selector : selectors) { - IndexedInts row = selector.getRow(); - blackhole.consume(selector.lookupName(row.get(0))); - } - input.advance(); - } - return null; - } + List selectors = new ArrayList<>(); + selectors.add( + cursor.getColumnSelectorFactory().makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null)) ); - - blackhole.consume(voids.toList()); + while (!cursor.isDone()) { + for (DimensionSelector selector : selectors) { + IndexedInts row = selector.getRow(); + blackhole.consume(selector.lookupName(row.get(0))); + } + cursor.advance(); + } } - private void readCursorsLong(Sequence cursors, final Blackhole blackhole) + private void readCursorLong(Cursor cursor, final Blackhole blackhole) { - final Sequence voids = Sequences.map( - cursors, - input -> { - BaseLongColumnValueSelector selector = input.getColumnSelectorFactory() - .makeColumnValueSelector("sumLongSequential"); - while (!input.isDone()) { - long rowval = selector.getLong(); - blackhole.consume(rowval); - input.advance(); - } - return null; - } - ); - - blackhole.consume(voids.toList()); + BaseLongColumnValueSelector selector = cursor.getColumnSelectorFactory() + .makeColumnValueSelector("sumLongSequential"); + while (!cursor.isDone()) { + long rowval = selector.getLong(); + blackhole.consume(rowval); + cursor.advance(); + } } private static class NoBitmapSelectorFilter extends SelectorFilter diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java index 00126ac2b5ec..d1fbe9ee2e70 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; @@ -38,6 +37,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -217,9 +217,11 @@ public void tearDown() throws IOException @OutputTimeUnit(TimeUnit.MILLISECONDS) public void hashJoinCursorColumnValueSelectors(Blackhole blackhole) { - final Sequence cursors = makeCursors(); - int rowCount = processRowsValueSelector(blackhole, cursors, projectionColumns); - blackhole.consume(rowCount); + try (final CursorMaker maker = makeCursorMaker()) { + final Cursor cursor = maker.makeCursor(); + int rowCount = processRowsValueSelector(blackhole, cursor, projectionColumns); + blackhole.consume(rowCount); + } } @Benchmark @@ -227,14 +229,16 @@ public void hashJoinCursorColumnValueSelectors(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void hashJoinCursorDimensionSelectors(Blackhole blackhole) { - final Sequence cursors = makeCursors(); - int rowCount = processRowsDimensionSelectors(blackhole, cursors, projectionColumns); - blackhole.consume(rowCount); + try (final CursorMaker maker = makeCursorMaker()) { + final Cursor cursor = maker.makeCursor(); + int rowCount = processRowsDimensionSelectors(blackhole, cursor, projectionColumns); + blackhole.consume(rowCount); + } } - private Sequence makeCursors() + private CursorMaker makeCursorMaker() { - return hashJoinSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); + return hashJoinSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN); } @@ -307,106 +311,96 @@ public static QueryableIndexSegment makeQueryableIndexSegment(Closer closer, Str private static int processRowsDimensionSelectors( final Blackhole blackhole, - final Sequence cursors, + final Cursor cursor, final Set columns ) { if (columns.size() == 1) { - return processRowsSingleDimensionSelector(blackhole, cursors, Iterables.getOnlyElement(columns)); + return processRowsSingleDimensionSelector(blackhole, cursor, Iterables.getOnlyElement(columns)); } - return cursors.map( - cursor -> { - List selectors = columns.stream().map(column -> { - ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); - return factory.makeDimensionSelector(DefaultDimensionSpec.of(column)); - }).collect(Collectors.toList()); - - int rowCount = 0; - while (!cursor.isDone()) { - for (DimensionSelector selector : selectors) { - if (selector.getValueCardinality() < 0) { - final IndexedInts row = selector.getRow(); - final int sz = row.size(); - for (int i = 0; i < sz; i++) { - blackhole.consume(selector.lookupName(row.get(i))); - } - } else { - final IndexedInts row = selector.getRow(); - final int sz = row.size(); - for (int i = 0; i < sz; i++) { - blackhole.consume(row.get(i)); - } - } - } - - rowCount++; - cursor.advance(); + List selectors = columns.stream().map(column -> { + ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + return factory.makeDimensionSelector(DefaultDimensionSpec.of(column)); + }).collect(Collectors.toList()); + + int rowCount = 0; + while (!cursor.isDone()) { + for (DimensionSelector selector : selectors) { + if (selector.getValueCardinality() < 0) { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + for (int i = 0; i < sz; i++) { + blackhole.consume(selector.lookupName(row.get(i))); + } + } else { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + for (int i = 0; i < sz; i++) { + blackhole.consume(row.get(i)); } - return rowCount; - }).accumulate(0, (acc, in) -> acc + in); + } + } + + rowCount++; + cursor.advance(); + } + return rowCount; } private static int processRowsSingleDimensionSelector( final Blackhole blackhole, - final Sequence cursors, + final Cursor cursor, final String dimension ) { - return cursors.map( - cursor -> { - final DimensionSelector selector = cursor.getColumnSelectorFactory() - .makeDimensionSelector(DefaultDimensionSpec.of(dimension)); - - int rowCount = 0; - if (selector.getValueCardinality() < 0) { - String lastValue; - while (!cursor.isDone()) { - final IndexedInts row = selector.getRow(); - final int sz = row.size(); - for (int i = 0; i < sz; i++) { - lastValue = selector.lookupName(row.get(i)); - blackhole.consume(lastValue); - } - rowCount++; - cursor.advance(); - } - return rowCount; - } else { - int lastValue; - while (!cursor.isDone()) { - final IndexedInts row = selector.getRow(); - final int sz = row.size(); - for (int i = 0; i < sz; i++) { - lastValue = row.get(i); - blackhole.consume(lastValue); - } - rowCount++; - cursor.advance(); - } - return rowCount; - } + final DimensionSelector selector = cursor.getColumnSelectorFactory() + .makeDimensionSelector(DefaultDimensionSpec.of(dimension)); + + int rowCount = 0; + if (selector.getValueCardinality() < 0) { + String lastValue; + while (!cursor.isDone()) { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + for (int i = 0; i < sz; i++) { + lastValue = selector.lookupName(row.get(i)); + blackhole.consume(lastValue); + } + rowCount++; + cursor.advance(); + } + return rowCount; + } else { + int lastValue; + while (!cursor.isDone()) { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + for (int i = 0; i < sz; i++) { + lastValue = row.get(i); + blackhole.consume(lastValue); } - ).accumulate(0, (acc, in) -> acc + in); + rowCount++; + cursor.advance(); + } + return rowCount; + } } - private static int processRowsValueSelector(final Blackhole blackhole, final Sequence cursors, final Set columns) + private static int processRowsValueSelector(final Blackhole blackhole, final Cursor cursor, final Set columns) { - return cursors.map( - cursor -> { - ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); - - List selectors = - columns.stream().map(factory::makeColumnValueSelector).collect(Collectors.toList()); - int rowCount = 0; - while (!cursor.isDone()) { - for (BaseObjectColumnValueSelector selector : selectors) { - blackhole.consume(selector.getObject()); - } - - rowCount++; - cursor.advance(); - } - return rowCount; - }).accumulate(0, (acc, in) -> acc + in); + ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + + List selectors = + columns.stream().map(factory::makeColumnValueSelector).collect(Collectors.toList()); + int rowCount = 0; + while (!cursor.isDone()) { + for (BaseObjectColumnValueSelector selector : selectors) { + blackhole.consume(selector.getObject()); + } + + rowCount++; + cursor.advance(); + } + return rowCount; } } 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 7e8d4f20d519..d93e7da592a2 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.LookupExprMacro; @@ -36,6 +35,7 @@ import org.apache.druid.query.lookup.MapLookupExtractorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -335,38 +335,34 @@ public String getCanonicalLookupName(String lookupName) ); } - private static String getLastValue(final Sequence cursors, final String dimension) + private static String getLastValue(final Cursor cursor, final String dimension) { - return cursors.map( - cursor -> { - final DimensionSelector selector = cursor.getColumnSelectorFactory() - .makeDimensionSelector(DefaultDimensionSpec.of(dimension)); - - if (selector.getValueCardinality() < 0) { - String lastValue = null; - while (!cursor.isDone()) { - final IndexedInts row = selector.getRow(); - final int sz = row.size(); - for (int i = 0; i < sz; i++) { - lastValue = selector.lookupName(row.get(i)); - } - cursor.advance(); - } - return lastValue; - } else { - int lastValue = -1; - while (!cursor.isDone()) { - final IndexedInts row = selector.getRow(); - final int sz = row.size(); - for (int i = 0; i < sz; i++) { - lastValue = row.get(i); - } - cursor.advance(); - } - return selector.lookupName(lastValue); - } + final DimensionSelector selector = cursor.getColumnSelectorFactory() + .makeDimensionSelector(DefaultDimensionSpec.of(dimension)); + + if (selector.getValueCardinality() < 0) { + String lastValue = null; + while (!cursor.isDone()) { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + for (int i = 0; i < sz; i++) { + lastValue = selector.lookupName(row.get(i)); } - ).accumulate(null, (acc, in) -> in); + cursor.advance(); + } + return lastValue; + } else { + int lastValue = -1; + while (!cursor.isDone()) { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + for (int i = 0; i < sz; i++) { + lastValue = row.get(i); + } + cursor.advance(); + } + return selector.lookupName(lastValue); + } } @Benchmark @@ -374,11 +370,10 @@ private static String getLastValue(final Sequence cursors, final String @OutputTimeUnit(TimeUnit.MILLISECONDS) public void baseSegment(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "countryIsoCode")); + try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "countryIsoCode")); + } } @Benchmark @@ -387,16 +382,14 @@ public void baseSegment(Blackhole blackhole) public void baseSegmentWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(); - final Sequence cursors = baseSegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setFilter(filter) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "countryIsoCode")); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build(); + try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "countryIsoCode")); + } } @Benchmark @@ -404,11 +397,10 @@ public void baseSegmentWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupStringKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.v")); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -417,16 +409,14 @@ public void joinLookupStringKey(Blackhole blackhole) public void joinLookupStringKeyWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter(); - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setFilter(filter) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.v")); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build(); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -434,11 +424,10 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupLongKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.v")); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -447,16 +436,14 @@ public void joinLookupLongKey(Blackhole blackhole) public void joinLookupLongKeyWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter(); - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setFilter(filter) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.v")); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build(); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -464,11 +451,10 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableLongKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -477,16 +463,14 @@ public void joinIndexedTableLongKey(Blackhole blackhole) public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter(); - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setFilter(filter) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build(); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -494,11 +478,10 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableStringKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -507,16 +490,14 @@ public void joinIndexedTableStringKey(Blackhole blackhole) public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter(); - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setFilter(filter) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .build(); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -524,16 +505,14 @@ public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void lookupVirtualColumnStringKey(Blackhole blackhole) { - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setVirtualColumns(lookupVirtualColumns) - .setGranularity(Granularities.ALL) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); + } } @Benchmark @@ -542,17 +521,15 @@ public void lookupVirtualColumnStringKey(Blackhole blackhole) public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_CODE_TO_NAME, "Canada", null).toFilter(); - final Sequence cursors = hashJoinLookupStringKeySegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setFilter(filter) - .setVirtualColumns(lookupVirtualColumns) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setFilter(filter) + .setVirtualColumns(lookupVirtualColumns) + .build(); + try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); + } } @Benchmark @@ -560,16 +537,14 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void lookupVirtualColumnLongKey(Blackhole blackhole) { - final Sequence cursors = baseSegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setVirtualColumns(lookupVirtualColumns) - .setGranularity(Granularities.ALL) - .build() - ) - .makeCursors(); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + } } @Benchmark @@ -578,17 +553,14 @@ public void lookupVirtualColumnLongKey(Blackhole blackhole) public void lookupVirtualColumnLongKeyWithFilter(Blackhole blackhole) { final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_NUMBER_TO_NAME, "Canada", null).toFilter(); - final Sequence cursors = baseSegment.asStorageAdapter() - .asCursorMaker( - CursorBuildSpec.builder() - .setGranularity(Granularities.ALL) - .setVirtualColumns(lookupVirtualColumns) - .setFilter(filter) - .build() - ) - .makeCursors(); - - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .setVirtualColumns(lookupVirtualColumns) + .setFilter(filter) + .build(); + try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + } } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 9f765f54d7ff..96b6feaa4c6d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -21,7 +21,6 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; @@ -37,6 +36,7 @@ import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.generator.DataGenerator; @@ -149,22 +149,23 @@ private IncrementalIndex makeIncIndex() public void read(Blackhole blackhole) { IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - Sequence cursors = makeCursors(sa, null); - Cursor cursor = cursors.limit(1).toList().get(0); - - List selectors = new ArrayList<>(); - selectors.add(makeDimensionSelector(cursor, "dimSequential")); - selectors.add(makeDimensionSelector(cursor, "dimZipf")); - selectors.add(makeDimensionSelector(cursor, "dimUniform")); - selectors.add(makeDimensionSelector(cursor, "dimSequentialHalfNull")); - - cursor.reset(); - while (!cursor.isDone()) { - for (DimensionSelector selector : selectors) { - IndexedInts row = selector.getRow(); - blackhole.consume(selector.lookupName(row.get(0))); + try (final CursorMaker maker = makeCursor(sa, null)) { + Cursor cursor = maker.makeCursor(); + + List selectors = new ArrayList<>(); + selectors.add(makeDimensionSelector(cursor, "dimSequential")); + selectors.add(makeDimensionSelector(cursor, "dimZipf")); + selectors.add(makeDimensionSelector(cursor, "dimUniform")); + selectors.add(makeDimensionSelector(cursor, "dimSequentialHalfNull")); + + cursor.reset(); + while (!cursor.isDone()) { + for (DimensionSelector selector : selectors) { + IndexedInts row = selector.getRow(); + blackhole.consume(selector.lookupName(row.get(0))); + } + cursor.advance(); } - cursor.advance(); } } @@ -184,26 +185,27 @@ public void readWithFilters(Blackhole blackhole) ); IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - Sequence cursors = makeCursors(sa, filter); - Cursor cursor = cursors.limit(1).toList().get(0); - - List selectors = new ArrayList<>(); - selectors.add(makeDimensionSelector(cursor, "dimSequential")); - selectors.add(makeDimensionSelector(cursor, "dimZipf")); - selectors.add(makeDimensionSelector(cursor, "dimUniform")); - selectors.add(makeDimensionSelector(cursor, "dimSequentialHalfNull")); - - cursor.reset(); - while (!cursor.isDone()) { - for (DimensionSelector selector : selectors) { - IndexedInts row = selector.getRow(); - blackhole.consume(selector.lookupName(row.get(0))); + try (final CursorMaker maker = makeCursor(sa, filter)) { + Cursor cursor = maker.makeCursor(); + + List selectors = new ArrayList<>(); + selectors.add(makeDimensionSelector(cursor, "dimSequential")); + selectors.add(makeDimensionSelector(cursor, "dimZipf")); + selectors.add(makeDimensionSelector(cursor, "dimUniform")); + selectors.add(makeDimensionSelector(cursor, "dimSequentialHalfNull")); + + cursor.reset(); + while (!cursor.isDone()) { + for (DimensionSelector selector : selectors) { + IndexedInts row = selector.getRow(); + blackhole.consume(selector.lookupName(row.get(0))); + } + cursor.advance(); } - cursor.advance(); } } - private Sequence makeCursors(IncrementalIndexStorageAdapter sa, DimFilter filter) + private CursorMaker makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter) { CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() .setGranularity(Granularities.ALL) @@ -211,7 +213,7 @@ private Sequence makeCursors(IncrementalIndexStorageAdapter sa, DimFilte if (filter != null) { builder.setFilter(filter.toFilter()); } - return sa.asCursorMaker(builder.build()).makeCursors(); + return sa.asCursorMaker(builder.build()); } private static DimensionSelector makeDimensionSelector(Cursor cursor, String name) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index 0c7b1db5ee50..911d13579583 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java @@ -38,16 +38,16 @@ import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.exec.MSQTasks; import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.util.SequenceUtils; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -181,42 +181,38 @@ private void addFrame(final Frame frame) // Reuse input row to avoid redoing allocations. final MSQInputRow inputRow = new MSQInputRow(); - final Sequence cursorSequence = - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - SequenceUtils.forEach( - cursorSequence, - cursor -> { - final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - //noinspection rawtypes - @SuppressWarnings("rawtypes") - final List selectors = - frameReader.signature() - .getColumnNames() - .stream() - .map(columnSelectorFactory::makeColumnValueSelector) - .collect(Collectors.toList()); - - while (!cursor.isDone()) { - for (int j = 0; j < signature.size(); j++) { - inputRow.getBackingArray()[j] = selectors.get(j).getObject(); - } - - try { - rowsWritten++; - appenderator.add(segmentIdWithShardSpec, inputRow, null); - segmentGenerationProgressCounter.incrementRowsProcessed(1); - } - catch (Exception e) { - throw new RuntimeException(e); - } - - cursor.advance(); - } + final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + //noinspection rawtypes + @SuppressWarnings("rawtypes") + final List selectors = + frameReader.signature() + .getColumnNames() + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + while (!cursor.isDone()) { + for (int j = 0; j < signature.size(); j++) { + inputRow.getBackingArray()[j] = selectors.get(j).getObject(); + } + + try { + rowsWritten++; + appenderator.add(segmentIdWithShardSpec, inputRow, null); + segmentGenerationProgressCounter.incrementRowsProcessed(1); + } + catch (Exception e) { + throw new RuntimeException(e); } - ); + + cursor.advance(); + } + } } private class MSQInputRow implements InputRow diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java index d8615f7ccd55..f54f8336f97d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java @@ -33,14 +33,14 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.exec.ResultsContext; -import org.apache.druid.msq.util.SequenceUtils; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -151,57 +151,54 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx private void exportFrame(final Frame frame) { - final Sequence cursorSequence = - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors(); - - SequenceUtils.forEach( - cursorSequence, - cursor -> { - try { - final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - //noinspection rawtypes - final List selectors = - frameReader.signature() - .getColumnNames() - .stream() - .map(columnSelectorFactory::makeColumnValueSelector) - .collect(Collectors.toList()); - - while (!cursor.isDone()) { - exportWriter.writeRowStart(); - for (int j = 0; j < exportRowSignature.size(); j++) { - String columnName = exportRowSignature.getColumnName(j); - BaseObjectColumnValueSelector selector = selectors.get(outputColumnNameToFrameColumnNumberMap.getInt(columnName)); - if (resultsContext == null) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Received null resultsContext from the controller. This is due to a version mismatch between the controller and the worker. Please ensure that the worker and the controller are on the same version before retrying the query."); - } - exportWriter.writeRowField( - columnName, - SqlResults.coerce( - jsonMapper, - resultsContext.getSqlResultsContext(), - selector.getObject(), - resultsContext.getSqlTypeNames().get(j), - columnName - ) - ); - } - channelCounter.incrementRowCount(partitionNum); - exportWriter.writeRowEnd(); - cursor.advance(); - } - } - catch (IOException e) { - throw DruidException.forPersona(DruidException.Persona.USER) + final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + //noinspection rawtypes + final List selectors = + frameReader.signature() + .getColumnNames() + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + while (!cursor.isDone()) { + exportWriter.writeRowStart(); + for (int j = 0; j < exportRowSignature.size(); j++) { + String columnName = exportRowSignature.getColumnName(j); + BaseObjectColumnValueSelector selector = selectors.get(outputColumnNameToFrameColumnNumberMap.getInt(columnName)); + if (resultsContext == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(e, "Exception occurred while writing file to the export location [%s].", exportFilePath); + .build("Received null resultsContext from the controller. This is due to a version mismatch between the controller and the worker. Please ensure that the worker and the controller are on the same version before retrying the query."); } + exportWriter.writeRowField( + columnName, + SqlResults.coerce( + jsonMapper, + resultsContext.getSqlResultsContext(), + selector.getObject(), + resultsContext.getSqlTypeNames().get(j), + columnName + ) + ); } - ); + channelCounter.incrementRowCount(partitionNum); + exportWriter.writeRowEnd(); + cursor.advance(); + } + } + catch (IOException e) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build( + e, + "Exception occurred while writing file to the export location [%s].", + exportFilePath + ); + } } private void createExportWriter() throws IOException diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 13be489b35da..095604e69dc4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.querykit.scan; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -41,7 +40,6 @@ import org.apache.druid.frame.write.InvalidFieldException; import org.apache.druid.frame.write.InvalidNullByteException; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.guava.Sequence; @@ -63,10 +61,9 @@ import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanResultValue; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; @@ -248,21 +245,23 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment if (cursor == null) { final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); - final Yielder cursorYielder = Yielders.each( - makeCursors( - query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), - mapSegment(segmentHolder.get()).asStorageAdapter() - ) - ); + final StorageAdapter adapter = mapSegment(segmentHolder.get()).asStorageAdapter(); + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } - if (cursorYielder.isDone()) { + final CursorMaker maker = closer.register(adapter.asCursorMaker(query.asCursorBuildSpec(null))); + final Cursor cursor = maker.makeCursor(); + + if (cursor == null) { // No cursors! - cursorYielder.close(); + maker.close(); return ReturnOrAwait.returnObject(Unit.instance()); } else { - final long rowsFlushed = setNextCursor(cursorYielder.get(), segmentHolder.get()); + final long rowsFlushed = setNextCursor(cursor, segmentHolder.get()); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment - closer.register(cursorYielder); } } @@ -290,15 +289,22 @@ protected ReturnOrAwait runWithInputChannel( final Frame frame = inputChannel.read(); final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("scan")); - final long rowsFlushed = setNextCursor( - Iterables.getOnlyElement( - makeCursors( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), - mapSegment(frameSegment).asStorageAdapter() - ).toList() - ), - frameSegment - ); + final StorageAdapter adapter = mapSegment(frameSegment).asStorageAdapter(); + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + + final CursorMaker maker = closer.register(adapter.asCursorMaker(query.asCursorBuildSpec(null))); + final Cursor cursor = maker.makeCursor(); + + if (cursor == null) { + // no cursor + maker.close(); + return ReturnOrAwait.returnObject(Unit.instance()); + } + final long rowsFlushed = setNextCursor(cursor, frameSegment); if (rowsFlushed > 0) { return ReturnOrAwait.runAgain(); @@ -426,15 +432,4 @@ private ColumnSelectorFactory wrapColumnSelectorFactoryIfNeeded(final ColumnSele } return baseColumnSelectorFactory; } - - private static Sequence makeCursors(final ScanQuery query, final StorageAdapter adapter) - { - if (adapter == null) { - throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." - ); - } - - return adapter.asCursorMaker(query.asCursorBuildSpec(null)).makeCursors(); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index d16106758170..7dcaf9a8b502 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -49,6 +49,7 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -129,7 +130,8 @@ protected CloseableIterator> intermediateRowIterator() throw .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = storageAdapter.getAdapter().asCursorMaker(cursorBuildSpec).makeCursors(); + final CursorMaker maker = storageAdapter.getAdapter().asCursorMaker(cursorBuildSpec); + final Cursor cursor = maker.makeCursor(); // Retain order of columns from the original segments. Useful for preserving dimension order if we're in // schemaless mode. @@ -140,12 +142,7 @@ protected CloseableIterator> intermediateRowIterator() throw ) ); - final Sequence> sequence = Sequences.concat( - Sequences.map( - cursors, - cursor -> cursorToSequence(cursor, columnsToRead) - ) - ); + final Sequence> sequence = cursorToSequence(cursor, columnsToRead).withBaggage(maker); return makeCloseableIteratorFromSequenceAndSegmentFile(sequence, segmentFile); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 0dff7cd4a87c..fbfc2401a162 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -61,7 +61,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -73,6 +72,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; @@ -1706,18 +1706,19 @@ public void testRunWithSpatialDimensions() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - final Sequence cursorSequence = adapter.getAdapter() - .asCursorMaker(buildSpec) - .makeCursors(); + try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); Assert.assertTrue(factory.getColumnCapabilities("spatial").hasSpatialIndexes()); while (!cursor.isDone()) { final ColumnValueSelector selector1 = factory.makeColumnValueSelector("ts"); final DimensionSelector selector2 = factory.makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); - final DimensionSelector selector3 = factory.makeDimensionSelector(new DefaultDimensionSpec("spatial", "spatial")); + final DimensionSelector selector3 = factory.makeDimensionSelector(new DefaultDimensionSpec( + "spatial", + "spatial" + )); final DimensionSelector selector4 = factory.makeDimensionSelector(new DefaultDimensionSpec("val", "val")); @@ -1733,11 +1734,9 @@ public void testRunWithSpatialDimensions() throws Exception cursor.advance(); } - - return accumulated; - }); + } + Assert.assertEquals(spatialrows, rowsFromSegment); } - Assert.assertEquals(spatialrows, rowsFromSegment); } @Test @@ -1839,11 +1838,8 @@ public void testRunWithAutoCastDimensions() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - final Sequence cursorSequence = adapter.getAdapter() - .asCursorMaker(buildSpec) - .makeCursors(); - - cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { + try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); Assert.assertEquals(ColumnType.STRING, factory.getColumnCapabilities("ts").toColumnType()); @@ -1871,11 +1867,9 @@ public void testRunWithAutoCastDimensions() throws Exception cursor.advance(); } - - return accumulated; - }); + } + Assert.assertEquals(rows, rowsFromSegment); } - Assert.assertEquals(rows, rowsFromSegment); } private Pair runIndexTask() throws Exception @@ -2072,11 +2066,9 @@ private List getCSVFormatRowsFromSegments(List segments) th .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - final Sequence cursorSequence = adapter.getAdapter() - .asCursorMaker(buildSpec) - .makeCursors(); + try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { cursor.reset(); while (!cursor.isDone()) { final DimensionSelector selector1 = cursor.getColumnSelectorFactory() @@ -2104,9 +2096,7 @@ private List getCSVFormatRowsFromSegments(List segments) th cursor.advance(); } - - return accumulated; - }); + } } return rowsFromSegment; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index f827271f6f4d..c01ba7ddea9f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -60,7 +60,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -69,6 +68,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; @@ -544,66 +544,65 @@ public void testTransformSpec() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - final Sequence cursorSequence = adapter.getAdapter() - .asCursorMaker(buildSpec) - .makeCursors(); - final List> transforms = cursorSequence - .map(cursor -> { - final DimensionSelector selector1 = cursor.getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("dimt", "dimt")); - final DimensionSelector selector2 = cursor.getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec( - "dimtarray1", - "dimtarray1" - )); - final DimensionSelector selector3 = cursor.getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec( - "dimtarray2", - "dimtarray2" - )); - final DimensionSelector selector4 = cursor.getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec( - "dimtnum_array", - "dimtnum_array" - )); - - - Map row = new HashMap<>(); - row.put("dimt", selector1.defaultGetObject()); - row.put("dimtarray1", selector2.defaultGetObject()); - row.put("dimtarray2", selector3.defaultGetObject()); - row.put("dimtnum_array", selector4.defaultGetObject()); - cursor.advance(); - return row; - }) - .toList(); - Assert.assertEquals(1, transforms.size()); - Assert.assertEquals("bb", transforms.get(0).get("dimt")); - Assert.assertEquals(ImmutableList.of("b", "b"), transforms.get(0).get("dimtarray1")); - Assert.assertEquals(ImmutableList.of("anotherfoo", "arrayfoo"), transforms.get(0).get("dimtarray2")); - Assert.assertEquals(ImmutableList.of("6.0", "7.0"), transforms.get(0).get("dimtnum_array")); - - Assert.assertEquals(DATASOURCE, segments.get(0).getDataSource()); - Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(0).getInterval()); - Assert.assertEquals(NumberedShardSpec.class, segments.get(0).getShardSpec().getClass()); - Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); - - verifySchemaAndAggFactory( - segmentWithSchemas, - RowSignature.builder() - .add("__time", ColumnType.LONG) - .add("ts", ColumnType.STRING) - .add("dim", ColumnType.STRING) - .add("dim_array", ColumnType.STRING) - .add("dim_num_array", ColumnType.STRING) - .add("dimt", ColumnType.STRING) - .add("dimtarray1", ColumnType.STRING) - .add("dimtarray2", ColumnType.STRING) - .add("dimtnum_array", ColumnType.STRING) - .add("val", ColumnType.LONG) - .build(), - Collections.singletonMap("val", new LongSumAggregatorFactory("val", "val")) - ); + try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final List> transforms = new ArrayList<>(); + + final DimensionSelector selector1 = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("dimt", "dimt")); + final DimensionSelector selector2 = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec( + "dimtarray1", + "dimtarray1" + )); + final DimensionSelector selector3 = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec( + "dimtarray2", + "dimtarray2" + )); + final DimensionSelector selector4 = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec( + "dimtnum_array", + "dimtnum_array" + )); + + + Map row = new HashMap<>(); + row.put("dimt", selector1.defaultGetObject()); + row.put("dimtarray1", selector2.defaultGetObject()); + row.put("dimtarray2", selector3.defaultGetObject()); + row.put("dimtnum_array", selector4.defaultGetObject()); + transforms.add(row); + cursor.advance(); + + Assert.assertEquals(1, transforms.size()); + Assert.assertEquals("bb", transforms.get(0).get("dimt")); + Assert.assertEquals(ImmutableList.of("b", "b"), transforms.get(0).get("dimtarray1")); + Assert.assertEquals(ImmutableList.of("anotherfoo", "arrayfoo"), transforms.get(0).get("dimtarray2")); + Assert.assertEquals(ImmutableList.of("6.0", "7.0"), transforms.get(0).get("dimtnum_array")); + + Assert.assertEquals(DATASOURCE, segments.get(0).getDataSource()); + Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(0).getInterval()); + Assert.assertEquals(NumberedShardSpec.class, segments.get(0).getShardSpec().getClass()); + Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); + + verifySchemaAndAggFactory( + segmentWithSchemas, + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("ts", ColumnType.STRING) + .add("dim", ColumnType.STRING) + .add("dim_array", ColumnType.STRING) + .add("dim_num_array", ColumnType.STRING) + .add("dimt", ColumnType.STRING) + .add("dimtarray1", ColumnType.STRING) + .add("dimtarray2", ColumnType.STRING) + .add("dimtnum_array", ColumnType.STRING) + .add("val", ColumnType.LONG) + .build(), + Collections.singletonMap("val", new LongSumAggregatorFactory("val", "val")) + ); + } } @Test @@ -779,26 +778,24 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - final Sequence cursorSequence = adapter.getAdapter() - .asCursorMaker(buildSpec) - .makeCursors(); - final List hashes = cursorSequence - .map(cursor -> { - final DimensionSelector selector = cursor.getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); - final int hash = HashPartitionFunction.MURMUR3_32_ABS.hash( - HashBasedNumberedShardSpec.serializeGroupKey( - jsonMapper, - Collections.singletonList(selector.getObject()) - ), - hashBasedNumberedShardSpec.getNumBuckets() - ); - cursor.advance(); - return hash; - }) - .toList(); + try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + final List hashes = new ArrayList<>(); + final DimensionSelector selector = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); + final int hash = HashPartitionFunction.MURMUR3_32_ABS.hash( + HashBasedNumberedShardSpec.serializeGroupKey( + jsonMapper, + Collections.singletonList(selector.getObject()) + ), + hashBasedNumberedShardSpec.getNumBuckets() + ); + hashes.add(hash); + cursor.advance(); - Assert.assertTrue(hashes.stream().allMatch(h -> h.intValue() == hashes.get(0))); + // todo (clint): wtf does this mean + Assert.assertTrue(hashes.stream().allMatch(h -> h.intValue() == hashes.get(0))); + } } } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index 7ee48d00bde4..f789f79e57d6 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -33,7 +33,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; @@ -122,9 +121,7 @@ public static FrameCursor makeCursor( .build(); // Safe to never close the Sequence that the FrameCursor comes from, because it does not need to be closed. // Refer to FrameStorageAdapter#makeCursors. - return (FrameCursor) Yielders.each( - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec).makeCursors() - ).get(); + return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec).makeCursor(); } /** diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 3d4160a7abb4..35672c0338d0 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -19,12 +19,10 @@ package org.apache.druid.frame.segment; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.BaseQuery; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; -import org.joda.time.DateTime; /** * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorFactory} @@ -53,12 +51,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return DateTimes.MIN; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java index df9d3bf5720d..b82529de506b 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java @@ -21,23 +21,16 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; -import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -153,37 +146,4 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) { return cursorFactory.asCursorMaker(spec); } - - @Override - public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) - { - return cursorFactory.canVectorize(filter, virtualColumns, descending); - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - - @Nullable - @Override - public VectorCursor makeVectorCursor( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); - } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java index 3714622740c4..e6aaf9d6fe6c 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java @@ -28,11 +28,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnCache; @@ -52,10 +48,10 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.vector.VectorOffset; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; /** @@ -85,6 +81,7 @@ public FrameCursorFactory( @Override public CursorMaker asCursorMaker(CursorBuildSpec spec) { + final Closer closer = Closer.create(); return new CursorMaker() { @Override @@ -96,12 +93,11 @@ public boolean canVectorize() } @Override - public Sequence makeCursors() + public Cursor makeCursor() { final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); if (Granularities.ALL.equals(spec.getGranularity())) { - final Closer closer = Closer.create(); final Cursor cursor = makeGranularityAllCursor( new ColumnCache(index, closer), frame.numRows(), @@ -111,7 +107,7 @@ public Sequence makeCursors() spec.isDescending() ); - return Sequences.simple(Collections.singletonList(cursor)).withBaggage(closer); + return cursor; } else { // Not currently needed for the intended use cases of frame-based cursors. throw new UOE("Granularity [%s] not supported", spec.getGranularity()); @@ -126,7 +122,6 @@ public VectorCursor makeVectorCursor() throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); } - final Closer closer = Closer.create(); final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); final VectorOffset baseOffset = new NoFilterVectorOffset( @@ -163,42 +158,13 @@ public VectorCursor makeVectorCursor() return new FrameVectorCursor(filteredOffset, filteredColumnSelectorFactory, closer); } } - }; - } - - @Override - public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) - { - return (filter == null || filter.canVectorizeMatcher(signature)) - && virtualColumns.canVectorize(signature) - && !descending; - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Nullable - @Override - public VectorCursor makeVectorCursor( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; } private static Cursor makeGranularityAllCursor( diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java index d5977ba6d478..a2005d91405a 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java @@ -28,24 +28,15 @@ import org.apache.druid.frame.segment.FrameFilteredOffset; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; -import org.apache.druid.segment.VirtualColumns; -import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; /** @@ -108,20 +99,7 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) // Note: if anything closeable is ever added to this Sequence, make sure to update FrameProcessors.makeCursor. // Currently, it assumes that closing the Sequence does nothing. - return Sequences.simple(Collections.singletonList(cursor)); + return cursor; }; } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } } diff --git a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java new file mode 100644 index 000000000000..ddaebf8aec38 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java @@ -0,0 +1,190 @@ +/* + * 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.query; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnHolder; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +public class CursorGranularizer +{ + @Nullable + public static CursorGranularizer create( + final StorageAdapter storageAdapter, + final Cursor cursor, + final Granularity granularity, + final Interval queryInterval, + final boolean descending + ) + { + final DateTime minTime = storageAdapter.getMinTime(); + final DateTime maxTime = storageAdapter.getMaxTime(); + + final Interval storageAdapterInterval = new Interval(minTime, granularity.bucketEnd(maxTime)); + final Interval clippedQueryInterval = queryInterval.overlap(storageAdapterInterval); + + if (clippedQueryInterval == null) { + return null; + } + + Iterable bucketIterable = granularity.getIterable(clippedQueryInterval); + if (descending) { + bucketIterable = Lists.reverse(ImmutableList.copyOf(bucketIterable)); + } + final Interval firstBucket = granularity.bucket(clippedQueryInterval.getStart()); + + final ColumnValueSelector timeSelector; + if (firstBucket.contains(clippedQueryInterval)) { + // Only one bucket, no need to read the time column. + assert Iterables.size(bucketIterable) == 1; + timeSelector = null; + } else { + // Multiple buckets, need to read the time column to know when we move from one to the next. + timeSelector = cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); + } + + return new CursorGranularizer(cursor, bucketIterable, timeSelector, descending); + } + + // And a cursor that has been made from it. + private final Cursor cursor; + + // Iterable that iterates over time buckets. + private final Iterable bucketIterable; + + // Vector selector for the "__time" column. + @Nullable + private final ColumnValueSelector timeSelector; + private final boolean descending; + + private long currentBucketStart; + private long currentBucketEnd; + + private CursorGranularizer( + Cursor cursor, + Iterable bucketIterable, + @Nullable ColumnValueSelector timeSelector, + boolean descending + ) + { + this.cursor = cursor; + this.bucketIterable = bucketIterable; + this.timeSelector = timeSelector; + this.descending = descending; + } + + public Iterable getBucketIterable() + { + return bucketIterable; + } + + public DateTime getBucketStart() + { + return DateTimes.utc(currentBucketStart); + } + + public boolean advanceToBucket(final Interval bucketInterval) + { + if (cursor.isDone()) { + return false; + } + currentBucketStart = bucketInterval.getStartMillis(); + currentBucketEnd = bucketInterval.getEndMillis(); + if (timeSelector == null) { + return true; + } + long currentTime = timeSelector.getLong(); + if (descending) { + while (currentTime >= currentBucketEnd && !cursor.isDone()) { + cursor.advance(); + currentTime = timeSelector.getLong(); + } + } else { + while (currentTime < currentBucketStart && !cursor.isDone()) { + cursor.advance(); + currentTime = timeSelector.getLong(); + } + } + + return currentBucketStart <= currentTime && currentTime < currentBucketEnd; + } + + public void resetBucket() + { + cursor.reset(); + if (timeSelector == null) { + return; + } + long currentTime = timeSelector.getLong(); + if (descending) { + while (currentTime >= currentBucketEnd && !cursor.isDone()) { + cursor.advance(); + currentTime = timeSelector.getLong(); + } + } else { + while (currentTime < currentBucketStart && !cursor.isDone()) { + cursor.advance(); + currentTime = timeSelector.getLong(); + } + } + } + + public boolean advanceCursorWithinBucket() + { + if (cursor.isDone()) { + return false; + } + cursor.advance(); + return currentOffsetWithinBucket(); + } + + /** + */ + public boolean advanceCursorWithinBucketUninterruptedly() + { + if (cursor.isDone()) { + return false; + } + cursor.advanceUninterruptibly(); + return currentOffsetWithinBucket(); + } + + public boolean currentOffsetWithinBucket() + { + if (cursor.isDone()) { + return false; + } + if (timeSelector == null) { + return true; + } + final long currentTime = timeSelector.getLong(); + return currentBucketStart <= currentTime && currentTime < currentBucketEnd; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index 4edbd9170689..c75e44025722 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -80,17 +80,16 @@ public RowSignature getRowSignature() public Sequence getRowsAsSequence() { - final Sequence cursorSequence = Sequences.simple(frames) - .flatMap( + .map( frameSignaturePair -> { Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( CursorBuildSpec.FULL_SCAN - ).makeCursors(); + ).makeCursor(); } ); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index d2c6c4b32682..58599cda1c0b 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -47,6 +47,7 @@ import org.apache.druid.java.util.common.guava.LazySequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Query; import org.apache.druid.query.QueryCapacityExceededException; @@ -487,6 +488,8 @@ public Sequence process( final ResourceHolder bufferHolder = bufferPool.take(); + Closer closer = Closer.create(); + closer.register(bufferHolder); try { final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded( query.context().getString(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP) @@ -499,15 +502,12 @@ public Sequence process( // group by specific vectorization check: final CursorBuildSpec buildSpec = query.asCursorBuildSpec(groupByQueryMetrics); - final CursorMaker cursorMaker = storageAdapter.asCursorMaker(buildSpec); + final CursorMaker cursorMaker = closer.register(storageAdapter.asCursorMaker(buildSpec)); final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); final boolean canVectorize = cursorMaker.canVectorize() && VectorGroupByEngine.canVectorizeDimensions(inspector, query.getDimensions()); - final boolean shouldVectorize = query.context().getVectorize().shouldVectorize( - canVectorize, - cursorMaker::cleanup - ); + final boolean shouldVectorize = query.context().getVectorize().shouldVectorize(canVectorize); final Sequence result; if (shouldVectorize) { result = VectorGroupByEngine.process( @@ -525,6 +525,7 @@ public Sequence process( query, storageAdapter, cursorMaker, + buildSpec, bufferHolder.get(), fudgeTimestamp, querySpecificConfig, @@ -532,10 +533,10 @@ public Sequence process( ); } - return result.withBaggage(bufferHolder); + return result.withBaggage(closer); } catch (Throwable e) { - bufferHolder.close(); + CloseableUtils.closeAndWrapExceptions(closer); throw e; } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index b54f56c71f73..0191a0f4fa8c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -26,10 +26,13 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -47,6 +50,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; @@ -94,80 +98,143 @@ public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, final CursorMaker cursorMaker, + final CursorBuildSpec buildSpec, final ByteBuffer processingBuffer, @Nullable final DateTime fudgeTimestamp, final GroupByQueryConfig querySpecificConfig, final DruidProcessingConfig processingConfig ) { - final Sequence cursors = cursorMaker.makeCursors(); + final Cursor cursor = cursorMaker.makeCursor(); + if (cursor == null) { + return Sequences.empty(); + } + final CursorGranularizer granularizer = CursorGranularizer.create( + storageAdapter, + cursor, + buildSpec.getGranularity(), + buildSpec.getInterval(), + buildSpec.isDescending() + ); + if (granularizer == null) { + return Sequences.empty(); + } + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final ColumnSelectorPlus[] selectorPlus = DimensionHandlerUtils + .createColumnSelectorPluses( + STRATEGY_FACTORY, + query.getDimensions(), + columnSelectorFactory + ); + GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length]; + int curPos = 0; + for (int i = 0; i < dims.length; i++) { + dims[i] = new GroupByColumnSelectorPlus( + selectorPlus[i], + curPos, + query.getResultRowDimensionStart() + i + ); + curPos += dims[i].getColumnSelectorStrategy().getGroupingKeySizeBytes(); + } + + final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( + querySpecificConfig, + query, + storageAdapter, + processingBuffer + ); - return cursors.flatMap( - cursor -> new BaseSequence<>( - new BaseSequence.IteratorMaker>() + if (query.getGranularity().equals(Granularities.ALL)) { + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public GroupByEngineIterator make() { - @Override - public GroupByEngineIterator make() - { - final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - final ColumnSelectorPlus[] selectorPlus = DimensionHandlerUtils - .createColumnSelectorPluses( - STRATEGY_FACTORY, - query.getDimensions(), - columnSelectorFactory - ); - GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length]; - int curPos = 0; - for (int i = 0; i < dims.length; i++) { - dims[i] = new GroupByColumnSelectorPlus( - selectorPlus[i], - curPos, - query.getResultRowDimensionStart() + i - ); - curPos += dims[i].getColumnSelectorStrategy().getGroupingKeySizeBytes(); - } - - final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( + if (cardinalityForArrayAggregation >= 0) { + return new ArrayAggregateIterator( + query, querySpecificConfig, + processingConfig, + cursor, + granularizer, + processingBuffer, + fudgeTimestamp, + dims, + hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()), + cardinalityForArrayAggregation + ); + } else { + return new HashAggregateIterator( query, - storageAdapter, - processingBuffer + querySpecificConfig, + processingConfig, + cursor, + granularizer, + processingBuffer, + fudgeTimestamp, + dims, + hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()) ); - - if (cardinalityForArrayAggregation >= 0) { - return new ArrayAggregateIterator( - query, - querySpecificConfig, - processingConfig, - cursor, - processingBuffer, - fudgeTimestamp, - dims, - hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()), - cardinalityForArrayAggregation - ); - } else { - return new HashAggregateIterator( - query, - querySpecificConfig, - processingConfig, - cursor, - processingBuffer, - fudgeTimestamp, - dims, - hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()) - ); - } } + } - @Override - public void cleanup(GroupByEngineIterator iterFromMake) - { - iterFromMake.close(); - } + @Override + public void cleanup(GroupByEngineIterator iterFromMake) + { + iterFromMake.close(); } - ) - ); + } + ); + } + + return Sequences.simple(granularizer.getBucketIterable()) + .flatMap(bucketInterval -> { + if (!granularizer.advanceToBucket(bucketInterval)) { + return Sequences.empty(); + } + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public GroupByEngineIterator make() + { + if (cardinalityForArrayAggregation >= 0) { + return new ArrayAggregateIterator( + query, + querySpecificConfig, + processingConfig, + cursor, + granularizer, + processingBuffer, + fudgeTimestamp, + dims, + hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()), + cardinalityForArrayAggregation + ); + } else { + return new HashAggregateIterator( + query, + querySpecificConfig, + processingConfig, + cursor, + granularizer, + processingBuffer, + fudgeTimestamp, + dims, + hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()) + ); + } + } + + @Override + public void cleanup(GroupByEngineIterator iterFromMake) + { + iterFromMake.close(); + } + } + ); + }); } /** @@ -227,6 +294,7 @@ private abstract static class GroupByEngineIterator implements Iterator protected final GroupByQuery query; protected final GroupByQueryConfig querySpecificConfig; protected final Cursor cursor; + protected final CursorGranularizer granularizer; protected final ByteBuffer buffer; protected final Grouper.KeySerde keySerde; protected final GroupByColumnSelectorPlus[] dims; @@ -243,6 +311,7 @@ public GroupByEngineIterator( final GroupByQueryConfig querySpecificConfig, final DruidProcessingConfig processingConfig, final Cursor cursor, + final CursorGranularizer granularizer, final ByteBuffer buffer, @Nullable final DateTime fudgeTimestamp, final GroupByColumnSelectorPlus[] dims, @@ -253,12 +322,13 @@ public GroupByEngineIterator( this.querySpecificConfig = querySpecificConfig; this.maxSelectorFootprint = querySpecificConfig.getActualMaxSelectorDictionarySize(processingConfig); this.cursor = cursor; + this.granularizer = granularizer; this.buffer = buffer; this.keySerde = new GroupByEngineKeySerde(dims, query); this.dims = dims; // Time is the same for every row in the cursor - this.timestamp = fudgeTimestamp != null ? fudgeTimestamp : cursor.getTime(); + this.timestamp = fudgeTimestamp != null ? fudgeTimestamp : granularizer.getBucketStart(); this.allSingleValueDims = allSingleValueDims; this.allowMultiValueGrouping = query.context().getBoolean( GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, @@ -322,7 +392,7 @@ public boolean hasNext() if (delegate != null && delegate.hasNext()) { return true; } else { - if (!cursor.isDone()) { + if (!cursor.isDone() && granularizer.currentOffsetWithinBucket()) { if (delegate != null) { delegate.close(); } @@ -407,13 +477,14 @@ private HashAggregateIterator( GroupByQueryConfig querySpecificConfig, DruidProcessingConfig processingConfig, Cursor cursor, + CursorGranularizer granularizer, ByteBuffer buffer, @Nullable DateTime fudgeTimestamp, GroupByColumnSelectorPlus[] dims, boolean allSingleValueDims ) { - super(query, querySpecificConfig, processingConfig, cursor, buffer, fudgeTimestamp, dims, allSingleValueDims); + super(query, querySpecificConfig, processingConfig, cursor, granularizer, buffer, fudgeTimestamp, dims, allSingleValueDims); final int dimCount = query.getDimensions().size(); stack = new int[dimCount]; @@ -517,7 +588,9 @@ protected void aggregateSingleValueDims(Grouper grouper) return; } - cursor.advance(); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } // Check selectorInternalFootprint after advancing the cursor. (We reset after the first row that causes // us to go past the limit.) @@ -605,7 +678,9 @@ protected void aggregateMultiValueDims(Grouper grouper) } // Advance to next row - cursor.advance(); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } currentRowWasPartiallyAggregated = false; // Check selectorInternalFootprint after advancing the cursor. (We reset after the first row that causes @@ -646,6 +721,7 @@ private ArrayAggregateIterator( GroupByQueryConfig querySpecificConfig, DruidProcessingConfig processingConfig, Cursor cursor, + CursorGranularizer granularizer, ByteBuffer buffer, @Nullable DateTime fudgeTimestamp, GroupByColumnSelectorPlus[] dims, @@ -653,7 +729,7 @@ private ArrayAggregateIterator( int cardinality ) { - super(query, querySpecificConfig, processingConfig, cursor, buffer, fudgeTimestamp, dims, allSingleValueDims); + super(query, querySpecificConfig, processingConfig, cursor, granularizer, buffer, fudgeTimestamp, dims, allSingleValueDims); this.cardinality = cardinality; if (dims.length == 1) { this.dim = dims[0]; @@ -703,7 +779,9 @@ private void aggregateSingleValueDims(IntGrouper grouper) if (!grouper.aggregate(key).isOk()) { return; } - cursor.advance(); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } } @@ -741,7 +819,9 @@ private void aggregateMultiValueDims(IntGrouper grouper) } } - cursor.advance(); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } if (!cursor.isDone()) { // dim is always an indexed string dimension multiValues = ((DimensionSelector) dim.getSelector()).getRow(); diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index d0f46a47e712..8a0fe376f8c3 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -24,14 +24,13 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; @@ -280,37 +279,25 @@ private ColumnAnalysis analyzeStringColumn( .setInterval(new Interval(start, end)) .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = storageAdapter.asCursorMaker(buildSpec).makeCursors(); - - size = cursors.accumulate( - 0L, - new Accumulator() - { - @Override - public Long accumulate(Long accumulated, Cursor cursor) - { - DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(columnName, columnName)); - if (selector == null) { - return accumulated; + try (final CursorMaker maker = storageAdapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + if (cursor != null) { + final DimensionSelector selector = + cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(columnName, columnName)); + while (!cursor.isDone()) { + final IndexedInts row = selector.getRow(); + for (int i = 0, rowSize = row.size(); i < rowSize; ++i) { + final String dimVal = selector.lookupName(row.get(i)); + if (dimVal != null && !dimVal.isEmpty()) { + size += StringUtils.estimatedBinaryLengthAsUTF8(dimVal); } - long current = accumulated; - while (!cursor.isDone()) { - final IndexedInts row = selector.getRow(); - for (int i = 0, rowSize = row.size(); i < rowSize; ++i) { - final String dimVal = selector.lookupName(row.get(i)); - if (dimVal != null && !dimVal.isEmpty()) { - current += StringUtils.estimatedBinaryLengthAsUTF8(dimVal); - } - } - cursor.advance(); - } - - return current; } + cursor.advance(); } - ); + } + } } if (analyzingMinMax()) { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index c553368fa160..a7dbd192b504 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.operator.ColumnWithDirection; @@ -46,6 +45,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -228,21 +228,15 @@ private Pair materializeStorageAdapter(StorageAdapter as) if (virtualColumns != null) { builder.setVirtualColumns(virtualColumns); } - final Sequence cursors = as.asCursorMaker(builder.build()).makeCursors(); + try (final CursorMaker maker = as.asCursorMaker(builder.build())) { + final Cursor cursor = maker.makeCursor(); - AtomicReference siggy = new AtomicReference<>(null); - - FrameWriter writer = cursors.accumulate(null, (accumulated, in) -> { - if (accumulated != null) { - // We should not get multiple cursors because we set the granularity to ALL. So, this should never - // actually happen, but it doesn't hurt us to defensive here, so we test against it. - throw new ISE("accumulated[%s] non-null, why did we get multiple cursors?", accumulated); - } + final AtomicReference siggy = new AtomicReference<>(null); long remainingRowsToSkip = limit.getOffset(); long remainingRowsToFetch = limit.getLimitOrMax(); - final ColumnSelectorFactory columnSelectorFactory = in.getColumnSelectorFactory(); + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); final RowSignature.Builder sigBob = RowSignature.builder(); for (String col : cols) { @@ -281,26 +275,24 @@ private Pair materializeStorageAdapter(StorageAdapter as) sortColumns ); - final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory); - for (; !in.isDoneOrInterrupted() && remainingRowsToSkip > 0; remainingRowsToSkip--) { - in.advance(); + final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory); + for (; !cursor.isDoneOrInterrupted() && remainingRowsToSkip > 0; remainingRowsToSkip--) { + cursor.advance(); } - for (; !in.isDoneOrInterrupted() && remainingRowsToFetch > 0; remainingRowsToFetch--) { - frameWriter.addSelection(); - in.advance(); + for (; !cursor.isDoneOrInterrupted() && remainingRowsToFetch > 0; remainingRowsToFetch--) { + writer.addSelection(); + cursor.advance(); } - return frameWriter; - }); - - if (writer == null) { - // This means that the accumulate was never called, which can only happen if we didn't have any cursors. - // We would only have zero cursors if we essentially didn't match anything, meaning that our RowsAndColumns - // should be completely empty. - return null; - } else { - final byte[] bytes = writer.toByteArray(); - return Pair.of(bytes, siggy.get()); + if (writer == null) { + // This means that the accumulate was never called, which can only happen if we didn't have any cursors. + // We would only have zero cursors if we essentially didn't match anything, meaning that our RowsAndColumns + // should be completely empty. + return null; + } else { + final byte[] bytes = writer.toByteArray(); + return Pair.of(bytes, siggy.get()); + } } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java index d4fa5e12b19f..29bdad4aafc9 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java @@ -24,14 +24,13 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; import org.apache.druid.segment.CloseableShapeshifter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; @@ -96,18 +95,12 @@ protected RowsAndColumns getRealRAC() @Nonnull private static RowsAndColumns materialize(StorageAdapter as) { - final Sequence cursors = as.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); + try (final CursorMaker maker = as.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); - RowSignature rowSignature = as.getRowSignature(); + final RowSignature rowSignature = as.getRowSignature(); - FrameWriter writer = cursors.accumulate(null, (accumulated, in) -> { - if (accumulated != null) { - // We should not get multiple cursors because we set the granularity to ALL. So, this should never - // actually happen, but it doesn't hurt us to defensive here, so we test against it. - throw new ISE("accumulated[%s] non-null, why did we get multiple cursors?", accumulated); - } - - final ColumnSelectorFactory columnSelectorFactory = in.getColumnSelectorFactory(); + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); final FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( new ArenaMemoryAllocatorFactory(200 << 20), // 200 MB, because, why not? @@ -115,19 +108,18 @@ private static RowsAndColumns materialize(StorageAdapter as) Collections.emptyList() ); - final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory); - while (!in.isDoneOrInterrupted()) { - frameWriter.addSelection(); - in.advance(); + final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory); + while (!cursor.isDoneOrInterrupted()) { + writer.addSelection(); + cursor.advance(); + } + + if (writer == null) { + return new EmptyRowsAndColumns(); + } else { + final byte[] bytes = writer.toByteArray(); + return new ColumnBasedFrameRowsAndColumns(Frame.wrap(bytes), rowSignature); } - return frameWriter; - }); - - if (writer == null) { - return new EmptyRowsAndColumns(); - } else { - final byte[] bytes = writer.toByteArray(); - return new ColumnBasedFrameRowsAndColumns(Frame.wrap(bytes), rowSignature); } } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 20b7e191cb33..094c38c8a97e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -34,6 +34,8 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; @@ -112,114 +114,112 @@ public Sequence process( // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); - return Sequences.concat( - adapter.asCursorMaker(query.asCursorBuildSpec(queryMetrics)) - .makeCursors() - .map(cursor -> new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - final List columnSelectors = new ArrayList<>(allColumns.size()); - final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); - - for (String column : allColumns) { - final BaseObjectColumnValueSelector selector = factory.makeColumnValueSelector(column); - ColumnCapabilities columnCapabilities = factory.getColumnCapabilities(column); - rowSignatureBuilder.add( - column, - columnCapabilities == null ? null : columnCapabilities.toColumnType() - ); - - columnSelectors.add(selector); - } - - final int batchSize = query.getBatchSize(); - return new Iterator() - { - private long offset = 0; - - @Override - public boolean hasNext() - { - return !cursor.isDone() && offset < limit; - } - - @Override - public ScanResultValue next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - if (hasTimeout && System.currentTimeMillis() >= timeoutAt) { - throw new QueryTimeoutException(StringUtils.nonStrictFormat("Query [%s] timed out", query.getId())); - } - final long lastOffset = offset; - final Object events; - final ScanQuery.ResultFormat resultFormat = query.getResultFormat(); - if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) { - events = rowsToCompactedList(); - } else if (ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { - events = rowsToList(); - } else { - throw new UOE("resultFormat[%s] is not supported", resultFormat.toString()); - } - responseContext.addRowScanCount(offset - lastOffset); - return new ScanResultValue(segmentId.toString(), allColumns, events, rowSignatureBuilder.build()); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - - private List> rowsToCompactedList() - { - final List> events = new ArrayList<>(batchSize); - final long iterLimit = Math.min(limit, offset + batchSize); - for (; !cursor.isDone() && offset < iterLimit; cursor.advance(), offset++) { - final List theEvent = new ArrayList<>(allColumns.size()); - for (int j = 0; j < allColumns.size(); j++) { - theEvent.add(getColumnValue(j)); - } - events.add(theEvent); - } - return events; - } - - private List> rowsToList() - { - List> events = Lists.newArrayListWithCapacity(batchSize); - final long iterLimit = Math.min(limit, offset + batchSize); - for (; !cursor.isDone() && offset < iterLimit; cursor.advance(), offset++) { - final Map theEvent = new LinkedHashMap<>(); - for (int j = 0; j < allColumns.size(); j++) { - theEvent.put(allColumns.get(j), getColumnValue(j)); - } - events.add(theEvent); - } - return events; - } - - private Object getColumnValue(int i) - { - final BaseObjectColumnValueSelector selector = columnSelectors.get(i); - final Object value = selector == null ? null : selector.getObject(); - return value; - } - }; - } - - @Override - public void cleanup(Iterator iterFromMake) - { - } - } - )) - ); + final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(queryMetrics)); + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + final Cursor cursor = maker.makeCursor(); + final List columnSelectors = new ArrayList<>(allColumns.size()); + final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + + for (String column : allColumns) { + final BaseObjectColumnValueSelector selector = factory.makeColumnValueSelector(column); + ColumnCapabilities columnCapabilities = factory.getColumnCapabilities(column); + rowSignatureBuilder.add( + column, + columnCapabilities == null ? null : columnCapabilities.toColumnType() + ); + + columnSelectors.add(selector); + } + + final int batchSize = query.getBatchSize(); + return new Iterator() + { + private long offset = 0; + + @Override + public boolean hasNext() + { + return !cursor.isDone() && offset < limit; + } + + @Override + public ScanResultValue next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + if (hasTimeout && System.currentTimeMillis() >= timeoutAt) { + throw new QueryTimeoutException(StringUtils.nonStrictFormat("Query [%s] timed out", query.getId())); + } + final long lastOffset = offset; + final Object events; + final ScanQuery.ResultFormat resultFormat = query.getResultFormat(); + if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) { + events = rowsToCompactedList(); + } else if (ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { + events = rowsToList(); + } else { + throw new UOE("resultFormat[%s] is not supported", resultFormat.toString()); + } + responseContext.addRowScanCount(offset - lastOffset); + return new ScanResultValue(segmentId.toString(), allColumns, events, rowSignatureBuilder.build()); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + private List> rowsToCompactedList() + { + final List> events = new ArrayList<>(batchSize); + final long iterLimit = Math.min(limit, offset + batchSize); + for (; !cursor.isDone() && offset < iterLimit; cursor.advance(), offset++) { + final List theEvent = new ArrayList<>(allColumns.size()); + for (int j = 0; j < allColumns.size(); j++) { + theEvent.add(getColumnValue(j)); + } + events.add(theEvent); + } + return events; + } + + private List> rowsToList() + { + List> events = Lists.newArrayListWithCapacity(batchSize); + final long iterLimit = Math.min(limit, offset + batchSize); + for (; !cursor.isDone() && offset < iterLimit; cursor.advance(), offset++) { + final Map theEvent = new LinkedHashMap<>(); + for (int j = 0; j < allColumns.size(); j++) { + theEvent.put(allColumns.get(j), getColumnValue(j)); + } + events.add(theEvent); + } + return events; + } + + private Object getColumnValue(int i) + { + final BaseObjectColumnValueSelector selector = columnSelectors.get(i); + final Object value = selector == null ? null : selector.getObject(); + return value; + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + } + } + ).withBaggage(maker); } /** diff --git a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index ffcdfdb0930b..53ac9f679c77 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -21,11 +21,12 @@ import com.google.common.collect.ImmutableList; import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.search.SearchQueryRunner.SearchColumnSelectorStrategy; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; @@ -73,47 +74,43 @@ public CursorBasedExecutor( public Object2IntRBTreeMap execute(final int limit) { final StorageAdapter adapter = segment.asStorageAdapter(); - final Sequence cursors = adapter.asCursorMaker(query.asCursorBuildSpec(null)).makeCursors(); - - final Object2IntRBTreeMap retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator()); - retVal.defaultReturnValue(0); - - cursors.accumulate( - retVal, - (map, cursor) -> { - if (map.size() >= limit) { - return map; - } - - final ColumnSelectorPlus[] selectorPlusList = DimensionHandlerUtils.createColumnSelectorPluses( - SearchQueryRunner.SEARCH_COLUMN_SELECTOR_STRATEGY_FACTORY, - dimsToSearch, - cursor.getColumnSelectorFactory() + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + final Object2IntRBTreeMap retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator()); + retVal.defaultReturnValue(0); + + if (cursor == null) { + return retVal; + } + + final ColumnSelectorPlus[] selectorPlusList = DimensionHandlerUtils.createColumnSelectorPluses( + SearchQueryRunner.SEARCH_COLUMN_SELECTOR_STRATEGY_FACTORY, + dimsToSearch, + cursor.getColumnSelectorFactory() + ); + + while (!cursor.isDone()) { + for (ColumnSelectorPlus selectorPlus : selectorPlusList) { + selectorPlus.getColumnSelectorStrategy().updateSearchResultSet( + selectorPlus.getOutputName(), + selectorPlus.getSelector(), + searchQuerySpec, + limit, + retVal ); - while (!cursor.isDone()) { - for (ColumnSelectorPlus selectorPlus : selectorPlusList) { - selectorPlus.getColumnSelectorStrategy().updateSearchResultSet( - selectorPlus.getOutputName(), - selectorPlus.getSelector(), - searchQuerySpec, - limit, - map - ); - - if (map.size() >= limit) { - return map; - } - } - - cursor.advance(); + if (retVal.size() >= limit) { + return retVal; } - - return map; } - ); - return retVal; + cursor.advance(); + } + + return retVal; + } } } } diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index cbfe856ab182..96da640110a5 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.ChainedExecutionQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -41,6 +40,7 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; @@ -48,9 +48,8 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Iterator; -import java.util.List; -import java.util.Objects; /** * @@ -113,24 +112,20 @@ public Result apply(Cursor cursor) }; } + @Nullable private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legacyQuery, boolean descending) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder(legacyQuery.asCursorBuildSpec(null)) .isDescending(descending) .build(); - final Sequence> resultSequence = Sequences.filter( - Sequences.map( - adapter.asCursorMaker(buildSpec).makeCursors(), - this.skipToFirstMatching - ), - Objects::nonNull - ); - final List> resultList = resultSequence.limit(1).toList(); - if (resultList.size() > 0) { - return resultList.get(0).getValue(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return null; + } + final Result result = skipToFirstMatching.apply(maker.makeCursor()); + return result == null ? null : result.getValue(); } - - return null; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index cfd68c1f39dd..fd450a6359b6 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -31,11 +31,14 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.vector.VectorCursorGranularizer; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; @@ -95,17 +98,17 @@ public Sequence> process( final CursorMaker cursorMaker = adapter.asCursorMaker(query.asCursorBuildSpec(timeseriesQueryMetrics)); final Sequence> result; - if (query.context().getVectorize().shouldVectorize(cursorMaker.canVectorize(), cursorMaker::cleanup)) { + if (query.context().getVectorize().shouldVectorize(cursorMaker.canVectorize(), cursorMaker::close)) { result = processVectorized(query, adapter, cursorMaker, interval, gran); } else { - result = processNonVectorized(query, cursorMaker); + result = processNonVectorized(query, adapter, cursorMaker, interval, gran); } final int limit = query.getLimit(); if (limit < Integer.MAX_VALUE) { - return result.limit(limit); + return result.limit(limit).withBaggage(cursorMaker); } else { - return result; + return result.withBaggage(cursorMaker); } } @@ -229,12 +232,84 @@ private Sequence> processVectorized( private Sequence> processNonVectorized( final TimeseriesQuery query, - final CursorMaker cursorMaker + final StorageAdapter adapter, + final CursorMaker cursorMaker, + final Interval queryInterval, + final Granularity gran ) { final boolean skipEmptyBuckets = query.isSkipEmptyBuckets(); final List aggregatorSpecs = query.getAggregatorSpecs(); + final Cursor cursor = cursorMaker.makeCursor(); + if (cursor == null) { + return Sequences.empty(); + } + final CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + gran, + queryInterval, + query.isDescending() + ); + if (granularizer == null) { + return Sequences.empty(); + } + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + return Sequences.simple(granularizer.getBucketIterable()) + .map( + bucketInterval -> { + // Whether or not the current bucket is empty + boolean emptyBucket = true; + boolean advancedToBucket = granularizer.advanceToBucket(bucketInterval); + if ((!advancedToBucket || cursor.isDone()) && skipEmptyBuckets) { + return null; + } + final Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()]; + final String[] aggregatorNames = new String[aggregatorSpecs.size()]; + + for (int i = 0; i < aggregatorSpecs.size(); i++) { + aggregators[i] = aggregatorSpecs.get(i).factorize(columnSelectorFactory); + aggregatorNames[i] = aggregatorSpecs.get(i).getName(); + } + try { + if (advancedToBucket) { + while (!cursor.isDone()) { + for (Aggregator aggregator : aggregators) { + aggregator.aggregate(); + } + emptyBucket = false; + + if (!granularizer.advanceCursorWithinBucket()) { + break; + } + } + } + + if (emptyBucket && skipEmptyBuckets) { + // Return null, will get filtered out later by the Objects::nonNull filter. + return null; + } + + final TimeseriesResultBuilder bob = new TimeseriesResultBuilder( + gran.toDateTime(bucketInterval.getStartMillis()) + ); + for (int i = 0; i < aggregatorSpecs.size(); i++) { + bob.addMetric(aggregatorNames[i], aggregators[i].get()); + } + + return bob.build(); + } + finally { + // cleanup + for (Aggregator agg : aggregators) { + agg.close(); + } + } + } + ) + .filter(Objects::nonNull); + /* return Sequences.filter( Sequences.map( cursorMaker.makeCursors(), @@ -277,5 +352,6 @@ private Sequence> processNonVectorized( ), Objects::nonNull ); + */ } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 2180b73253ce..839091cb079d 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; @@ -62,9 +63,9 @@ public AggregateTopNMetricFirstAlgorithm( } @Override - public TopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor) + public TopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor, CursorGranularizer granularizer) { - return new TopNParams(selectorPlus, cursor, Integer.MAX_VALUE); + return new TopNParams(selectorPlus, cursor, granularizer, Integer.MAX_VALUE); } @Override @@ -93,7 +94,7 @@ public void run( PooledTopNAlgorithm.PooledTopNParams singleMetricParam = null; int[] dimValSelector; try { - singleMetricParam = singleMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor()); + singleMetricParam = singleMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor(), params.getGranularizer()); singleMetricAlgo.run( singleMetricParam, singleMetricResultBuilder, @@ -112,7 +113,7 @@ public void run( PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null; try { // Run topN for all metrics for top N dimension values - allMetricsParam = allMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor()); + allMetricsParam = allMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor(), params.getGranularizer()); allMetricAlgo.run( allMetricsParam, resultBuilder, diff --git a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java index f34464a49d0e..7236a294197c 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java @@ -123,7 +123,7 @@ private void runWithCardinalityKnown( resetAggregators(aggregatesStore); numProcessed += numToProcess; - params.getCursor().reset(); + params.getGranularizer().resetBucket(); } if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); @@ -152,7 +152,7 @@ private void runWithCardinalityUnknown( long processedRows = scanAndAggregate(params, null, aggregatesStore); updateResults(params, null, aggregatesStore, resultBuilder); resetAggregators(aggregatesStore); - params.getCursor().reset(); + params.getGranularizer().resetBucket(); if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); queryMetrics.stopRecordingScanTime(); @@ -330,7 +330,7 @@ public static TopNResultBuilder makeResultBuilder(TopNParams params, TopNQuery q final Comparator comparator = query.getTopNMetricSpec() .getComparator(query.getAggregatorSpecs(), query.getPostAggregatorSpecs()); return query.getTopNMetricSpec().getResultBuilder( - params.getCursor().getTime(), + params.getGranularizer().getBucketStart(), query.getDimensionSpec(), query.getThreshold(), comparator, diff --git a/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScanner.java b/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScanner.java index 544cef0dfaf1..c4d653ec0446 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScanner.java +++ b/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScanner.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; @@ -37,6 +38,7 @@ long scanAndAggregate( BufferAggregator aggregator, int aggregatorSize, Cursor cursor, + CursorGranularizer granularizer, int[] positions, ByteBuffer resultsBuffer ); diff --git a/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScannerPrototype.java b/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScannerPrototype.java index 3200c625b2f0..f4c2ba1863bd 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScannerPrototype.java +++ b/processing/src/main/java/org/apache/druid/query/topn/Generic1AggPooledTopNScannerPrototype.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; @@ -46,6 +47,7 @@ public long scanAndAggregate( BufferAggregator aggregator, int aggregatorSize, Cursor cursor, + CursorGranularizer granularizer, int[] positions, ByteBuffer resultsBuffer ) @@ -69,7 +71,9 @@ public long scanAndAggregate( } } processedRows++; - cursor.advanceUninterruptibly(); + if (!granularizer.advanceCursorWithinBucketUninterruptedly()) { + break; + } } return processedRows; } diff --git a/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScanner.java b/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScanner.java index 6b6adb881c03..7f74b4b7c391 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScanner.java +++ b/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScanner.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; @@ -40,6 +41,7 @@ long scanAndAggregate( BufferAggregator aggregator2, int aggregator2Size, Cursor cursor, + CursorGranularizer granularizer, int[] positions, ByteBuffer resultsBuffer ); diff --git a/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java b/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java index d7536cb087fc..4de281d8a0b8 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java +++ b/processing/src/main/java/org/apache/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; @@ -48,6 +49,7 @@ public long scanAndAggregate( BufferAggregator aggregator2, int aggregator2Size, Cursor cursor, + CursorGranularizer granularizer, int[] positions, ByteBuffer resultsBuffer ) @@ -76,7 +78,9 @@ public long scanAndAggregate( } } processedRows++; - cursor.advanceUninterruptibly(); + if (!granularizer.advanceCursorWithinBucketUninterruptedly()) { + break; + } } return processedRows; } diff --git a/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java index ba5fbf251084..6f8ed23bc29d 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.segment.Cursor; @@ -48,12 +49,14 @@ public HeapBasedTopNAlgorithm( @Override public TopNParams makeInitParams( final ColumnSelectorPlus selectorPlus, - final Cursor cursor + final Cursor cursor, + final CursorGranularizer granularizer ) { return new TopNParams( selectorPlus, cursor, + granularizer, Integer.MAX_VALUE ); } @@ -96,6 +99,7 @@ protected long scanAndAggregate( query, selectorPlus.getSelector(), cursor, + params.getGranularizer(), rowSelector ); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java index d0c0fb064e03..d9213b94e09b 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.SimpleDoubleBufferAggregator; import org.apache.druid.query.monomorphicprocessing.SpecializationService; @@ -191,7 +192,7 @@ private static void computeSpecializedScanAndAggregateImplementations() if (SPECIALIZE_GENERIC_ONE_AGG_POOLED_TOPN) { SPECIALIZED_SCAN_AND_AGGREGATE_IMPLEMENTATIONS.add((params, positions, theAggregators) -> { if (theAggregators.length == 1) { - return scanAndAggregateGeneric1Agg(params, positions, theAggregators[0], params.getCursor()); + return scanAndAggregateGeneric1Agg(params, positions, theAggregators[0]); } return -1; }); @@ -199,7 +200,7 @@ private static void computeSpecializedScanAndAggregateImplementations() if (SPECIALIZE_GENERIC_TWO_AGG_POOLED_TOPN) { SPECIALIZED_SCAN_AND_AGGREGATE_IMPLEMENTATIONS.add((params, positions, theAggregators) -> { if (theAggregators.length == 2) { - return scanAndAggregateGeneric2Agg(params, positions, theAggregators, params.getCursor()); + return scanAndAggregateGeneric2Agg(params, positions, theAggregators); } return -1; }); @@ -222,7 +223,7 @@ public PooledTopNAlgorithm( } @Override - public PooledTopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor) + public PooledTopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor, CursorGranularizer granularizer) { final DimensionSelector dimSelector = (DimensionSelector) selectorPlus.getSelector(); final int cardinality = dimSelector.getValueCardinality(); @@ -272,6 +273,7 @@ public int[] build() return PooledTopNParams.builder() .withSelectorPlus(selectorPlus) .withCursor(cursor) + .withGranularizer(granularizer) .withResultsBufHolder(resultsBufHolder) .withResultsBuf(resultsBuf) .withArrayProvider(arrayProvider) @@ -387,8 +389,7 @@ private static long scanAndAggregateHistorical1SimpleDoubleAgg( private static long scanAndAggregateGeneric1Agg( PooledTopNParams params, int[] positions, - BufferAggregator aggregator, - Cursor cursor + BufferAggregator aggregator ) { String runtimeShape = StringRuntimeShape.of(aggregator); @@ -400,7 +401,8 @@ private static long scanAndAggregateGeneric1Agg( params.getDimSelector(), aggregator, params.getAggregatorSizes()[0], - cursor, + params.getCursor(), + params.getGranularizer(), positions, params.getResultsBuf() ); @@ -411,8 +413,7 @@ private static long scanAndAggregateGeneric1Agg( private static long scanAndAggregateGeneric2Agg( PooledTopNParams params, int[] positions, - BufferAggregator[] theAggregators, - Cursor cursor + BufferAggregator[] theAggregators ) { String runtimeShape = StringRuntimeShape.of(theAggregators); @@ -427,7 +428,8 @@ private static long scanAndAggregateGeneric2Agg( aggregatorSizes[0], theAggregators[1], aggregatorSizes[1], - cursor, + params.getCursor(), + params.getGranularizer(), positions, params.getResultsBuf() ); @@ -467,6 +469,7 @@ private static long scanAndAggregateDefault( final int numBytesPerRecord = params.getNumBytesPerRecord(); final int[] aggregatorSizes = params.getAggregatorSizes(); final Cursor cursor = params.getCursor(); + final CursorGranularizer granularizer = params.getGranularizer(); final DimensionSelector dimSelector = params.getDimSelector(); final int[] aggregatorOffsets = new int[aggregatorSizes.length]; @@ -666,8 +669,10 @@ private static long scanAndAggregateDefault( currentPosition ); } - cursor.advanceUninterruptibly(); processedRows++; + if (!granularizer.advanceCursorWithinBucketUninterruptedly()) { + break; + } } return processedRows; } @@ -799,6 +804,7 @@ public static class PooledTopNParams extends TopNParams public PooledTopNParams( ColumnSelectorPlus selectorPlus, Cursor cursor, + CursorGranularizer granularizer, ResourceHolder resultsBufHolder, ByteBuffer resultsBuf, int[] aggregatorSizes, @@ -807,7 +813,7 @@ public PooledTopNParams( TopNMetricSpecBuilder arrayProvider ) { - super(selectorPlus, cursor, numValuesPerPass); + super(selectorPlus, cursor, granularizer, numValuesPerPass); this.resultsBufHolder = resultsBufHolder; this.resultsBuf = resultsBuf; @@ -850,6 +856,7 @@ public static class Builder { private ColumnSelectorPlus selectorPlus; private Cursor cursor; + private CursorGranularizer granularizer; private ResourceHolder resultsBufHolder; private ByteBuffer resultsBuf; private int[] aggregatorSizes; @@ -869,6 +876,12 @@ public Builder withCursor(Cursor cursor) return this; } + public Builder withGranularizer(CursorGranularizer granularizer) + { + this.granularizer = granularizer; + return this; + } + public Builder withResultsBufHolder(ResourceHolder resultsBufHolder) { this.resultsBufHolder = resultsBufHolder; @@ -910,6 +923,7 @@ public PooledTopNParams build() return new PooledTopNParams( selectorPlus, cursor, + granularizer, resultsBufHolder, resultsBuf, aggregatorSizes, diff --git a/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java index 3b60bb65ee17..7e5d43b96ade 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; @@ -53,11 +54,12 @@ public TimeExtractionTopNAlgorithm(StorageAdapter storageAdapter, TopNQuery quer @Override @SuppressWarnings("unchecked") - public TopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor) + public TopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor, CursorGranularizer granularizer) { return new TopNParams( selectorPlus, cursor, + granularizer, Integer.MAX_VALUE ); } @@ -88,6 +90,7 @@ protected long scanAndAggregate( ) { final Cursor cursor = params.getCursor(); + final CursorGranularizer granularizer = params.getGranularizer(); final DimensionSelector dimSelector = params.getDimSelector(); long processedRows = 0; @@ -102,9 +105,10 @@ protected long scanAndAggregate( for (Aggregator aggregator : theAggregators) { aggregator.aggregate(); } - - cursor.advance(); processedRows++; + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } return processedRows; } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java index 31b4d9204e8d..3f197aa8c8af 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.segment.Cursor; @@ -34,7 +35,7 @@ public interface TopNAlgorithm int INIT_POSITION_VALUE = -1; int SKIP_POSITION_VALUE = -2; - TopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor); + TopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor, CursorGranularizer granularizer); void run( Parameters params, diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java b/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java index 96fb62f9012a..dafff313afb1 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.Result; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; @@ -44,7 +45,7 @@ public TopNMapFn( @SuppressWarnings("unchecked") @Nullable - public Result apply(final Cursor cursor, final @Nullable TopNQueryMetrics queryMetrics) + public Result apply(final Cursor cursor, final CursorGranularizer granularizer, final @Nullable TopNQueryMetrics queryMetrics) { final ColumnSelectorPlus> selectorPlus = DimensionHandlerUtils.createColumnSelectorPlus( @@ -59,7 +60,7 @@ public Result apply(final Cursor cursor, final @Nullable TopNQu TopNParams params = null; try { - params = topNAlgorithm.makeInitParams(selectorPlus, cursor); + params = topNAlgorithm.makeInitParams(selectorPlus, cursor, granularizer); if (queryMetrics != null) { queryMetrics.columnValueSelector(selectorPlus.getSelector()); queryMetrics.numValuesPerPass(params); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNParams.java b/processing/src/main/java/org/apache/druid/query/topn/TopNParams.java index cdc541f99248..3a932d00d28f 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNParams.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNParams.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; @@ -30,6 +31,7 @@ public class TopNParams { public static final int CARDINALITY_UNKNOWN = -1; private final Cursor cursor; + private final CursorGranularizer granularizer; private final int cardinality; private final int numValuesPerPass; private final ColumnSelectorPlus selectorPlus; @@ -37,11 +39,13 @@ public class TopNParams protected TopNParams( ColumnSelectorPlus selectorPlus, Cursor cursor, + CursorGranularizer granularizer, int numValuesPerPass ) { this.selectorPlus = selectorPlus; this.cursor = cursor; + this.granularizer = granularizer; this.cardinality = selectorPlus.getColumnSelectorStrategy().getCardinality(selectorPlus.getSelector()); this.numValuesPerPass = numValuesPerPass; } @@ -63,6 +67,11 @@ public Cursor getCursor() return cursor; } + public CursorGranularizer getGranularizer() + { + return granularizer; + } + public int getCardinality() { return cardinality; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index 632de4461ff2..076747666ccc 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -23,9 +23,13 @@ import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -68,18 +72,34 @@ public Sequence> query( final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); - return Sequences.filter( - Sequences.map( - adapter.asCursorMaker(query.asCursorBuildSpec(queryMetrics)).makeCursors(), - input -> { - if (queryMetrics != null) { - queryMetrics.cursor(input); - } - return mapFn.apply(input, queryMetrics); - } - ), - Predicates.notNull() + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(queryMetrics); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); + final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return Sequences.withBaggage(Sequences.empty(), maker); + } + final CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + buildSpec.getGranularity(), + buildSpec.getInterval(), + buildSpec.isDescending() ); + if (granularizer == null) { + return Sequences.withBaggage(Sequences.empty(), maker); + } + + if (queryMetrics != null) { + queryMetrics.cursor(cursor); + } + return Sequences.filter( + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + granularizer.advanceToBucket(bucketInterval); + return mapFn.apply(cursor, granularizer, queryMetrics); + }), + Predicates.notNull() + ).withBaggage(maker); } /** diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/NullableNumericTopNColumnAggregatesProcessor.java b/processing/src/main/java/org/apache/druid/query/topn/types/NullableNumericTopNColumnAggregatesProcessor.java index 36253c8d069a..4b56a833f8a5 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/types/NullableNumericTopNColumnAggregatesProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/topn/types/NullableNumericTopNColumnAggregatesProcessor.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn.types; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.topn.BaseTopNAlgorithm; import org.apache.druid.query.topn.TopNParams; @@ -88,6 +89,7 @@ public long scanAndAggregate( TopNQuery query, Selector selector, Cursor cursor, + CursorGranularizer granularizer, Aggregator[][] rowSelector ) { @@ -106,8 +108,10 @@ public long scanAndAggregate( aggregator.aggregate(); } } - cursor.advance(); processedRows++; + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } return processedRows; } diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java index c1afe548dbbd..6993b6ae9ffb 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn.types; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.topn.BaseTopNAlgorithm; import org.apache.druid.query.topn.TopNParams; @@ -112,6 +113,7 @@ public long scanAndAggregate( TopNQuery query, DimensionSelector selector, Cursor cursor, + CursorGranularizer granularizer, Aggregator[][] rowSelector ) { @@ -121,9 +123,9 @@ public long scanAndAggregate( // values (1:* or *:*), results can be entirely incorrect since an aggregator for a different value might be // chosen from the array based on the re-used dictionary id if (notUnknown && hasDictionary) { - return scanAndAggregateWithCardinalityKnown(query, cursor, selector, rowSelector); + return scanAndAggregateWithCardinalityKnown(query, cursor, granularizer, selector, rowSelector); } else { - return scanAndAggregateWithCardinalityUnknown(query, cursor, selector); + return scanAndAggregateWithCardinalityUnknown(query, cursor, granularizer, selector); } } @@ -141,6 +143,7 @@ public void initAggregateStore() private long scanAndAggregateWithCardinalityKnown( TopNQuery query, Cursor cursor, + CursorGranularizer granularizer, DimensionSelector selector, Aggregator[][] rowSelector ) @@ -164,8 +167,10 @@ private long scanAndAggregateWithCardinalityKnown( aggregator.aggregate(); } } - cursor.advance(); processedRows++; + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } return processedRows; } @@ -181,6 +186,7 @@ private long scanAndAggregateWithCardinalityKnown( private long scanAndAggregateWithCardinalityUnknown( TopNQuery query, Cursor cursor, + CursorGranularizer granularizer, DimensionSelector selector ) { @@ -198,8 +204,10 @@ private long scanAndAggregateWithCardinalityUnknown( aggregator.aggregate(); } } - cursor.advance(); processedRows++; + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } return processedRows; } diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessor.java b/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessor.java index d7c2fcd5b2b6..5fd141e64289 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessor.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn.types; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.dimension.ColumnSelectorStrategy; import org.apache.druid.query.topn.HeapBasedTopNAlgorithm; @@ -96,6 +97,7 @@ long scanAndAggregate( TopNQuery query, ValueSelectorType selector, Cursor cursor, + CursorGranularizer granularizer, Aggregator[][] rowSelector ); diff --git a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java index 163befcf2817..d1f3d7e3c98c 100644 --- a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java @@ -37,37 +37,6 @@ */ public class VectorCursorGranularizer { - // And a cursor that has been made from it. - private final VectorCursor cursor; - - // Iterable that iterates over time buckets. - private final Iterable bucketIterable; - - // Vector selector for the "__time" column. - @Nullable - private final VectorValueSelector timeSelector; - - // Current time vector. - @Nullable - private long[] timestamps = null; - - // Offset into the vector that we should start reading from. - private int startOffset = 0; - - // Offset into the vector that is one past the last one we should read. - private int endOffset = 0; - - private VectorCursorGranularizer( - VectorCursor cursor, - Iterable bucketIterable, - @Nullable VectorValueSelector timeSelector - ) - { - this.cursor = cursor; - this.bucketIterable = bucketIterable; - this.timeSelector = timeSelector; - } - @Nullable public static VectorCursorGranularizer create( final StorageAdapter storageAdapter, @@ -102,6 +71,37 @@ public static VectorCursorGranularizer create( return new VectorCursorGranularizer(cursor, bucketIterable, timeSelector); } + // And a cursor that has been made from it. + private final VectorCursor cursor; + + // Iterable that iterates over time buckets. + private final Iterable bucketIterable; + + // Vector selector for the "__time" column. + @Nullable + private final VectorValueSelector timeSelector; + + // Current time vector. + @Nullable + private long[] timestamps = null; + + // Offset into the vector that we should start reading from. + private int startOffset = 0; + + // Offset into the vector that is one past the last one we should read. + private int endOffset = 0; + + private VectorCursorGranularizer( + VectorCursor cursor, + Iterable bucketIterable, + @Nullable VectorValueSelector timeSelector + ) + { + this.cursor = cursor; + this.bucketIterable = bucketIterable; + this.timeSelector = timeSelector; + } + public void setCurrentOffsets(final Interval bucketInterval) { final long timeStart = bucketInterval.getStartMillis(); diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index dce076f3a91e..d2bddba99fa2 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -19,8 +19,6 @@ package org.apache.druid.segment; -import org.joda.time.DateTime; - /** * Cursor is an interface for iteration over a range of data points, used during query execution. {@link * QueryableIndexCursorMaker.QueryableIndexCursor} is an implementation for historical segments, and {@link @@ -36,10 +34,12 @@ public interface Cursor { ColumnSelectorFactory getColumnSelectorFactory(); - DateTime getTime(); + void advance(); void advanceUninterruptibly(); + boolean isDone(); boolean isDoneOrInterrupted(); + void reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index 5c72d8dbd50e..8ebd2fd444df 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -19,11 +19,11 @@ package org.apache.druid.segment; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryContext; -import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.vector.VectorCursor; @@ -40,7 +40,6 @@ public interface CursorFactory { default CursorMaker asCursorMaker(CursorBuildSpec spec) { - return new CursorMaker() { @Override @@ -50,15 +49,17 @@ public boolean canVectorize() } @Override - public Sequence makeCursors() + public Cursor makeCursor() { - return CursorFactory.this.makeCursors( - spec.getFilter(), - spec.getInterval(), - spec.getVirtualColumns(), - spec.getGranularity(), - spec.isDescending(), - spec.getQueryMetrics() + return Iterables.getOnlyElement( + CursorFactory.this.makeCursors( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + Granularities.ALL, + spec.isDescending(), + spec.getQueryMetrics() + ).toList() ); } @@ -74,6 +75,12 @@ public VectorCursor makeVectorCursor() spec.getQueryMetrics() ); } + + @Override + public void close() + { + // consuming sequences of CursorFactory are expected to close themselves. + } }; } @@ -84,7 +91,8 @@ public VectorCursor makeVectorCursor() * mode if this method returns false. * * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#canVectorize()}. - * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. + * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. This method is no longer + * implemented by any built-in factories. */ @Deprecated default boolean canVectorize( @@ -93,13 +101,13 @@ default boolean canVectorize( boolean descending ) { - return false; + throw DruidException.defensive("canVectorize is no longer supported, use asCursorMaker instead"); } /** * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity). * - * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#makeCursors()}. + * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#makeCursor()}. * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. Recommend for implementors to fill * this method in with: *
@@ -113,16 +121,20 @@ default boolean canVectorize(
    *                                                      .build();
    *     return asCursorMaker(buildSpec).makeCursors();
    * 
+ * This method is no longer implemented by any built-in factories. */ @Deprecated - Sequence makeCursors( + default Sequence makeCursors( @Nullable Filter filter, Interval interval, VirtualColumns virtualColumns, Granularity gran, boolean descending, @Nullable QueryMetrics queryMetrics - ); + ) + { + throw DruidException.defensive("makeCursors is no longer supported, use asCursorMaker instead"); + } /** * Creates a VectorCursor. Unlike the Cursor returned by "makeCursor", there is just one of these. Hence, this method @@ -134,7 +146,7 @@ Sequence makeCursors( * * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call * {@link CursorMaker#makeVectorCursor()}. Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} - * instead. + * instead. This method is no longer implemented by any built-in factories. */ @Deprecated @Nullable @@ -147,51 +159,6 @@ default VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - throw new UnsupportedOperationException("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); - } - - default Sequence delegateMakeCursorToMaker( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return asCursorMaker( - CursorBuildSpec.builder() - .setFilter(filter) - .setInterval(interval) - .setVirtualColumns(virtualColumns) - .setGranularity(gran) - .isDescending(descending) - .setQueryMetrics(queryMetrics) - .build() - ).makeCursors(); - } - - default VectorCursor delegateMakeVectorCursorToMaker( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @Nullable QueryMetrics queryMetrics - ) - { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setInterval(interval) - .setVirtualColumns(virtualColumns) - .isDescending(descending) - .setQueryContext( - QueryContext.of( - ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, vectorSize) - ) - ) - .setQueryMetrics(queryMetrics) - .build(); - return asCursorMaker(buildSpec).makeVectorCursor(); + throw DruidException.defensive("makeVectorCursor is no longer supported, use asCursorMaker instead"); } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java b/processing/src/main/java/org/apache/druid/segment/CursorMaker.java index 6922814c5bb5..871ecc8f0ffa 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorMaker.java @@ -20,30 +20,21 @@ package org.apache.druid.segment; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.segment.vector.VectorCursor; import javax.annotation.Nullable; +import java.io.Closeable; -public interface CursorMaker +public interface CursorMaker extends Closeable { /** - * Create a {@link Sequence} of {@link Cursor} for use with non-vectorized query engines. Each {@link Cursor} of the - * sequence corresponds to a {@link org.apache.druid.java.util.common.granularity.Granularity} bucket determined by - * {@link CursorBuildSpec#getGranularity()}. - *

- * Consuming this {@link Sequence} will automatically close all resources associated with this {@link CursorMaker} - * so calling {@link #cleanup()} is not needed. + * Create a {@link Cursor} for use with non-vectorized query engines. */ - Sequence makeCursors(); + @Nullable + Cursor makeCursor(); /** * Create a {@link VectorCursor} for use with vectorized query engines. - *

- * Advancing this {@link VectorCursor} to the end or explicitly calling {@link VectorCursor#close()} will - * automatically close all resources associated with this {@link CursorMaker} so calling {@link #cleanup()} is not - * needed. */ @Nullable default VectorCursor makeVectorCursor() @@ -52,8 +43,7 @@ default VectorCursor makeVectorCursor() } /** - * Returns true if this {@link CursorMaker} supports creating vectorized selectors. This operation may acquire - * underlying resources, so calling {@link #cleanup()} is necessary if no cursors are created and consumed. + * Returns true if this {@link CursorMaker} supports calling {@link #makeVectorCursor()}. */ default boolean canVectorize() { @@ -61,12 +51,12 @@ default boolean canVectorize() } /** - * Release any resources acquired if cursors are not consumed. Typically consuming a cursor or vector cursor releases - * the resources upon completion, but if for some reason this will not happen, this method must be called. + * Release any resources acquired by cursors. */ - default void cleanup() + @Override + default void close() { - // nothing to cleanup + // nothing to close } CursorMaker EMPTY = new CursorMaker() @@ -78,9 +68,9 @@ public boolean canVectorize() } @Override - public Sequence makeCursors() + public Cursor makeCursor() { - return Sequences.empty(); + return null; } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index e2f8ecadb681..6275a8fea9fd 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -20,9 +20,6 @@ package org.apache.druid.segment; import com.google.common.collect.ImmutableList; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -62,19 +59,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) return baseStorageAdapter.asCursorMaker(buildSpecBuilder.build()); } - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Override public Interval getInterval() { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index b0ce54d66d08..9bfda8a20cd3 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -20,17 +20,12 @@ package org.apache.druid.segment; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.BaseQuery; @@ -65,7 +60,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.List; -import java.util.Objects; import java.util.concurrent.TimeUnit; public class QueryableIndexCursorMaker implements CursorMaker @@ -140,7 +134,7 @@ public boolean canVectorize() } @Override - public Sequence makeCursors() + public Cursor makeCursor() { if (metrics != null) { metrics.vectorized(false); @@ -163,84 +157,59 @@ public Sequence makeCursors() baseOffset = BitmapOffset.of(filterBundle.getIndex().getBitmap(), descending, index.getNumRows()); } - Iterable iterable = gran.getIterable(interval); + final long timeStart = Math.max(interval.getStartMillis(), minDataTimestamp); + final long timeEnd = interval.getEndMillis(); + if (descending) { - iterable = Lists.reverse(ImmutableList.copyOf(iterable)); - } - - return Sequences.filter( - Sequences.withBaggage( - Sequences.map( - Sequences.simple(iterable), - new Function() - { - @Override - public Cursor apply(final Interval inputInterval) - { - final long timeStart = Math.max(interval.getStartMillis(), inputInterval.getStartMillis()); - final long timeEnd = Math.min( - interval.getEndMillis(), - gran.increment(inputInterval.getStartMillis()) - ); - - if (descending) { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { - break; - } - } - } else { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { - break; - } - } - } - - final Offset offset = descending ? - new DescendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeStart, - minDataTimestamp >= timeStart - ) : - new AscendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeEnd, - maxDataTimestamp < timeEnd - ); - - - final Offset baseCursorOffset = offset.clone(); - final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( - virtualColumns, - descending, - baseCursorOffset.getBaseReadableOffset(), - columnCache - ); - final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis()); - // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter - // needs to use a value matcher - if (filterBundle != null && filterBundle.getMatcherBundle() != null) { - final ValueMatcher matcher = filterBundle.getMatcherBundle() - .valueMatcher( - columnSelectorFactory, - baseCursorOffset, - descending - ); - final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); - return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, myBucket); - } else { - return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket); - } - } - } - ), - resources - ), - Objects::nonNull + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { + break; + } + } + } else { + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { + break; + } + } + } + + final Offset offset = descending ? + new DescendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeStart, + minDataTimestamp >= timeStart + ) : + new AscendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeEnd, + maxDataTimestamp < timeEnd + ); + + + final Offset baseCursorOffset = offset.clone(); + final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( + virtualColumns, + descending, + baseCursorOffset.getBaseReadableOffset(), + columnCache ); + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter + // needs to use a value matcher + if (filterBundle != null && filterBundle.getMatcherBundle() != null) { + final ValueMatcher matcher = filterBundle.getMatcherBundle() + .valueMatcher( + columnSelectorFactory, + baseCursorOffset, + descending + ); + final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); + return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, DateTimes.utc(timeStart)); + } else { + return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, DateTimes.utc(timeStart)); + } } @Nullable @@ -260,7 +229,7 @@ public VectorCursor makeVectorCursor() // sanity check if (!canVectorize()) { - cleanup(); + close(); throw new IllegalStateException("canVectorize()"); } if (metrics != null) { @@ -332,7 +301,7 @@ public VectorCursor makeVectorCursor() } @Override - public void cleanup() + public void close() { CloseableUtils.closeAndWrapExceptions(resourcesSupplier.get()); } @@ -539,12 +508,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return bucketStart; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 810f59cb4dbe..7d13e57e6531 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -21,9 +21,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -32,7 +29,6 @@ import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -189,56 +185,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) ); } - @Override - public boolean canVectorize( - @Nullable final Filter filter, - final VirtualColumns virtualColumns, - final boolean descending - ) - { - // For safety, this uses the old-school canVectorize implementation instead of delegating to the CursorMaker, - // because QueryableIndexCursorMaker expects to make cursors one way or another and so opens stuff that must be - // cleaned if for some reason a cursor or vector cursor is not constructed - if (filter != null) { - - final boolean filterCanVectorize = filter.canVectorizeMatcher(this); - - if (!filterCanVectorize) { - return false; - } - } - - // vector cursors can't iterate backwards yet - return !descending; - } - - @Override - @Nullable - public VectorCursor makeVectorCursor( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final boolean descending, - final int vectorSize, - @Nullable final QueryMetrics queryMetrics - ) - { - return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Override public Metadata getMetadata() { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index f843fe631311..d591e0cd11d2 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -91,12 +91,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return cursorTime; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index d8b16cf900af..2092b17f428d 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -20,7 +20,6 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; @@ -29,12 +28,12 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.SimpleSequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -172,15 +171,17 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) { return new CursorMaker() { + final Closer closer = Closer.create(); + @Override - public Sequence makeCursors() + public Cursor makeCursor() { final Granularity gran = spec.getGranularity(); final Interval actualInterval = spec.getInterval() .overlap(new Interval(getMinTime(), gran.bucketEnd(getMaxTime()))); if (actualInterval == null) { - return Sequences.empty(); + return null; } if (!isQueryGranularityAllowed(actualInterval, gran)) { @@ -195,42 +196,27 @@ public Sequence makeCursors() spec.isDescending() ? reverse(rowSequence) : rowSequence, rowAdapter ); + closer.register(rowWalker); + return new RowBasedCursor<>( + rowWalker, + rowAdapter, + spec.getFilter(), + actualInterval, + spec.getVirtualColumns(), + gran, + spec.isDescending(), + rowSignature + ); + } - final Iterable bucketIntervals = gran.getIterable(actualInterval); - - return Sequences.simple( - Iterables.transform( - spec.isDescending() ? reverse(bucketIntervals) : bucketIntervals, - bucketInterval -> - (Cursor) new RowBasedCursor<>( - rowWalker, - rowAdapter, - spec.getFilter(), - bucketInterval, - spec.getVirtualColumns(), - gran, - spec.isDescending(), - rowSignature - ) - ) - ).withBaggage(rowWalker::close); + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); } }; } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval queryInterval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable final QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, queryInterval, virtualColumns, gran, descending, queryMetrics); - } - /** * Reverse a Sequence. * diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index 5404937f5196..8d3ab329e032 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -24,7 +24,6 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Arrays; @@ -210,12 +209,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 4d4aeaf70465..86874a5140a1 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -259,12 +258,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index 4400dcd52267..59923c81db54 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -23,10 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.QueryMetrics; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.filter.BooleanFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.EqualityFilter; @@ -49,6 +46,7 @@ import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.join.PostJoinCursor; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -102,67 +100,55 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) input, input == null ? null : spec.getVirtualColumns().getColumnCapabilitiesWithFallback(baseAdapter, input) ); - final CursorBuildSpec unnestBuildSpec = CursorBuildSpec.builder(spec) - .setFilter(filterPair.lhs) - .setVirtualColumns( - VirtualColumns.create( - Collections.singletonList(unnestColumn) - ) - ) - .build(); + final CursorBuildSpec unnestBuildSpec = + CursorBuildSpec.builder(spec) + .setFilter(filterPair.lhs) + .setVirtualColumns(VirtualColumns.create(Collections.singletonList(unnestColumn))) + .build(); + return new CursorMaker() { + final Closer closer = Closer.create(); @Override - public Sequence makeCursors() + public Cursor makeCursor() { - final Sequence baseCursorSequence = baseAdapter.asCursorMaker(unnestBuildSpec).makeCursors(); - return Sequences.map( - baseCursorSequence, - cursor -> { - Objects.requireNonNull(cursor); - final ColumnCapabilities capabilities = unnestColumn.capabilities( - cursor.getColumnSelectorFactory(), - unnestColumn.getOutputName() - ); - final Cursor unnestCursor; - - if (useDimensionCursor(capabilities)) { - unnestCursor = new UnnestDimensionCursor( - cursor, - cursor.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } else { - unnestCursor = new UnnestColumnValueSelectorCursor( - cursor, - cursor.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } - return PostJoinCursor.wrap( - unnestCursor, - spec.getVirtualColumns(), - filterPair.rhs - ); - } + final CursorMaker maker = closer.register(baseAdapter.asCursorMaker(unnestBuildSpec)); + final Cursor cursor = maker.makeCursor(); + Objects.requireNonNull(cursor); + final ColumnCapabilities capabilities = unnestColumn.capabilities( + cursor.getColumnSelectorFactory(), + unnestColumn.getOutputName() + ); + final Cursor unnestCursor; + + if (useDimensionCursor(capabilities)) { + unnestCursor = new UnnestDimensionCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); + } else { + unnestCursor = new UnnestColumnValueSelectorCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); + } + return PostJoinCursor.wrap( + unnestCursor, + spec.getVirtualColumns(), + filterPair.rhs ); } - }; - } - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java index 22f58693b7c0..442666f084dd 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java @@ -19,12 +19,8 @@ package org.apache.druid.segment.incremental; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -58,10 +54,10 @@ public IncrementalIndexCursorMaker( } @Override - public Sequence makeCursors() + public Cursor makeCursor() { if (index.isEmpty()) { - return Sequences.empty(); + return null; } if (builder.getQueryMetrics() != null) { @@ -74,26 +70,19 @@ public Sequence makeCursors() ); if (!builder.getInterval().overlaps(dataInterval)) { - return Sequences.empty(); + return null; } final Interval actualInterval = builder.getInterval().overlap(dataInterval); - Iterable intervals = builder.getGranularity().getIterable(actualInterval); - if (builder.isDescending()) { - intervals = Lists.reverse(ImmutableList.copyOf(intervals)); - } - return Sequences - .simple(intervals) - .map(i -> new IncrementalIndexCursor( - storageAdapter, - index, - builder.getVirtualColumns(), - builder.isDescending(), - builder.getFilter(), - i, - actualInterval, - builder.getGranularity() - )); + return new IncrementalIndexCursor( + storageAdapter, + index, + builder.getVirtualColumns(), + builder.isDescending(), + builder.getFilter(), + actualInterval, + builder.getGranularity() + ); } static class IncrementalIndexCursor implements Cursor @@ -115,7 +104,6 @@ static class IncrementalIndexCursor implements Cursor VirtualColumns virtualColumns, boolean descending, @Nullable Filter filter, - Interval interval, Interval actualInterval, Granularity gran ) @@ -131,14 +119,13 @@ static class IncrementalIndexCursor implements Cursor maxRowIndex = index.getLastRowIndex(); filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; - final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); cursorIterable = index.getFacts().timeRangeIterable( descending, - timeStart, - Math.min(actualInterval.getEndMillis(), gran.increment(interval.getStartMillis())) + actualInterval.getStartMillis(), + actualInterval.getEndMillis() ); emptyRange = !cursorIterable.iterator().hasNext(); - time = gran.toDateTime(interval.getStartMillis()); + time = gran.toDateTime(actualInterval.getStartMillis()); reset(); } @@ -149,12 +136,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return time; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 5fb9adba1d57..b4a9270ee582 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -19,11 +19,6 @@ package org.apache.druid.segment.incremental; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; @@ -31,7 +26,6 @@ import org.apache.druid.segment.Metadata; import org.apache.druid.segment.NestedDataColumnIndexerV4; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; @@ -254,19 +248,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) return new IncrementalIndexCursorMaker(this, index, spec); } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Override public Metadata getMetadata() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index e6f67a1683e5..ee8a70aea84c 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -28,7 +28,6 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowIdSupplier; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -167,13 +166,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return joinColumnSelectorFactory; } - @Override - @Nonnull - public DateTime getTime() - { - return leftCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 1f5bec5cebce..547fb7be28f1 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -23,11 +23,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; @@ -44,10 +40,10 @@ import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey; import org.apache.druid.segment.join.filter.JoinFilterSplit; import org.apache.druid.segment.vector.VectorCursor; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.Arrays; import java.util.LinkedHashSet; @@ -240,11 +236,13 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) .build(); return baseAdapter.asCursorMaker(newSpec); } + return new CursorMaker() { + final Closer joinablesCloser = Closer.create(); @Override - public Sequence makeCursors() + public Cursor makeCursor() { // Filter pre-analysis key implied by the call to "makeCursors". We need to sanity-check that it matches // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursors" call (it @@ -289,66 +287,43 @@ public Sequence makeCursors() ); cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns); - final Sequence baseCursorSequence = baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build()) - .makeCursors(); + final Cursor baseCursor = joinablesCloser.register(baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build())) + .makeCursor(); - Closer joinablesCloser = Closer.create(); - return Sequences.map( - baseCursorSequence, - cursor -> { - assert cursor != null; - Cursor retVal = cursor; + assert baseCursor != null; + Cursor retVal = baseCursor; - for (JoinableClause clause : clauses) { - retVal = HashJoinEngine.makeJoinCursor(retVal, clause, spec.isDescending(), joinablesCloser); - } + for (JoinableClause clause : clauses) { + retVal = HashJoinEngine.makeJoinCursor(retVal, clause, spec.isDescending(), joinablesCloser); + } - return PostJoinCursor.wrap( - retVal, - VirtualColumns.fromIterable(preAnalysis.getPostJoinVirtualColumns()), - joinFilterSplit.getJoinTableFilter().orElse(null) - ); - } - ).withBaggage(joinablesCloser); + return PostJoinCursor.wrap( + retVal, + VirtualColumns.fromIterable(preAnalysis.getPostJoinVirtualColumns()), + joinFilterSplit.getJoinTableFilter().orElse(null) + ); } - }; - } - @Override - public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) - { - // HashJoinEngine isn't vectorized yet. - // However, we can still vectorize if there are no clauses, since that means all we need to do is apply - // a base filter. That's easy enough! - return clauses.isEmpty() && baseAdapter.canVectorize(baseFilterAnd(filter), virtualColumns, descending); - } + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(joinablesCloser); + } - @Nullable - @Override - public VectorCursor makeVectorCursor( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); - } + @Override + public boolean canVectorize() + { + return CursorMaker.super.canVectorize(); + } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - @Nonnull final Interval interval, - @Nonnull final VirtualColumns virtualColumns, - @Nonnull final Granularity gran, - final boolean descending, - @Nullable final QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); + @Nullable + @Override + public VectorCursor makeVectorCursor() + { + return CursorMaker.super.makeVectorCursor(); + } + }; } /** diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 38413b57a4fc..81cd032bc3f9 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -26,7 +26,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -106,12 +105,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Nullable public Filter getPostJoinFilter() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index ee8d70cab305..edc9bb056158 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -22,8 +22,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.cache.CacheKeyBuilder; @@ -32,6 +30,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; @@ -121,59 +120,50 @@ public BroadcastSegmentIndexedTable( ) .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - final Sequence sequence = Sequences.map( - cursors, - cursor -> { - if (cursor == null) { - return 0; - } - int rowNumber = 0; - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap - // indexes, but, an optimization for another day - final List selectors = keyColumnNames - .stream() - .map(columnName -> { - // multi-value dimensions are not currently supported - if (adapter.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) { - return NilColumnValueSelector.instance(); - } - return columnSelectorFactory.makeColumnValueSelector(columnName); - }) - .collect(Collectors.toList()); + int rowNumber = 0; + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - while (!cursor.isDone()) { - for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { - final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); - final int columnPosition = rowSignature.indexOf(keyColumnName); - final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); - keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); + // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap + // indexes, but, an optimization for another day + final List selectors = keyColumnNames + .stream() + .map(columnName -> { + // multi-value dimensions are not currently supported + if (adapter.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) { + return NilColumnValueSelector.instance(); } + return columnSelectorFactory.makeColumnValueSelector(columnName); + }) + .collect(Collectors.toList()); - if (rowNumber % 100_000 == 0) { - if (rowNumber == 0) { - LOG.debug("Indexed first row for table %s", theSegment.getId()); - } else { - LOG.debug("Indexed row %s for table %s", rowNumber, theSegment.getId()); - } - } - rowNumber++; - cursor.advance(); - } - return rowNumber; + while (!cursor.isDone()) { + for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { + final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); + final int columnPosition = rowSignature.indexOf(keyColumnName); + final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); + keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); } - ); - Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in); + if (rowNumber % 100_000 == 0) { + if (rowNumber == 0) { + LOG.debug("Indexed first row for table %s", theSegment.getId()); + } else { + LOG.debug("Indexed row %s for table %s", rowNumber, theSegment.getId()); + } + } + rowNumber++; + cursor.advance(); + } - this.keyColumnsIndexes = indexBuilders.stream() - .map(builder -> builder != null ? builder.build() : null) - .collect(Collectors.toList()); + this.keyColumnsIndexes = indexBuilders.stream() + .map(builder -> builder != null ? builder.build() : null) + .collect(Collectors.toList()); - LOG.info("Created BroadcastSegmentIndexedTable with %s rows.", totalRows); + LOG.info("Created BroadcastSegmentIndexedTable with %s rows.", rowNumber); + } } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index 7f63fbbe0f83..3eaf8a463f52 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -113,7 +113,7 @@ public FrameBasedIndexedTable( indexBuilders.add(m); } - final Sequence cursors = Sequences.concat( + final Sequence cursors = Sequences.simple( frameBasedInlineDataSource .getFrames() .stream() @@ -122,7 +122,7 @@ public FrameBasedIndexedTable( RowSignature rowSignature = frameSignaturePair.getRowSignature(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); - return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); + return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursor(); }) .collect(Collectors.toList()) ); diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index da9a7eeeb3c5..07e9714cb981 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -38,7 +38,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -56,7 +55,21 @@ public TestArrayStorageAdapter(QueryableIndex index) public CursorMaker asCursorMaker(CursorBuildSpec spec) { final CursorMaker delegate = super.asCursorMaker(spec); - return () -> delegate.makeCursors().map(DecoratedCursor::new); + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return new DecoratedCursor(delegate.makeCursor()); + } + + @Override + public void close() + { + delegate.close(); + } + }; } @@ -171,12 +184,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return cursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java index e84d46852605..582efbd9a27b 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; @@ -128,25 +129,32 @@ protected void verifySingleFrameReadableChannel( readableFrameChannel.close(); // build list of rows from written and read data to verify - List> writtenData = - adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())) - .toList(); - - FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( - readbackFrame, - FrameReader.create(adapter.getRowSignature()), - Intervals.ETERNITY - ); - List> readData = - adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())) - .toList(); - - Assert.assertEquals("Read rows count is different from written rows count", writtenData.size(), readData.size()); - Assert.assertEquals("Read data is different from written data", writtenData, readData); + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN);) { + List> writtenData = + FrameTestUtil.readRowsFromCursor( + maker.makeCursor(), + adapter.getRowSignature() + ).toList(); + + FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( + readbackFrame, + FrameReader.create(adapter.getRowSignature()), + Intervals.ETERNITY + ); + try (final CursorMaker frameMaker = frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + List> readData = FrameTestUtil.readRowsFromCursor( + frameMaker.makeCursor(), + frameStorageAdapter.getRowSignature() + ).toList(); + + Assert.assertEquals( + "Read rows count is different from written rows count", + writtenData.size(), + readData.size() + ); + Assert.assertEquals("Read data is different from written data", writtenData, readData); + } + } } @Test diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index 9b0d46f2fa80..3fa224a62b5f 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -365,7 +365,7 @@ public void tearDown() @Test public void test_makeCursors() { - assertCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec).makeCursors()); + assertCursorMatch(adapter -> adapter.asCursorMaker(buildSpec)); } @Test @@ -374,15 +374,17 @@ public void test_makeVectorCursor() assertVectorCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec)); } - private void assertCursorsMatch(final Function> call) + private void assertCursorMatch(final Function call) { final RowSignature signature = frameAdapter.getRowSignature(); - final Sequence> queryableRows = - call.apply(queryableAdapter).flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, signature)); - final Sequence> frameRows = - call.apply(frameAdapter) - .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(advanceAndReset(cursor), signature)); - FrameTestUtil.assertRowsEqual(queryableRows, frameRows); + try (final CursorMaker queryableMaker = call.apply(queryableAdapter); + final CursorMaker frameMaker = call.apply(frameAdapter)) { + final Sequence> queryableRows = + FrameTestUtil.readRowsFromCursor(queryableMaker.makeCursor(), signature); + final Sequence> frameRows = + FrameTestUtil.readRowsFromCursor(frameMaker.makeCursor(), signature); + FrameTestUtil.assertRowsEqual(queryableRows, frameRows); + } } private void assertVectorCursorsMatch(final Function call) @@ -397,8 +399,8 @@ private void assertVectorCursorsMatch(final Function frames() throw DruidException.defensive("Unrecognized frame type"); } - final Sequence cursors = FrameTestUtil.makeCursorsForAdapter(adapter, populateRowNumber); - - return cursors.flatMap( - cursor -> new BaseSequence<>( - new BaseSequence.IteratorMaker>() + final Pair cursorAndCloseable = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber); + final Cursor cursor = cursorAndCloseable.lhs; + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + return new Iterator() { @Override - public Iterator make() + public boolean hasNext() { - final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - return new Iterator() - { - @Override - public boolean hasNext() - { - return !cursor.isDone(); - } - - @Override - public Frame next() - { - if (cursor.isDone()) { - throw new NoSuchElementException(); - } - - try (final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { - while (!cursor.isDone()) { - if (!writer.addSelection()) { - if (writer.getNumRows() == 0) { - throw new FrameRowTooLargeException(allocator.capacity()); - } + return !cursor.isDone(); + } - return makeFrame(writer); - } + @Override + public Frame next() + { + if (cursor.isDone()) { + throw new NoSuchElementException(); + } + + try (final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + while (!cursor.isDone()) { + if (!writer.addSelection()) { + if (writer.getNumRows() == 0) { + throw new FrameRowTooLargeException(allocator.capacity()); + } - cursor.advance(); + return makeFrame(writer); + } - if (writer.getNumRows() >= maxRowsPerFrame) { - return makeFrame(writer); - } - } + cursor.advance(); + if (writer.getNumRows() >= maxRowsPerFrame) { return makeFrame(writer); } } - private Frame makeFrame(final FrameWriter writer) - { - return Frame.wrap(writer.toByteArray()); - } - }; + return makeFrame(writer); + } } - @Override - public void cleanup(Iterator iterFromMake) + private Frame makeFrame(final FrameWriter writer) { - // Nothing to do. + return Frame.wrap(writer.toByteArray()); } - } - ) - ); + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // Nothing to do. + } + } + ).withBaggage(cursorAndCloseable.rhs); } } 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 7cd94b4f3972..1ae8561cd602 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 @@ -33,6 +33,7 @@ import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -42,6 +43,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -55,6 +57,7 @@ import org.junit.Assert; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -232,10 +235,14 @@ public static Sequence> readRowsFromFrameChannel( return new FrameChannelSequence(channel) .flatMap( frame -> - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .flatMap(cursor -> readRowsFromCursor(cursor, frameReader.signature())) + readRowsFromCursor( + // if FrameStorageAdapter.asCursorMaker ever needs closing.. this needs to change to add to a + // closer that is tied to baggage of this sequence... + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursor(), + frameReader.signature() + ) ); } @@ -256,9 +263,8 @@ public static Sequence> readRowsFromAdapter( ) { final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature; - return makeCursorsForAdapter(adapter, populateRowNumber).flatMap( - cursor -> readRowsFromCursor(cursor, signatureToUse) - ); + final Pair cursorAndCloseable = makeCursorForAdapter(adapter, populateRowNumber); + return readRowsFromCursor(cursorAndCloseable.lhs, signatureToUse).withBaggage(cursorAndCloseable.rhs); } /** @@ -269,7 +275,7 @@ public static Sequence> readRowsFromAdapter( * @param adapter the adapter * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static Sequence makeCursorsForAdapter( + public static Pair makeCursorForAdapter( final StorageAdapter adapter, final boolean populateRowNumber ) @@ -289,15 +295,14 @@ public static Sequence makeCursorsForAdapter( .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - return adapter.asCursorMaker(buildSpec) - .makeCursors() - .map(cursor -> { - if (populateRowNumber) { - return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn); - } else { - return cursor; - } - }); + + final CursorMaker maker = adapter.asCursorMaker(buildSpec); + final Cursor cursor = maker.makeCursor(); + if (populateRowNumber) { + return new Pair<>(new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn), maker); + } else { + return new Pair<>(cursor, maker); + } } public static Sequence> readRowsFromCursor(final Cursor cursor, final RowSignature signature) diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 3a47434891cc..34cdab25a48b 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -22,7 +22,6 @@ import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.joda.time.DateTime; /** * Used by {@link FrameTestUtil#readRowsFromAdapter} and {@link FrameTestUtil#readRowsFromCursor}. @@ -44,12 +43,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return baseCursor.getColumnSelectorFactory(); } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index a9424a0f5272..9ea17a258027 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -48,7 +48,9 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.RowIdSupplier; @@ -575,49 +577,44 @@ private static Pair writeFrame( inputSegment = new FrameSegment(inputFrame, FrameReader.create(signature), SegmentId.dummy("xxx")); } - return inputSegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .accumulate( - null, - (retVal, cursor) -> { - int numRows = 0; - final FrameWriterFactory frameWriterFactory; - if (FrameType.ROW_BASED.equals(outputFrameType)) { - frameWriterFactory = FrameWriters.makeRowBasedFrameWriterFactory( - new SingleMemoryAllocatorFactory(allocator), - signature, - keyColumns, - false - ); - } else { - frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( - new SingleMemoryAllocatorFactory(allocator), - signature, - keyColumns - ); - } - - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - if (capabilitiesAdjustFn != null) { - columnSelectorFactory = new OverrideCapabilitiesColumnSelectorFactory( - columnSelectorFactory, - capabilitiesAdjustFn - ); - } - - try (final FrameWriter frameWriter = - frameWriterFactory.newFrameWriter(columnSelectorFactory)) { - while (!cursor.isDone() && frameWriter.addSelection()) { - numRows++; - cursor.advance(); - } - - return Pair.of(Frame.wrap(frameWriter.toByteArray()), numRows); - } - } - ); + try (final CursorMaker maker = inputSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + + int numRows = 0; + final FrameWriterFactory frameWriterFactory; + if (FrameType.ROW_BASED.equals(outputFrameType)) { + frameWriterFactory = FrameWriters.makeRowBasedFrameWriterFactory( + new SingleMemoryAllocatorFactory(allocator), + signature, + keyColumns, + false + ); + } else { + frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( + new SingleMemoryAllocatorFactory(allocator), + signature, + keyColumns + ); + } + + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + if (capabilitiesAdjustFn != null) { + columnSelectorFactory = new OverrideCapabilitiesColumnSelectorFactory( + columnSelectorFactory, + capabilitiesAdjustFn + ); + } + + try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + while (!cursor.isDone() && frameWriter.addSelection()) { + numRows++; + cursor.advance(); + } + + return Pair.of(Frame.wrap(frameWriter.toByteArray()), numRows); + } + } } /** diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java new file mode 100644 index 000000000000..f708714cbcef --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -0,0 +1,255 @@ +/* + * 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.query; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.ListBasedInputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class CursorGranularizerTest extends InitializedNullHandlingTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private StorageAdapter adapter; + + @Before + public void setup() throws IOException + { + final RowSignature signature = RowSignature.builder().add("x", ColumnType.STRING).build(); + final List dims = ImmutableList.of("x"); + final IncrementalIndexSchema schema = + IncrementalIndexSchema.builder() + .withDimensionsSpec(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .withQueryGranularity(Granularities.MINUTE) + .build(); + IndexBuilder bob = + IndexBuilder.create() + .schema(schema) + .rows( + ImmutableList.of( + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:00Z"), + dims, + ImmutableList.of("a") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:01Z"), + dims, + ImmutableList.of("b") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:02Z"), + dims, + ImmutableList.of("c") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:03Z"), + dims, + ImmutableList.of("d") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T01:00Z"), + dims, + ImmutableList.of("e") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T01:01Z"), + dims, + ImmutableList.of("f") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T03:04Z"), + dims, + ImmutableList.of("g") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T03:05Z"), + dims, + ImmutableList.of("h") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T03:15Z"), + dims, + ImmutableList.of("i") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T05:03Z"), + dims, + ImmutableList.of("j") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T06:00Z"), + dims, + ImmutableList.of("k") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T09:01Z"), + dims, + ImmutableList.of("l") + ) + ) + ) + .tmpDir(temporaryFolder.newFolder()); + + adapter = new QueryableIndexStorageAdapter(bob.buildMMappedIndex()); + } + + @Test + public void testGranularizeFullScan() + { + try (CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + Granularities.HOUR, + adapter.getInterval(), + false + ); + + final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); + final ColumnValueSelector xSelector = selectorFactory.makeColumnValueSelector("x"); + final Sequence> theSequence = + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + List bucket = new ArrayList<>(); + if (!granularizer.advanceToBucket(bucketInterval)) { + return bucket; + } + while (!cursor.isDone()) { + bucket.add((String) xSelector.getObject()); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } + } + return bucket; + }); + + List> granularized = theSequence.toList(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("a", "b", "c", "d"), + ImmutableList.of("e", "f"), + ImmutableList.of(), + ImmutableList.of("g", "h", "i"), + ImmutableList.of(), + ImmutableList.of("j"), + ImmutableList.of("k"), + ImmutableList.of(), + ImmutableList.of(), + ImmutableList.of("l") + ), + granularized + ); + } + } + + @Test + public void testGranularizeFullScanDescending() + { + final CursorBuildSpec descending = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .isDescending(true) + .build(); + try (CursorMaker maker = adapter.asCursorMaker(descending)) { + final Cursor cursor = maker.makeCursor(); + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + Granularities.HOUR, + adapter.getInterval(), + true + ); + + final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); + final ColumnValueSelector xSelector = selectorFactory.makeColumnValueSelector("x"); + final Sequence> theSequence = + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + List bucket = new ArrayList<>(); + if (!granularizer.advanceToBucket(bucketInterval)) { + return bucket; + } + while (!cursor.isDone()) { + bucket.add((String) xSelector.getObject()); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } + } + return bucket; + }); + + List> granularized = theSequence.toList(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("l"), + ImmutableList.of(), + ImmutableList.of(), + ImmutableList.of("k"), + ImmutableList.of("j"), + ImmutableList.of(), + ImmutableList.of("i", "h", "g"), + ImmutableList.of(), + ImmutableList.of("f", "e"), + ImmutableList.of("d", "c", "b", "a") + ), + granularized + ); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index ea4af2ffeced..79a0ae80875b 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -662,7 +662,7 @@ private void runResults( final CursorMaker maker = s.asStorageAdapter() .asCursorMaker(spec); final boolean canVectorize = maker.canVectorize(); - maker.cleanup(); + maker.close(); return canVectorize; }); diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java index aa369dc63777..486ef7e20293 100644 --- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java @@ -25,11 +25,11 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.RowBasedStorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -192,37 +192,31 @@ public void test_asStorageAdapter_getDimensionCardinalityV() @Test public void test_asStorageAdapter_makeCursors() { - final Sequence cursors = LOOKUP_SEGMENT.asStorageAdapter().asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970/PT1H")) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final List> kvs = new ArrayList<>(); - - cursors.accumulate( - null, - (ignored, cursor) -> { - final ColumnValueSelector keySelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("k"); - final ColumnValueSelector valueSelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - - while (!cursor.isDone()) { - kvs.add(Pair.of(String.valueOf(keySelector.getObject()), String.valueOf(valueSelector.getObject()))); - cursor.advanceUninterruptibly(); - } - - return null; - } - ); - - Assert.assertEquals( - ImmutableList.of( - Pair.of("a", "b"), - Pair.of("x", "y") - ), - kvs - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970/PT1H")) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = LOOKUP_SEGMENT.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + final List> kvs = new ArrayList<>(); + + final ColumnValueSelector keySelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("k"); + final ColumnValueSelector valueSelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + + while (!cursor.isDone()) { + kvs.add(Pair.of(String.valueOf(keySelector.getObject()), String.valueOf(valueSelector.getObject()))); + cursor.advanceUninterruptibly(); + } + + Assert.assertEquals( + ImmutableList.of( + Pair.of("a", "b"), + Pair.of("x", "y") + ), + kvs + ); + } } @Test diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java index 367c29653473..b9fc63db8808 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.groupby.ResultRow; @@ -38,6 +37,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.TypeStrategy; @@ -256,22 +256,16 @@ private void validateDecorated( if (interval != null) { builder.setInterval(interval); } - final Sequence cursors = seggy - .asStorageAdapter() - .asCursorMaker(builder.build()) - .makeCursors(); - - vals = cursors.accumulate( - new ArrayList<>(), - (accumulated, in) -> { - final ColumnValueSelector idSupplier = in.getColumnSelectorFactory().makeColumnValueSelector("arrayIndex"); - while (!in.isDone()) { - accumulated.add(originalVals[(int) idSupplier.getLong()]); - in.advance(); - } - return accumulated; - } - ); + try (final CursorMaker maker = seggy.asStorageAdapter().asCursorMaker(builder.build())) { + final Cursor cursor = maker.makeCursor(); + + vals = new ArrayList<>(); + final ColumnValueSelector idSupplier = cursor.getColumnSelectorFactory().makeColumnValueSelector("arrayIndex"); + while (!cursor.isDone()) { + vals.add(originalVals[(int) idSupplier.getLong()]); + cursor.advance(); + } + } } if (ordering != null) { diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index b1b16103a77b..fbac92a4adac 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -669,7 +669,7 @@ private void runResults( final CursorMaker maker = s.asStorageAdapter() .asCursorMaker(spec); final boolean canVectorize = maker.canVectorize(); - maker.cleanup(); + maker.close(); return canVectorize; }); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index f68104ff4815..16a7b7066af0 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -24,23 +24,20 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; import org.apache.druid.query.filter.DruidPredicateFactory; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; @@ -321,16 +318,17 @@ public Metadata getMetadata() } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return null; + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return null; + } + }; } }; diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index ff0f4e22133b..2f5af7240e86 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; @@ -45,7 +44,6 @@ import org.junit.Test; import javax.annotation.Nonnull; -import java.util.List; import java.util.Map; public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest @@ -150,51 +148,44 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("b", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("b", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("c", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("c", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(STRING_COL).toColumnType()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("b", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("b", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("c", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("c", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(STRING_COL).toColumnType()); + } } @Test @@ -210,81 +201,74 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1L, valueSelector.getObject()); - Assert.assertEquals(1L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertEquals(2L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3L, valueSelector.getObject()); - Assert.assertEquals(3L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1L, valueSelector.getObject()); + Assert.assertEquals(1L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); - } - + Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertEquals(2L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3L, valueSelector.getObject()); + Assert.assertEquals(3L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } + + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } + Assert.assertEquals(ColumnType.LONG, storageAdapter.getColumnCapabilities(LONG_COL).toColumnType()); } - Assert.assertEquals(ColumnType.LONG, storageAdapter.getColumnCapabilities(LONG_COL).toColumnType()); } @Test @@ -300,80 +284,74 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1.1, valueSelector.getObject()); - Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1.1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2.2, valueSelector.getObject()); - Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2.2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1.1, valueSelector.getObject()); + Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); - } + Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1.1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + + cursor.advance(); + Assert.assertEquals(2.2, valueSelector.getObject()); + Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2.2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } + Assert.assertEquals(ColumnType.DOUBLE, storageAdapter.getColumnCapabilities(DOUBLE_COL).toColumnType()); } - Assert.assertEquals(ColumnType.DOUBLE, storageAdapter.getColumnCapabilities(DOUBLE_COL).toColumnType()); } @Test @@ -389,50 +367,39 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec( + STRING_ARRAY_COL, + STRING_ARRAY_COL, + ColumnType.STRING + ); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(ColumnType.STRING_ARRAY, storageAdapter.getColumnCapabilities(STRING_ARRAY_COL).toColumnType()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals( + ColumnType.STRING_ARRAY, + storageAdapter.getColumnCapabilities(STRING_ARRAY_COL).toColumnType() + ); + } } @Test @@ -448,43 +415,35 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(VARIANT_COL).toColumnType()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + DimensionSelector dimensionSelector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(VARIANT_COL).toColumnType()); + } } @Test @@ -500,50 +459,31 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(ColumnType.NESTED_DATA, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(ColumnType.NESTED_DATA, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + } } @Test @@ -578,42 +518,35 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("2", valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + DimensionSelector dimensionSelector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("2", valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + } } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index fc1eba743d4f..2ed9e8431501 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -26,7 +26,6 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.List; @@ -189,12 +188,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - throw new UnsupportedOperationException(); - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index bedf83c20dad..ce18715d1c2c 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; @@ -44,7 +43,6 @@ import org.junit.Test; import javax.annotation.Nonnull; -import java.util.List; import java.util.Map; public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest @@ -150,49 +148,42 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("b", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("b", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("c", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("c", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("b", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("b", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("c", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("c", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } } @Test @@ -208,78 +199,71 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1L, valueSelector.getObject()); - Assert.assertEquals(1L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertEquals(2L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3L, valueSelector.getObject()); - Assert.assertEquals(3L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1L, valueSelector.getObject()); + Assert.assertEquals(1L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); - } + Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertEquals(2L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3L, valueSelector.getObject()); + Assert.assertEquals(3L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } } } @@ -296,78 +280,71 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1.1, valueSelector.getObject()); - Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1.1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2.2, valueSelector.getObject()); - Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2.2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1.1, valueSelector.getObject()); + Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); - } + Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1.1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + cursor.advance(); + Assert.assertEquals(2.2, valueSelector.getObject()); + Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2.2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } } } @@ -384,49 +361,34 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec( + STRING_ARRAY_COL, + STRING_ARRAY_COL, + ColumnType.STRING + ); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + } } @Test @@ -442,41 +404,34 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + DimensionSelector dimensionSelector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + } } @Test @@ -492,49 +447,30 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + } } @Nonnull diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java index 1f5c18fdd056..f4b4d41145e7 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java @@ -21,9 +21,6 @@ import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -86,9 +83,8 @@ public void setUp() partialNullSelector = columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column")); } else { - final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - final Yielder yielder = closer.register(Yielders.each(cursors)); - final Cursor cursor = yielder.get(); + final CursorMaker maker = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)); + final Cursor cursor = maker.makeCursor(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); qualitySelector = @@ -232,11 +228,9 @@ public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - final Yielder cursorYielder = Yielders.each(cursors); - cursor = cursorYielder.get(); + final CursorMaker maker = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)); + cursor = maker.makeCursor(); columnSelectorFactory = cursor.getColumnSelectorFactory(); - closer.register(cursorYielder); } @After diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 2382b6d0e071..d12ca72d638f 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; @@ -31,6 +32,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -49,6 +51,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import java.util.function.ToLongFunction; @@ -79,10 +82,10 @@ public class RowBasedStorageAdapterTest } ); - private static final List>> READ_TIME_AND_STRING = + private static final List>> READ_TIME_AND_STRING_GRAN = ImmutableList.of( - cursor -> cursor::getTime, - cursor -> { + (cursor, granularizer) -> granularizer::getBucketStart, + (cursor, granularizer) -> { final BaseObjectColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector(ValueType.STRING.name()); return selector::getObject; @@ -99,11 +102,6 @@ public static void setUpClass() PROCESSORS.clear(); - PROCESSORS.put( - "cursor-time", - cursor -> cursor::getTime - ); - // Read all the types as all the other types. for (final String valueTypeName : ROW_SIGNATURE.getColumnNames()) { @@ -456,19 +454,21 @@ public void test_makeCursors_filterOnLong() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); Assert.assertEquals(1, numCloses.get()); } @@ -478,20 +478,21 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("0"), - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("0"), + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -501,31 +502,32 @@ public void test_makeCursors_filterOnVirtualColumn() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) - .setVirtualColumns( - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "vc", - "\"LONG\" + 1", - ColumnType.LONG, - ExprMacroTable.nil() - ) - ) - ) - ) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) + .setVirtualColumns( + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "vc", + "\"LONG\" + 1", + ColumnType.LONG, + ExprMacroTable.nil() + ) + ) + ) + ) + .setGranularity(Granularities.ALL) + .build(); + + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -535,18 +537,18 @@ public void test_makeCursors_descending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("2"), - ImmutableList.of("1"), - ImmutableList.of("0") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("2"), + ImmutableList.of("1"), + ImmutableList.of("0") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -556,17 +558,17 @@ public void test_makeCursors_intervalDoesNotMatch() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("2000/P1D")) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of(), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("2000/P1D")) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of(), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -576,19 +578,19 @@ public void test_makeCursors_intervalPartiallyMatches() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970-01-01T01/PT1H")) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT1H")) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -598,24 +600,23 @@ public void test_makeCursors_hourGranularity() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970/1971")) - .setGranularity(Granularities.HOUR) - .build() - ).makeCursors(); - - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(DateTimes.of("1970-01-01T00"), "0"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), - ImmutableList.of(DateTimes.of("1970-01-01T03"), "3") - ), - walkCursors(cursors, READ_TIME_AND_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970/1971")) + .setGranularity(Granularities.HOUR) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(DateTimes.of("1970-01-01T00"), "0"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), + ImmutableList.of(DateTimes.of("1970-01-01T03"), "3") + ), + walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -625,21 +626,23 @@ public void test_makeCursors_hourGranularityWithInterval() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970-01-01T01/PT2H")) - .setGranularity(Granularities.HOUR) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T02"), "2") - ), - walkCursors(cursors, READ_TIME_AND_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setGranularity(Granularities.HOUR) + .build(); + + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T02"), "2") + ), + walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -649,22 +652,23 @@ public void test_makeCursors_hourGranularityWithIntervalDescending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970-01-01T01/PT2H")) - .setGranularity(Granularities.HOUR) - .isDescending(true) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1") - ), - walkCursors(cursors, READ_TIME_AND_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setGranularity(Granularities.HOUR) + .isDescending(true) + .build(); + + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1") + ), + walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -674,103 +678,102 @@ public void test_makeCursors_allProcessors() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - Lists.newArrayList( - Intervals.ETERNITY.getStart(), - - // FLOAT - 0f, - 0d, - 0L, - "0.0", - 0f, - - // DOUBLE - 0f, - 0d, - 0L, - "0.0", - 0d, - - // LONG - 0f, - 0d, - 0L, - "0", - 0L, - - // STRING - 0f, - 0d, - 0L, - "0", - "0", - - // COMPLEX - NullHandling.defaultFloatValue(), - NullHandling.defaultDoubleValue(), - NullHandling.defaultLongValue(), - null, - null, - - // unknownType - 0f, - 0d, - 0L, - "0", - 0 - ), - Lists.newArrayList( - Intervals.ETERNITY.getStart(), - - // FLOAT - 1f, - 1d, - 1L, - "1.0", - 1f, - - // DOUBLE - 1f, - 1d, - 1L, - "1.0", - 1d, - - // LONG - 1f, - 1d, - 1L, - "1", - 1L, - - // STRING - 1f, - 1d, - 1L, - "1", - "1", - - // COMPLEX - NullHandling.defaultFloatValue(), - NullHandling.defaultDoubleValue(), - NullHandling.defaultLongValue(), - null, - null, - - // unknownType - 1f, - 1d, - 1L, - "1", - 1 - ) - ), - walkCursors(cursors, new ArrayList<>(PROCESSORS.values())) - ); + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + Lists.newArrayList( + + // FLOAT + 0f, + 0d, + 0L, + "0.0", + 0f, + + // DOUBLE + 0f, + 0d, + 0L, + "0.0", + 0d, + + // LONG + 0f, + 0d, + 0L, + "0", + 0L, + + // STRING + 0f, + 0d, + 0L, + "0", + "0", + + // COMPLEX + NullHandling.defaultFloatValue(), + NullHandling.defaultDoubleValue(), + NullHandling.defaultLongValue(), + null, + null, + + // unknownType + 0f, + 0d, + 0L, + "0", + 0 + ), + Lists.newArrayList( + + // FLOAT + 1f, + 1d, + 1L, + "1.0", + 1f, + + // DOUBLE + 1f, + 1d, + 1L, + "1.0", + 1d, + + // LONG + 1f, + 1d, + 1L, + "1", + 1L, + + // STRING + 1f, + 1d, + 1L, + "1", + "1", + + // COMPLEX + NullHandling.defaultFloatValue(), + NullHandling.defaultDoubleValue(), + NullHandling.defaultLongValue(), + null, + null, + + // unknownType + 1f, + 1d, + 1L, + "1", + 1 + ) + ), + walkCursor(cursor, new ArrayList<>(PROCESSORS.values())) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -780,17 +783,18 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNonnull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) + .setGranularity(Granularities.ALL) + .build(); - Assert.assertEquals( - ImmutableList.of(), - walkCursors(cursors, new ArrayList<>(PROCESSORS.values())) - ); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of(), + walkCursor(cursor, new ArrayList<>(PROCESSORS.values())) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -804,38 +808,78 @@ public void test_makeCursors_eternityIntervalWithMonthGranularity() CursorBuildSpec.builder() .setGranularity(Granularities.MONTH) .build() - ).makeCursors(); + ).makeCursor(); }); } - private static List> walkCursors( - final Sequence cursors, + private static List> walkCursor( + final Cursor cursor, final List>> processors ) { - return cursors.flatMap( - cursor -> { - // Gather test-value suppliers together. - final List> suppliers = new ArrayList<>(); - for (Function> processor : processors) { - suppliers.add(processor.apply(cursor)); - } + final List> suppliers = new ArrayList<>(); + for (Function> processor : processors) { + suppliers.add(processor.apply(cursor)); + } - final List> retVal = new ArrayList<>(); + final List> retVal = new ArrayList<>(); - while (!cursor.isDone()) { - final List row = new ArrayList<>(); + while (!cursor.isDone()) { + final List row = new ArrayList<>(); - for (Supplier supplier : suppliers) { - row.add(supplier.get()); - } + for (Supplier supplier : suppliers) { + row.add(supplier.get()); + } - retVal.add(row); - cursor.advanceUninterruptibly(); - } + retVal.add(row); + cursor.advanceUninterruptibly(); + } - return Sequences.simple(retVal); - } - ).toList(); + return retVal; + } + + private static List> walkCursorGranularized( + final StorageAdapter adapter, + final Cursor cursor, + final CursorBuildSpec buildSpec, + final List>> processors + ) + { + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + buildSpec.getGranularity(), + buildSpec.getInterval(), + buildSpec.isDescending() + ); + + final List> suppliers = new ArrayList<>(); + for (BiFunction> processor : processors) { + suppliers.add(processor.apply(cursor, granularizer)); + } + + final Sequence> theSequence = + Sequences.simple(granularizer.getBucketIterable()) + .flatMap(bucketInterval -> { + if (!granularizer.advanceToBucket(bucketInterval)) { + return Sequences.empty(); + } + final List> retVal = new ArrayList<>(); + while (!cursor.isDone()) { + final List row = new ArrayList<>(); + + for (Supplier supplier : suppliers) { + row.add(supplier.get()); + } + + retVal.add(row); + if (!granularizer.advanceCursorWithinBucketUninterruptedly()) { + break; + } + } + return Sequences.simple(retVal); + }) + .filter(Predicates.notNull()); + return theSequence.toList(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java index c93abe864f16..7d51b9e46e69 100644 --- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java @@ -19,10 +19,6 @@ package org.apache.druid.segment; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; import org.joda.time.DateTime; @@ -41,15 +37,17 @@ public void testTombstoneDefaultInterface() StorageAdapter sa = new StorageAdapter() { @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return null; + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return null; + } + }; } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index d171ea21b2d4..4667ffe0a36e 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.NestedDataTestUtils; @@ -229,14 +228,13 @@ public void test_unnest_adapter_column_capabilities() @Test public void test_unnest_adapters_basic() { - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(UNNEST_STORAGE_ADAPTER.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(UNNEST_STORAGE_ADAPTER.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = UNNEST_STORAGE_ADAPTER.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); @@ -254,22 +252,18 @@ public void test_unnest_adapters_basic() unnest 2 rows -> 16 rows after unnest */ Assert.assertEquals(count, 16); - return null; - }); - + } } @Test public void test_two_levels_of_unnest_adapters() { - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER1.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = UNNEST_STORAGE_ADAPTER1.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME1)); @@ -295,8 +289,7 @@ public void test_two_levels_of_unnest_adapters() */ Assert.assertEquals(count, 128); Assert.assertEquals(dimSelector.getValueCardinality(), 17); - return null; - }); + } } @Test @@ -322,25 +315,22 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest selector(inputColumn, "2") )); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(baseFilter) - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); - - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(baseFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); + + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); // OR-case so base filter should match the postJoinFilter Assert.assertEquals(baseFilter, postFilter); - return null; - }); + } } @Test @@ -372,25 +362,22 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap )) )); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(baseFilter) - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); - - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(baseFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); + + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); // OR-case so base filter should match the postJoinFilter Assert.assertEquals(baseFilter, postFilter); - return null; - }); + } } @Test @@ -655,19 +642,17 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() final Filter expectedPushDownFilter = selector(inputColumn, "1"); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); Assert.assertEquals(unnestStorageAdapter.getUnnestFilter(), postFilter); @@ -678,8 +663,7 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() count++; } Assert.assertEquals(1, count); - return null; - }); + } } @@ -700,19 +684,18 @@ public void test_pushdown_filters_unnested_dimension_outside() selector(inputColumn, "1"); final Filter queryFilter = new SelectorFilter(OUTPUT_COLUMN_NAME, "1", null); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(queryFilter) - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); - - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(queryFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); + + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); Assert.assertEquals(queryFilter, postFilter); @@ -723,8 +706,7 @@ public void test_pushdown_filters_unnested_dimension_outside() count++; } Assert.assertEquals(1, count); - return null; - }); + } } @Test @@ -751,14 +733,12 @@ public void testUnnestValueMatcherValueDoesntExist() new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), null ); - Sequence cursorSequence = withNullsStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(withNullsStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(withNullsStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = withNullsStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); @@ -776,9 +756,7 @@ public void testUnnestValueMatcherValueDoesntExist() count++; } Assert.assertEquals(count, 618); - return null; - }); - + } } public void testComputeBaseAndPostUnnestFilters( 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 082c6cfcf451..f232b70b82ba 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 @@ -47,8 +47,6 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprType; import org.apache.druid.math.expr.ExpressionType; @@ -72,6 +70,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; @@ -792,29 +791,33 @@ private DimFilter maybeOptimize(final DimFilter dimFilter) return optimize ? dimFilter.optimize(false) : dimFilter; } + private CursorBuildSpec makeCursorBuildSpec(@Nullable Filter filter) + { + return CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(VIRTUAL_COLUMNS) + .setGranularity(Granularities.ALL) + .build(); + + } - private Sequence makeCursorSequence(final Filter filter) + private CursorBuildSpec makeVectorCursorBuildSpec(@Nullable Filter filter) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setVirtualColumns(VIRTUAL_COLUMNS) - .setGranularity(Granularities.ALL) - .build(); - return adapter.asCursorMaker(buildSpec).makeCursors(); + return CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(virtualColumns) + .setGranularity(Granularities.ALL) + .setQueryContext( + QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) + ) + ) + .build(); } private VectorCursor makeVectorCursor(final Filter filter) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setVirtualColumns(virtualColumns) - .setGranularity(Granularities.ALL) - .setQueryContext( - QueryContext.of( - ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) - ) - ) - .build(); + final CursorBuildSpec buildSpec = makeVectorCursorBuildSpec(filter); return adapter.asCursorMaker(buildSpec).makeVectorCursor(); } @@ -823,48 +826,39 @@ private VectorCursor makeVectorCursor(final Filter filter) */ private List selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn) { - final Sequence cursors = makeCursorSequence(makeFilter(filter)); - Sequence> seq = Sequences.map( - cursors, - cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - - final List values = new ArrayList<>(); - - while (!cursor.isDone()) { - IndexedInts row = selector.getRow(); - Preconditions.checkState(row.size() == 1); - values.add(selector.lookupName(row.get(0))); - cursor.advance(); - } + try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(makeFilter(filter)))) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - return values; - } - ); - return seq.toList().get(0); + final List values = new ArrayList<>(); + + while (!cursor.isDone()) { + IndexedInts row = selector.getRow(); + Preconditions.checkState(row.size() == 1); + values.add(selector.lookupName(row.get(0))); + cursor.advance(); + } + return values; + } } private long selectCountUsingFilteredAggregator(final DimFilter filter) { - final Sequence cursors = makeCursorSequence(null); - Sequence aggSeq = Sequences.map( - cursors, - cursor -> { - Aggregator agg = new FilteredAggregatorFactory( - new CountAggregatorFactory("count"), - maybeOptimize(filter) - ).factorize(cursor.getColumnSelectorFactory()); - - for (; !cursor.isDone(); cursor.advance()) { - agg.aggregate(); - } + try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(null))) { + final Cursor cursor = maker.makeCursor(); + Aggregator agg = new FilteredAggregatorFactory( + new CountAggregatorFactory("count"), + maybeOptimize(filter) + ).factorize(cursor.getColumnSelectorFactory()); - return agg; - } - ); - return aggSeq.toList().get(0).getLong(); + for (; !cursor.isDone(); cursor.advance()) { + agg.aggregate(); + } + + return agg.getLong(); + } } private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFilter) @@ -875,7 +869,9 @@ private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFil dimFilter ); - try (final VectorCursor cursor = makeVectorCursor(null)) { + + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(null)); + final VectorCursor cursor = maker.makeVectorCursor()) { final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), maybeOptimize(dimFilter) @@ -941,27 +937,23 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - final Sequence cursors = makeCursorSequence(postFilteringFilter); - Sequence> seq = Sequences.map( - cursors, - cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - - final List values = new ArrayList<>(); - - while (!cursor.isDone()) { - IndexedInts row = selector.getRow(); - Preconditions.checkState(row.size() == 1); - values.add(selector.lookupName(row.get(0))); - cursor.advance(); - } + try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(postFilteringFilter))) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - return values; - } - ); - return seq.toList().get(0); + final List values = new ArrayList<>(); + + while (!cursor.isDone()) { + IndexedInts row = selector.getRow(); + Preconditions.checkState(row.size() == 1); + values.add(selector.lookupName(row.get(0))); + cursor.advance(); + } + + return values; + } } private List selectColumnValuesMatchingFilterUsingVectorizedPostFiltering( @@ -1005,7 +997,8 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - try (final VectorCursor cursor = makeVectorCursor(postFilteringFilter)) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(postFilteringFilter)); + final VectorCursor cursor = maker.makeVectorCursor()) { final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1029,7 +1022,8 @@ private List selectColumnValuesMatchingFilterUsingVectorCursor( final String selectColumn ) { - try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter))); + final VectorCursor cursor = maker.makeVectorCursor()) { final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1055,7 +1049,8 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur ) { final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); - try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter))); + final VectorCursor cursor = maker.makeVectorCursor()) { final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); final List values = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index c620917eb01e..9521257c25b1 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -149,11 +149,13 @@ public void testSanity() throws Exception .addOrderByColumn("billy") .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); final Sequence rows = GroupByQueryEngine.process( query, adapter, maker, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), @@ -223,11 +225,13 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception .addOrderByColumn("billy") .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); final Sequence rows = GroupByQueryEngine.process( query, adapter, maker, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), @@ -280,31 +284,31 @@ public void testResetSanity() throws IOException .setInterval(interval) .isDescending(descending) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - Cursor cursor = cursorSequence.limit(1).toList().get(0); - DimensionSelector dimSelector; - - dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); - - index.add( - new MapBasedInputRow( - t.minus(1).getMillis(), - Collections.singletonList("sally"), - ImmutableMap.of("sally", "ah") - ) - ); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + DimensionSelector dimSelector; + + dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + + index.add( + new MapBasedInputRow( + t.minus(1).getMillis(), + Collections.singletonList("sally"), + ImmutableMap.of("sally", "ah") + ) + ); - // Cursor reset should not be affected by out of order values - cursor.reset(); + // Cursor reset should not be affected by out of order values + cursor.reset(); - dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + } } } @@ -387,11 +391,13 @@ public void testFilterByNull() throws Exception .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); final Sequence rows = GroupByQueryEngine.process( query, adapter, maker, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), @@ -429,41 +435,38 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .setGranularity(Granularities.ALL) .build(); - Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); - - cursors - .map(cursor -> { - DimensionSelector dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - int cardinality = dimSelector.getValueCardinality(); - - //index gets more rows at this point, while other thread is iterating over the cursor - try { - for (int i = 0; i < 1; i++) { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2" + i))); - } - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - int rowNumInCursor = 0; - // and then, cursoring continues in the other thread - while (!cursor.isDone()) { - IndexedInts row = dimSelector.getRow(); - row.forEach(i -> Assert.assertTrue(i < cardinality)); - cursor.advance(); - rowNumInCursor++; - } - Assert.assertEquals(2, rowNumInCursor); - assertCursorsNotEmpty.incrementAndGet(); - - return null; - }) - .toList(); - Assert.assertEquals(1, assertCursorsNotEmpty.get()); + try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); + DimensionSelector dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + int cardinality = dimSelector.getValueCardinality(); + + //index gets more rows at this point, while other thread is iterating over the cursor + try { + for (int i = 0; i < 1; i++) { + index.add(new MapBasedInputRow( + timestamp, + Collections.singletonList("billy"), + ImmutableMap.of("billy", "v2" + i) + )); + } + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + int rowNumInCursor = 0; + // and then, cursoring continues in the other thread + while (!cursor.isDone()) { + IndexedInts row = dimSelector.getRow(); + row.forEach(i -> Assert.assertTrue(i < cardinality)); + cursor.advance(); + rowNumInCursor++; + } + Assert.assertEquals(2, rowNumInCursor); + } } @Test @@ -491,30 +494,23 @@ public void testCursorDictionaryRaceConditionFix() throws Exception .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .setGranularity(Granularities.ALL) .build(); - Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); - - cursors - .map(cursor -> { - DimensionSelector dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - int cardinality = dimSelector.getValueCardinality(); - - int rowNumInCursor = 0; - while (!cursor.isDone()) { - IndexedInts row = dimSelector.getRow(); - row.forEach(i -> Assert.assertTrue(i < cardinality)); - cursor.advance(); - rowNumInCursor++; - } - Assert.assertEquals(5, rowNumInCursor); - assertCursorsNotEmpty.incrementAndGet(); - - return null; - }) - .toList(); - Assert.assertEquals(1, assertCursorsNotEmpty.get()); + try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); + DimensionSelector dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + int cardinality = dimSelector.getValueCardinality(); + + int rowNumInCursor = 0; + while (!cursor.isDone()) { + IndexedInts row = dimSelector.getRow(); + row.forEach(i -> Assert.assertTrue(i < cardinality)); + cursor.advance(); + rowNumInCursor++; + } + Assert.assertEquals(5, rowNumInCursor); + } } @Test @@ -539,86 +535,84 @@ public void testCursoringAndSnapshot() throws Exception .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .setGranularity(Granularities.ALL) .build(); - Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); - - cursors - .map(cursor -> { - DimensionSelector dimSelector1A = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - int cardinalityA = dimSelector1A.getValueCardinality(); - - //index gets more rows at this point, while other thread is iterating over the cursor - try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1"))); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - DimensionSelector dimSelector1B = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - //index gets more rows at this point, while other thread is iterating over the cursor - try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2"))); - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy2"), ImmutableMap.of("billy2", "v3"))); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - DimensionSelector dimSelector1C = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - - DimensionSelector dimSelector2D = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2")); - //index gets more rows at this point, while other thread is iterating over the cursor - try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3"))); - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", ""))); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - DimensionSelector dimSelector3E = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy3", "billy3")); - - int rowNumInCursor = 0; - // and then, cursoring continues in the other thread - while (!cursor.isDone()) { - IndexedInts rowA = dimSelector1A.getRow(); - rowA.forEach(i -> Assert.assertTrue(i < cardinalityA)); - IndexedInts rowB = dimSelector1B.getRow(); - rowB.forEach(i -> Assert.assertTrue(i < cardinalityA)); - IndexedInts rowC = dimSelector1C.getRow(); - rowC.forEach(i -> Assert.assertTrue(i < cardinalityA)); - IndexedInts rowD = dimSelector2D.getRow(); - // no null id, so should get empty dims array - Assert.assertEquals(0, rowD.size()); - IndexedInts rowE = dimSelector3E.getRow(); - if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(1, rowE.size()); - // the null id - Assert.assertEquals(0, rowE.get(0)); - } else { - Assert.assertEquals(0, rowE.size()); - } - cursor.advance(); - rowNumInCursor++; - } - Assert.assertEquals(2, rowNumInCursor); - assertCursorsNotEmpty.incrementAndGet(); - - return null; - }) - .toList(); - Assert.assertEquals(1, assertCursorsNotEmpty.get()); + try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); + + DimensionSelector dimSelector1A = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + int cardinalityA = dimSelector1A.getValueCardinality(); + + //index gets more rows at this point, while other thread is iterating over the cursor + try { + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1"))); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + DimensionSelector dimSelector1B = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + //index gets more rows at this point, while other thread is iterating over the cursor + try { + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2"))); + index.add(new MapBasedInputRow( + timestamp, + Collections.singletonList("billy2"), + ImmutableMap.of("billy2", "v3") + )); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + DimensionSelector dimSelector1C = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + + DimensionSelector dimSelector2D = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2")); + //index gets more rows at this point, while other thread is iterating over the cursor + try { + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3"))); + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", ""))); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + DimensionSelector dimSelector3E = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy3", "billy3")); + + int rowNumInCursor = 0; + // and then, cursoring continues in the other thread + while (!cursor.isDone()) { + IndexedInts rowA = dimSelector1A.getRow(); + rowA.forEach(i -> Assert.assertTrue(i < cardinalityA)); + IndexedInts rowB = dimSelector1B.getRow(); + rowB.forEach(i -> Assert.assertTrue(i < cardinalityA)); + IndexedInts rowC = dimSelector1C.getRow(); + rowC.forEach(i -> Assert.assertTrue(i < cardinalityA)); + IndexedInts rowD = dimSelector2D.getRow(); + // no null id, so should get empty dims array + Assert.assertEquals(0, rowD.size()); + IndexedInts rowE = dimSelector3E.getRow(); + if (NullHandling.replaceWithDefault()) { + Assert.assertEquals(1, rowE.size()); + // the null id + Assert.assertEquals(0, rowE.get(0)); + } else { + Assert.assertEquals(0, rowE.size()); + } + cursor.advance(); + rowNumInCursor++; + } + Assert.assertEquals(2, rowNumInCursor); + } } private static class DictionaryRaceTestFilter implements Filter diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index 31de72b1110d..24becbd4a298 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -306,12 +306,12 @@ public void test_makeCursors_factToCountryLeft() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -363,12 +363,12 @@ public void test_makeCursors_factToCountryLeftUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -418,12 +418,12 @@ public void test_makeCursors_factToCountryInner() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -468,12 +468,12 @@ public void test_makeCursors_factToCountryInnerUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -520,14 +520,14 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -580,14 +580,14 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -636,14 +636,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -667,14 +667,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -697,14 +697,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -730,14 +730,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -762,14 +762,14 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -795,14 +795,14 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -832,14 +832,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -869,14 +869,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -906,14 +906,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -943,14 +943,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -993,14 +993,14 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1064,14 +1064,14 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1118,12 +1118,12 @@ public void test_makeCursors_factToRegionToCountryLeft() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1175,12 +1175,12 @@ public void test_makeCursors_factToRegionToCountryInnerIncludeNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1241,14 +1241,14 @@ public void test_makeCursors_factToCountryAlwaysTrue() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1300,14 +1300,14 @@ public void test_makeCursors_factToCountryAlwaysFalse() VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1341,14 +1341,14 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1400,14 +1400,14 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1445,14 +1445,14 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() joinableClauses, virtualColumns ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1496,14 +1496,14 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() joinableClauses, virtualColumns ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1544,12 +1544,12 @@ public void test_makeCursors_factToCountryUsingExpression() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1589,12 +1589,12 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1636,14 +1636,14 @@ public void test_makeCursors_factToRegionTheWrongWay() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "regionIsoCode", @@ -1687,12 +1687,12 @@ public void test_makeCursors_errorOnNonEquiJoin() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1722,12 +1722,12 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1757,12 +1757,12 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1792,12 +1792,12 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1813,14 +1813,14 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1842,14 +1842,14 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1877,7 +1877,7 @@ public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowIS joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors().toList(); + ); } @Test @@ -1895,13 +1895,13 @@ public void test_makeCursors_factToCountryLeftWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1932,13 +1932,13 @@ public void test_makeCursors_factToCountryInnerWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1968,13 +1968,13 @@ public void test_makeCursors_factToCountryRightWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -2021,13 +2021,13 @@ public void test_makeCursors_factToCountryFullWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", 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 8e6ceef8fc40..4d5966662ba7 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 @@ -82,10 +82,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -154,10 +154,10 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -205,10 +205,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -267,10 +267,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -333,10 +333,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -398,14 +398,14 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) .setGranularity(Granularities.ALL) .build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -471,14 +471,14 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) .setGranularity(Granularities.ALL) .build() - ).makeCursors(), + ), ImmutableList.of( "page", "v0" @@ -574,10 +574,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -696,10 +696,10 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -785,10 +785,10 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan joinableClauses, joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -830,10 +830,10 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -927,10 +927,10 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1012,10 +1012,10 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1081,10 +1081,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1134,10 +1134,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1186,10 +1186,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1237,10 +1237,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1289,10 +1289,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1341,10 +1341,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1400,10 +1400,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1456,10 +1456,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1506,10 +1506,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1555,10 +1555,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1608,10 +1608,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1660,10 +1660,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1711,10 +1711,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1773,10 +1773,10 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1839,10 +1839,10 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1915,10 +1915,10 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR + "Equality{leftExpr=user, rightColumn='regionName', includeNull=false}" ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1971,10 +1971,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2046,10 +2046,10 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2244,10 +2244,10 @@ public boolean supportsRequiredColumnRewrite() expectedVirtualColumns = ImmutableSet.of(); } - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2373,10 +2373,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2448,10 +2448,10 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 0746c1047325..c6edcf1b7cd6 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -34,8 +34,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -48,6 +46,7 @@ import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; @@ -325,47 +324,49 @@ public static RowBasedIndexedTable> createRegionsIndexedTabl ); } - public static List readCursors(final Sequence cursors, final List columns) + public static List readCursor(final CursorMaker cursorMaker, final List columns) { - return cursors.flatMap( - cursor -> { - final List> readers = columns - .stream() - .map( - column -> - ColumnProcessors.makeProcessor( - column, - SIMPLE_READER, - cursor.getColumnSelectorFactory() - ) - ) - .collect(Collectors.toList()); - - final List rows = new ArrayList<>(); - - while (!cursor.isDone()) { - final Object[] row = new Object[columns.size()]; - - for (int i = 0; i < row.length; i++) { - row[i] = readers.get(i).get(); - } - - rows.add(row); - cursor.advance(); - } + try { + final Cursor cursor = cursorMaker.makeCursor(); + final List> readers = columns + .stream() + .map( + column -> + ColumnProcessors.makeProcessor( + column, + SIMPLE_READER, + cursor.getColumnSelectorFactory() + ) + ) + .collect(Collectors.toList()); + + final List rows = new ArrayList<>(); - return Sequences.simple(rows); + while (!cursor.isDone()) { + final Object[] row = new Object[columns.size()]; + + for (int i = 0; i < row.length; i++) { + row[i] = readers.get(i).get(); } - ).toList(); + + rows.add(row); + cursor.advance(); + } + + return rows; + } + finally { + cursorMaker.close(); + } } - public static void verifyCursors( - final Sequence cursors, + public static void verifyCursor( + final CursorMaker cursorMaker, final List columns, final List expectedRows ) { - final List rows = readCursors(cursors, columns); + final List rows = readCursor(cursorMaker, columns); for (int i = 0; i < rows.size(); i++) { try { diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index 3abfb020575b..e99e70f909c1 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.join; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -35,9 +34,9 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; -import org.joda.time.DateTime; import org.junit.Test; +import javax.annotation.Nullable; import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -73,7 +72,21 @@ public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch public CursorMaker asCursorMaker(CursorBuildSpec spec) { final CursorMaker delegate = super.asCursorMaker(spec); - return () -> delegate.makeCursors().map(cursor -> new CursorNoAdvance(cursor, countDownLatch)); + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return new CursorNoAdvance(delegate.makeCursor(), countDownLatch); + } + + @Override + public void close() + { + delegate.close(); + } + }; } private static class CursorNoAdvance implements Cursor @@ -93,12 +106,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return cursor.getColumnSelectorFactory(); } - @Override - public DateTime getTime() - { - return cursor.getTime(); - } - @Override public void advance() { @@ -222,25 +229,25 @@ public void makeCursorAndAdvance() joinFilterPreAnalysis ); - Cursor cursor = Iterables.getOnlyElement( - hashJoinSegmentStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors().toList() - ); + try (final CursorMaker maker = hashJoinSegmentStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = maker.makeCursor(); - ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() - { - @Override - public boolean matches(boolean includeUnknown) + ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() { - return false; - } + @Override + public boolean matches(boolean includeUnknown) + { + return false; + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { - } - }); + } + }); - cursor.advance(); + cursor.advance(); + } } } 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 6b3d4698ab98..46852491ce08 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 @@ -24,9 +24,6 @@ import org.apache.druid.error.DruidException; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.aggregation.AggregationTestHelper; @@ -35,6 +32,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; @@ -352,10 +350,8 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir .setVirtualColumns(virtualColumns) .setGranularity(Granularities.DAY) .build(); - Sequence cursorSequence = storageAdapter.asCursorMaker(buildSpec).makeCursors(); - final Yielder yielder = Yielders.each(cursorSequence); - closer.register(yielder); - final Cursor cursor = yielder.get(); + final CursorMaker maker = closer.register(storageAdapter.asCursorMaker(buildSpec)); + final Cursor cursor = maker.makeCursor(); return cursor.getColumnSelectorFactory(); } diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index 8e3ad9c64d4c..7bec5f1fa33d 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -87,12 +87,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return time; - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 43dd020971e4..ef0ef087e62e 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -29,7 +29,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; @@ -45,6 +44,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -142,9 +142,9 @@ public void test_single_value_string_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); - cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); ExpressionPlan plan = ExpressionPlanner.plan( adapter, @@ -198,9 +198,7 @@ public void test_single_value_string_bindings() cursor.advance(); } - - return null; - }); + } } } @@ -213,9 +211,8 @@ public void test_multi_value_string_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - cursorSequence.accumulate(null, (ignored, cursor) -> { + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // identifier, uses dimension selector supplier supplier, no null coercion @@ -282,8 +279,7 @@ public void test_multi_value_string_bindings() cursor.advance(); } - return ignored; - }); + } } } @@ -296,9 +292,8 @@ public void test_long_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans ExpressionPlan plan = ExpressionPlanner.plan( @@ -333,9 +328,7 @@ public void test_long_bindings() } cursor.advance(); } - - return null; - }); + } } } @@ -348,10 +341,8 @@ public void test_double_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans ExpressionPlan plan = ExpressionPlanner.plan( @@ -386,9 +377,7 @@ public void test_double_bindings() } cursor.advance(); } - - return null; - }); + } } } @@ -671,9 +660,8 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept ); IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - - Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - int rowsProcessed = cursors.map(cursor -> { + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = maker.makeCursor(); DimensionSelector xExprSelector = ExpressionSelectors.makeDimensionSelector( cursor.getColumnSelectorFactory(), Parser.parse("concat(x, 'foo')", ExprMacroTable.nil()), @@ -702,10 +690,9 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept rowCount++; cursor.advance(); } - return rowCount; - }).accumulate(0, (in, acc) -> in + acc); - Assert.assertEquals(2, rowsProcessed); + Assert.assertEquals(2, rowCount); + } } private static DimensionSelector dimensionSelectorFromSupplier( 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 d4b94104f8a2..8a7067015e33 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 @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; @@ -37,6 +36,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; @@ -240,95 +240,95 @@ public static void sanityTestVectorizedExpressionSelectors( .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - final VectorCursor cursor = storageAdapter.asCursorMaker(buildSpec).makeVectorCursor(); - - ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); - - int rowCount = 0; - if (capabilities.isDictionaryEncoded().isTrue()) { - SingleValueDimensionVectorSelector selector = cursor.getColumnSelectorFactory().makeSingleValueDimensionSelector( - DefaultDimensionSpec.of("v") - ); - while (!cursor.isDone()) { - int[] row = selector.getRowVector(); - for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - results.add(selector.lookupName(row[i])); + try (final CursorMaker maker = storageAdapter.asCursorMaker(buildSpec)) { + final VectorCursor cursor = maker.makeVectorCursor(); + + ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); + + int rowCount = 0; + if (capabilities.isDictionaryEncoded().isTrue()) { + SingleValueDimensionVectorSelector selector = cursor.getColumnSelectorFactory() + .makeSingleValueDimensionSelector( + DefaultDimensionSpec.of("v") + ); + while (!cursor.isDone()) { + int[] row = selector.getRowVector(); + for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { + results.add(selector.lookupName(row[i])); + } + cursor.advance(); } - cursor.advance(); - } - } else { - VectorValueSelector selector = null; - VectorObjectSelector objectSelector = null; - if (Types.isNumeric(outputType)) { - selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); } else { - objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v"); - } - GroupByVectorColumnSelector groupBySelector = - cursor.getColumnSelectorFactory().makeGroupByVectorColumnSelector("v", DeferExpressionDimensions.ALWAYS); - while (!cursor.isDone()) { - final List resultsVector = new ArrayList<>(); - boolean[] nulls; - switch (outputType.getType()) { - case LONG: - nulls = selector.getNullVector(); - long[] longs = selector.getLongVector(); - for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(nulls != null && nulls[i] ? null : longs[i]); - } - break; - case DOUBLE: - // special case to test floats just to get coverage on getFloatVector - if ("float2".equals(expression)) { + VectorValueSelector selector = null; + VectorObjectSelector objectSelector = null; + if (Types.isNumeric(outputType)) { + selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); + } else { + objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v"); + } + GroupByVectorColumnSelector groupBySelector = + cursor.getColumnSelectorFactory().makeGroupByVectorColumnSelector("v", DeferExpressionDimensions.ALWAYS); + while (!cursor.isDone()) { + final List resultsVector = new ArrayList<>(); + boolean[] nulls; + switch (outputType.getType()) { + case LONG: nulls = selector.getNullVector(); - float[] floats = selector.getFloatVector(); + long[] longs = selector.getLongVector(); for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(nulls != null && nulls[i] ? null : (double) floats[i]); + resultsVector.add(nulls != null && nulls[i] ? null : longs[i]); } - } else { - nulls = selector.getNullVector(); - double[] doubles = selector.getDoubleVector(); - for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(nulls != null && nulls[i] ? null : doubles[i]); + break; + case DOUBLE: + // special case to test floats just to get coverage on getFloatVector + if ("float2".equals(expression)) { + nulls = selector.getNullVector(); + float[] floats = selector.getFloatVector(); + for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { + resultsVector.add(nulls != null && nulls[i] ? null : (double) floats[i]); + } + } else { + nulls = selector.getNullVector(); + double[] doubles = selector.getDoubleVector(); + for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { + resultsVector.add(nulls != null && nulls[i] ? null : doubles[i]); + } } - } - break; - case STRING: - Object[] objects = objectSelector.getObjectVector(); - for (int i = 0; i < objectSelector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(objects[i]); - } - break; - } + break; + case STRING: + Object[] objects = objectSelector.getObjectVector(); + for (int i = 0; i < objectSelector.getCurrentVectorSize(); i++, rowCount++) { + resultsVector.add(objects[i]); + } + break; + } - verifyGroupBySelector(groupBySelector, resultsVector); - results.addAll(resultsVector); - cursor.advance(); + verifyGroupBySelector(groupBySelector, resultsVector); + results.addAll(resultsVector); + cursor.advance(); + } + } + closer.register(cursor); + + + final Cursor nonVectorized = maker.makeCursor(); + + final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory() + .makeColumnValueSelector("v"); + int rows = 0; + while (!nonVectorized.isDone()) { + Assert.assertEquals( + "Failed at row " + rows, + nonSelector.getObject(), + results.get(rows) + ); + rows++; + nonVectorized.advance(); } - } - closer.register(cursor); - - Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); - - int rowCountCursor = cursors - .map(nonVectorized -> { - final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory() - .makeColumnValueSelector("v"); - int rows = 0; - while (!nonVectorized.isDone()) { - Assert.assertEquals( - "Failed at row " + rows, - nonSelector.getObject(), - results.get(rows) - ); - rows++; - nonVectorized.advance(); - } - return rows; - }).accumulate(0, (acc, in) -> acc + in); - Assert.assertTrue(rowCountCursor > 0); - Assert.assertEquals(rowCountCursor, rowCount); + Assert.assertTrue(rows > 0); + Assert.assertEquals(rows, rowCount); + } } private static void verifyGroupBySelector( diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index c58c686074d0..2cd239ebde0b 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -27,11 +27,9 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; @@ -39,7 +37,6 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.StorageAdapter; -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.data.Indexed; @@ -332,16 +329,17 @@ public Metadata getMetadata() } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return null; + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return null; + } + }; } }; diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 940ef3ac1d68..3ad58603ea37 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -72,6 +72,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; @@ -308,64 +309,52 @@ public static void runDump( .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - withOutputStream( - new Function() - { - @Override - public Object apply(final OutputStream out) + withOutputStream( + new Function() { - final Sequence sequence = Sequences.map( - cursors, - new Function() - { - @Override - public Object apply(Cursor cursor) - { - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - final List selectors = columnNames - .stream() - .map(columnSelectorFactory::makeColumnValueSelector) - .collect(Collectors.toList()); - - while (!cursor.isDone()) { - final Map row = Maps.newLinkedHashMap(); - - for (int i = 0; i < columnNames.size(); i++) { - final String columnName = columnNames.get(i); - final Object value = selectors.get(i).getObject(); - - if (timeISO8601 && columnNames.get(i).equals(ColumnHolder.TIME_COLUMN_NAME)) { - row.put(columnName, new DateTime(value, DateTimeZone.UTC).toString()); - } else { - row.put(columnName, value); - } - } + @Override + public Object apply(final OutputStream out) + { + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final List selectors = columnNames + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); - try { - out.write(objectMapper.writeValueAsBytes(row)); - out.write('\n'); - } - catch (IOException e) { - throw new RuntimeException(e); - } + while (!cursor.isDone()) { + final Map row = Maps.newLinkedHashMap(); - cursor.advance(); - } + for (int i = 0; i < columnNames.size(); i++) { + final String columnName = columnNames.get(i); + final Object value = selectors.get(i).getObject(); - return null; + if (timeISO8601 && columnNames.get(i).equals(ColumnHolder.TIME_COLUMN_NAME)) { + row.put(columnName, new DateTime(value, DateTimeZone.UTC).toString()); + } else { + row.put(columnName, value); } } - ); - evaluateSequenceForSideEffects(sequence); + try { + out.write(objectMapper.writeValueAsBytes(row)); + out.write('\n'); + } + catch (IOException e) { + throw new RuntimeException(e); + } - return null; - } - }, - outputFileName - ); + cursor.advance(); + } + + return null; + } + }, + outputFileName + ); + } } @VisibleForTesting From 3824754c2c663f8eb57dc87fa23448083086365c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 26 Jul 2024 04:37:54 -0700 Subject: [PATCH 10/74] leaky leaky --- .../util/ToObjectVectorColumnProcessorFactoryTest.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index c0a5d3c962c8..7c7a7466c9fb 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -26,6 +26,7 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; @@ -56,7 +57,7 @@ public void setUp() @Test public void testRead() { - try (final VectorCursor cursor = makeCursor()) { + try (final CursorMaker maker = makeCursor(); final VectorCursor cursor = maker.makeVectorCursor()) { final Supplier qualitySupplier = ColumnProcessors.makeVectorProcessor( "quality", ToObjectVectorColumnProcessorFactory.INSTANCE, @@ -169,7 +170,7 @@ public void testComplexSketch() Assert.assertThat(sketch, CoreMatchers.instanceOf(HyperLogLogCollector.class)); } - private VectorCursor makeCursor() + private CursorMaker makeCursor() { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setQueryContext( @@ -178,12 +179,12 @@ private VectorCursor makeCursor() ) ) .build(); - return adapter.asCursorMaker(buildSpec).makeVectorCursor(); + return adapter.asCursorMaker(buildSpec); } private List readColumn(final String column, final int limit) { - try (final VectorCursor cursor = makeCursor()) { + try (final CursorMaker maker = makeCursor(); final VectorCursor cursor = maker.makeVectorCursor()) { final Supplier supplier = ColumnProcessors.makeVectorProcessor( column, ToObjectVectorColumnProcessorFactory.INSTANCE, From 6c327755c5da8f1a308ffc3cbd3453cf63484a65 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 26 Jul 2024 12:39:27 -0700 Subject: [PATCH 11/74] split out CursorMakerFactory interface, adjustments --- ...bjectVectorColumnProcessorFactoryTest.java | 8 +- .../SegmentGeneratorFrameProcessor.java | 3 + .../results/ExportResultsFrameProcessor.java | 4 + .../hadoop/DatasourceRecordReader.java | 180 +++++++++--------- .../indexing/input/DruidSegmentReader.java | 4 + .../apache/druid/frame/field/RowReader.java | 4 +- .../apache/druid/frame/read/FrameReader.java | 12 +- .../druid/frame/segment/FrameCursor.java | 5 +- .../frame/segment/FrameStorageAdapter.java | 6 +- ...tory.java => FrameCursorMakerFactory.java} | 10 +- .../segment/columnar/FrameQueryableIndex.java | 2 +- ...tory.java => FrameCursorMakerFactory.java} | 10 +- .../LazilyDecoratedRowsAndColumns.java | 4 + .../StorageAdapterRowsAndColumns.java | 4 + .../druid/query/scan/ScanQueryEngine.java | 3 + .../TimeBoundaryQueryRunnerFactory.java | 2 +- .../apache/druid/segment/CursorFactory.java | 3 +- .../druid/segment/CursorMakerFactory.java | 25 +++ .../druid/segment/UnnestStorageAdapter.java | 5 +- .../join/HashJoinSegmentStorageAdapter.java | 4 +- .../table/BroadcastSegmentIndexedTable.java | 5 + .../druid/query/scan/ScanQueryTest.java | 4 +- .../org/apache/druid/cli/DumpSegment.java | 3 + 23 files changed, 183 insertions(+), 127 deletions(-) rename processing/src/main/java/org/apache/druid/frame/segment/columnar/{FrameCursorFactory.java => FrameCursorMakerFactory.java} (95%) rename processing/src/main/java/org/apache/druid/frame/segment/row/{FrameCursorFactory.java => FrameCursorMakerFactory.java} (91%) create mode 100644 processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index 7c7a7466c9fb..68b2b82e8c6d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -57,7 +57,9 @@ public void setUp() @Test public void testRead() { - try (final CursorMaker maker = makeCursor(); final VectorCursor cursor = maker.makeVectorCursor()) { + try (final CursorMaker maker = makeCursor(); + final VectorCursor cursor = maker.makeVectorCursor() + ) { final Supplier qualitySupplier = ColumnProcessors.makeVectorProcessor( "quality", ToObjectVectorColumnProcessorFactory.INSTANCE, @@ -184,7 +186,9 @@ private CursorMaker makeCursor() private List readColumn(final String column, final int limit) { - try (final CursorMaker maker = makeCursor(); final VectorCursor cursor = maker.makeVectorCursor()) { + try (final CursorMaker maker = makeCursor(); + final VectorCursor cursor = maker.makeVectorCursor() + ) { final Supplier supplier = ColumnProcessors.makeVectorProcessor( column, ToObjectVectorColumnProcessorFactory.INSTANCE, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index 911d13579583..e865137d7263 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java @@ -184,6 +184,9 @@ private void addFrame(final Frame frame) final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return; + } final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java index f54f8336f97d..52fe8a9d6c3c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java @@ -154,6 +154,10 @@ private void exportFrame(final Frame frame) final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + exportWriter.writeRowEnd(); + return; + } final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); //noinspection rawtypes diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index 40a67f1236e0..5ddbef8842bf 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -44,10 +44,11 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; @@ -214,108 +215,99 @@ public SegmentReader( @Override public Sequence apply(WindowedStorageAdapter adapter) { - return Sequences.concat( - Sequences.map( - adapter.getAdapter().makeCursors( - Filters.toFilter(dimFilter), - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), new Function>() + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(Filters.toFilter(dimFilter)) + .setInterval(adapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec); + final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return Sequences.empty(); + } + final BaseLongColumnValueSelector timestampColumnSelector = + cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); + + final Map dimSelectors = new HashMap<>(); + for (String dim : dims) { + final DimensionSelector dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(dim, dim)); + // dimSelector is null if the dimension is not present + if (dimSelector != null) { + dimSelectors.put(dim, dimSelector); + } + } + + final Map metSelectors = new HashMap<>(); + for (String metric : metrics) { + final BaseObjectColumnValueSelector metricSelector = + cursor.getColumnSelectorFactory().makeColumnValueSelector(metric); + metSelectors.put(metric, metricSelector); + } + + return Sequences.simple( + new Iterable() + { + @Override + public Iterator iterator() + { + return new Iterator() { - @Nullable @Override - public Sequence apply(final Cursor cursor) + public boolean hasNext() { - final BaseLongColumnValueSelector timestampColumnSelector = - cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); - - final Map dimSelectors = new HashMap<>(); - for (String dim : dims) { - final DimensionSelector dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(dim, dim)); - // dimSelector is null if the dimension is not present - if (dimSelector != null) { - dimSelectors.put(dim, dimSelector); + return !cursor.isDone(); + } + + @Override + public InputRow next() + { + final Map theEvent = Maps.newLinkedHashMap(); + final long timestamp = timestampColumnSelector.getLong(); + theEvent.put(TimestampSpec.DEFAULT_COLUMN, DateTimes.utc(timestamp)); + + for (Map.Entry dimSelector : + dimSelectors.entrySet()) { + final String dim = dimSelector.getKey(); + final DimensionSelector selector = dimSelector.getValue(); + final IndexedInts vals = selector.getRow(); + + int valsSize = vals.size(); + if (valsSize == 1) { + final String dimVal = selector.lookupName(vals.get(0)); + theEvent.put(dim, dimVal); + } else if (valsSize > 1) { + List dimVals = new ArrayList<>(valsSize); + for (int i = 0; i < valsSize; ++i) { + dimVals.add(selector.lookupName(vals.get(i))); + } + theEvent.put(dim, dimVals); } } - final Map metSelectors = new HashMap<>(); - for (String metric : metrics) { - final BaseObjectColumnValueSelector metricSelector = - cursor.getColumnSelectorFactory().makeColumnValueSelector(metric); - metSelectors.put(metric, metricSelector); + for (Map.Entry metSelector : + metSelectors.entrySet()) { + final String metric = metSelector.getKey(); + final BaseObjectColumnValueSelector selector = metSelector.getValue(); + Object value = selector.getObject(); + if (value != null) { + theEvent.put(metric, value); + } } + cursor.advance(); + return new MapBasedInputRow(timestamp, dims, theEvent); + } - return Sequences.simple( - new Iterable() - { - @Override - public Iterator iterator() - { - return new Iterator() - { - @Override - public boolean hasNext() - { - return !cursor.isDone(); - } - - @Override - public InputRow next() - { - final Map theEvent = Maps.newLinkedHashMap(); - final long timestamp = timestampColumnSelector.getLong(); - theEvent.put(TimestampSpec.DEFAULT_COLUMN, DateTimes.utc(timestamp)); - - for (Map.Entry dimSelector : - dimSelectors.entrySet()) { - final String dim = dimSelector.getKey(); - final DimensionSelector selector = dimSelector.getValue(); - final IndexedInts vals = selector.getRow(); - - int valsSize = vals.size(); - if (valsSize == 1) { - final String dimVal = selector.lookupName(vals.get(0)); - theEvent.put(dim, dimVal); - } else if (valsSize > 1) { - List dimVals = new ArrayList<>(valsSize); - for (int i = 0; i < valsSize; ++i) { - dimVals.add(selector.lookupName(vals.get(i))); - } - theEvent.put(dim, dimVals); - } - } - - for (Map.Entry metSelector : - metSelectors.entrySet()) { - final String metric = metSelector.getKey(); - final BaseObjectColumnValueSelector selector = metSelector.getValue(); - Object value = selector.getObject(); - if (value != null) { - theEvent.put(metric, value); - } - } - cursor.advance(); - return new MapBasedInputRow(timestamp, dims, theEvent); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException("Remove Not Supported"); - } - }; - } - } - ); + @Override + public void remove() + { + throw new UnsupportedOperationException("Remove Not Supported"); } - } - ) - ); + }; + } + } + ).withBaggage(maker); } } ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index 7dcaf9a8b502..f04d2eb8f7d4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -33,6 +33,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -132,6 +133,9 @@ protected CloseableIterator> intermediateRowIterator() throw final CursorMaker maker = storageAdapter.getAdapter().asCursorMaker(cursorBuildSpec); final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return CloseableIterators.wrap(Collections.emptyIterator(), maker); + } // Retain order of columns from the original segments. Useful for preserving dimension order if we're in // schemaless mode. diff --git a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java index c79c9b25d677..f1563d044bfe 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java @@ -57,7 +57,7 @@ public int fieldCount() /** * Read a particular field value as an object. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorMakerFactory} * for reading many rows out of a frame. */ public Object readField(final Memory memory, final long rowPosition, final long rowLength, final int fieldNumber) @@ -77,7 +77,7 @@ public Object readField(final Memory memory, final long rowPosition, final long /** * Read an entire row as a list of objects. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorMakerFactory} * for reading many rows out of a frame. */ public List readRow(final Memory memory, final long rowPosition, final long rowLength) diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index 8ddf99325d39..46855570077a 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -28,11 +28,11 @@ import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.read.columnar.FrameColumnReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.row.FrameCursorFactory; +import org.apache.druid.frame.segment.row.FrameCursorMakerFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -136,15 +136,15 @@ public ColumnCapabilities columnCapabilities(final Frame frame, final String col } /** - * Create a {@link CursorFactory} for the given frame. + * Create a {@link CursorMakerFactory} for the given frame. */ - public CursorFactory makeCursorFactory(final Frame frame) + public CursorMakerFactory makeCursorMakerFactory(final Frame frame) { switch (frame.type()) { case COLUMNAR: - return new org.apache.druid.frame.segment.columnar.FrameCursorFactory(frame, signature, columnReaders); + return new org.apache.druid.frame.segment.columnar.FrameCursorMakerFactory(frame, signature, columnReaders); case ROW_BASED: - return new FrameCursorFactory(frame, this, fieldReaders); + return new FrameCursorMakerFactory(frame, this, fieldReaders); default: throw new ISE("Unrecognized frame type [%s]", frame.type()); } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 35672c0338d0..63f87f7efad7 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -19,14 +19,15 @@ package org.apache.druid.frame.segment; +import org.apache.druid.frame.segment.columnar.FrameCursorMakerFactory; import org.apache.druid.query.BaseQuery; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; /** - * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorFactory} - * and {@link org.apache.druid.frame.segment.columnar.FrameCursorFactory}. + * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorMakerFactory} + * and {@link FrameCursorMakerFactory}. * * Adds the methods {@link #getCurrentRow()} and {@link #setCurrentRow(int)} so the cursor can be moved to * particular rows. diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java index b82529de506b..fcc8cfdaa4f4 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java @@ -22,8 +22,8 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; @@ -47,14 +47,14 @@ public class FrameStorageAdapter implements StorageAdapter private final Frame frame; private final FrameReader frameReader; private final Interval interval; - private final CursorFactory cursorFactory; + private final CursorMakerFactory cursorFactory; public FrameStorageAdapter(Frame frame, FrameReader frameReader, Interval interval) { this.frame = frame; this.frameReader = frameReader; this.interval = interval; - this.cursorFactory = frameReader.makeCursorFactory(frame); + this.cursorFactory = frameReader.makeCursorMakerFactory(frame); } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java similarity index 95% rename from processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java index e6aaf9d6fe6c..81945b795be9 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java @@ -34,8 +34,8 @@ import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; @@ -55,19 +55,19 @@ import java.util.List; /** - * A {@link CursorFactory} implementation based on a single columnar {@link Frame}. + * A {@link CursorMakerFactory} implementation based on a single columnar {@link Frame}. * * This class is only used for columnar frames. It is not used for row-based frames. * - * @see org.apache.druid.frame.segment.row.FrameCursorFactory the row-based version + * @see org.apache.druid.frame.segment.row.FrameCursorMakerFactory the row-based version */ -public class FrameCursorFactory implements CursorFactory +public class FrameCursorMakerFactory implements CursorMakerFactory { private final Frame frame; private final RowSignature signature; private final List columnReaders; - public FrameCursorFactory( + public FrameCursorMakerFactory( final Frame frame, final RowSignature signature, final List columnReaders diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index f42a33ce6f78..94e4d637ddf5 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java @@ -42,7 +42,7 @@ * A {@link QueryableIndex} implementation based on a single columnar {@link Frame}. There is no internal caching * of columns here, so callers should generally wrap this in a {@link org.apache.druid.segment.ColumnCache}. * - * This class exists so {@link FrameCursorFactory} can reuse code meant for regular segment-backed + * This class exists so {@link FrameCursorMakerFactory} can reuse code meant for regular segment-backed * {@link QueryableIndex}. Some methods are implemented by throwing {@link UnsupportedOperationException}, wherever * it is not expected that those methods are actually going to be needed. */ diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java similarity index 91% rename from processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java index a2005d91405a..56e84387b35c 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java @@ -31,8 +31,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; @@ -40,19 +40,19 @@ import java.util.List; /** - * A {@link CursorFactory} implementation based on a single row-based {@link Frame}. + * A {@link CursorMakerFactory} implementation based on a single row-based {@link Frame}. * * This class is only used for row-based frames. * - * @see org.apache.druid.frame.segment.columnar.FrameCursorFactory the columnar version + * @see org.apache.druid.frame.segment.columnar.FrameCursorMakerFactory the columnar version */ -public class FrameCursorFactory implements CursorFactory +public class FrameCursorMakerFactory implements CursorMakerFactory { private final Frame frame; private final FrameReader frameReader; private final List fieldReaders; - public FrameCursorFactory( + public FrameCursorMakerFactory( final Frame frame, final FrameReader frameReader, final List fieldReaders diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index a7dbd192b504..19a093bdf98d 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -231,6 +231,10 @@ private Pair materializeStorageAdapter(StorageAdapter as) try (final CursorMaker maker = as.asCursorMaker(builder.build())) { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return null; + } + final AtomicReference siggy = new AtomicReference<>(null); long remainingRowsToSkip = limit.getOffset(); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java index 29bdad4aafc9..17aa2443544c 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java @@ -98,6 +98,10 @@ private static RowsAndColumns materialize(StorageAdapter as) try (final CursorMaker maker = as.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return new EmptyRowsAndColumns(); + } + final RowSignature rowSignature = as.getRowSignature(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 094c38c8a97e..4dc5841c78a8 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -122,6 +122,9 @@ public Sequence process( public Iterator make() { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return Collections.emptyIterator(); + } final List columnSelectors = new ArrayList<>(allColumns.size()); final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 96da640110a5..4d561a1c712f 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -123,7 +123,7 @@ private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legac if (cursor == null) { return null; } - final Result result = skipToFirstMatching.apply(maker.makeCursor()); + final Result result = skipToFirstMatching.apply(cursor); return result == null ? null : result.getValue(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index 8ebd2fd444df..68363e740fad 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -36,8 +36,9 @@ * * @see StorageAdapter */ -public interface CursorFactory +public interface CursorFactory extends CursorMakerFactory { + @Override default CursorMaker asCursorMaker(CursorBuildSpec spec) { return new CursorMaker() diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java new file mode 100644 index 000000000000..31f6be54f4c7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java @@ -0,0 +1,25 @@ +/* + * 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.segment; + +public interface CursorMakerFactory +{ + CursorMaker asCursorMaker(CursorBuildSpec spec); +} diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index eba801e224e2..4f8c9ac2ebfb 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -55,7 +55,6 @@ import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; -import java.util.Objects; import java.util.Set; /** @@ -114,7 +113,9 @@ public Cursor makeCursor() { final CursorMaker maker = closer.register(baseAdapter.asCursorMaker(unnestBuildSpec)); final Cursor cursor = maker.makeCursor(); - Objects.requireNonNull(cursor); + if (cursor == null) { + return null; + } final ColumnCapabilities capabilities = unnestColumn.capabilities( cursor.getColumnSelectorFactory(), unnestColumn.getOutputName() diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 4319fdc52b57..9f87694c0be1 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -291,8 +291,10 @@ public Cursor makeCursor() final Cursor baseCursor = joinablesCloser.register(baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build())) .makeCursor(); + if (baseCursor == null) { + return null; + } - assert baseCursor != null; Cursor retVal = baseCursor; for (JoinableClause clause : clauses) { diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index edc9bb056158..d170b0578544 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -50,6 +50,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.Set; @@ -122,6 +123,10 @@ public BroadcastSegmentIndexedTable( .build(); try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + this.keyColumnsIndexes = Collections.emptyList(); + return; + } int rowNumber = 0; ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index ce4a417e2bf3..694e29b4416e 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -30,13 +30,13 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.query.Query; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 3ad58603ea37..116ce0169fb7 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -311,6 +311,9 @@ public static void runDump( try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return; + } withOutputStream( new Function() From 04e5e472426b9cb63edb59604b3a706be24634d5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 26 Jul 2024 18:40:36 -0700 Subject: [PATCH 12/74] fixup inspections --- .../scan/ScanQueryFrameProcessor.java | 12 +++++----- .../druid/query/IterableRowsCursorHelper.java | 2 -- .../StorageAdapterRowsAndColumns.java | 23 ++++++++++--------- .../druid/query/topn/TopNQueryEngine.java | 4 ++-- .../segment/QueryableIndexCursorMaker.java | 10 ++++---- .../apache/druid/segment/RowBasedCursor.java | 5 ---- .../druid/segment/RowBasedStorageAdapter.java | 1 - .../IncrementalIndexCursorMaker.java | 10 ++------ .../StorageAdapterRowsAndColumnsTest.java | 10 ++++---- .../IncrementalIndexStorageAdapterTest.java | 4 ---- .../ExpressionVectorSelectorsTest.java | 8 +++++-- 11 files changed, 37 insertions(+), 52 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 095604e69dc4..3ffd3f8dd01b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -253,14 +253,14 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment } final CursorMaker maker = closer.register(adapter.asCursorMaker(query.asCursorBuildSpec(null))); - final Cursor cursor = maker.makeCursor(); + final Cursor nextCursor = maker.makeCursor(); - if (cursor == null) { + if (nextCursor == null) { // No cursors! maker.close(); return ReturnOrAwait.returnObject(Unit.instance()); } else { - final long rowsFlushed = setNextCursor(cursor, segmentHolder.get()); + final long rowsFlushed = setNextCursor(nextCursor, segmentHolder.get()); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment } } @@ -297,14 +297,14 @@ protected ReturnOrAwait runWithInputChannel( } final CursorMaker maker = closer.register(adapter.asCursorMaker(query.asCursorBuildSpec(null))); - final Cursor cursor = maker.makeCursor(); + final Cursor nextCursor = maker.makeCursor(); - if (cursor == null) { + if (nextCursor == null) { // no cursor maker.close(); return ReturnOrAwait.returnObject(Unit.instance()); } - final long rowsFlushed = setNextCursor(cursor, frameSegment); + final long rowsFlushed = setNextCursor(nextCursor, frameSegment); if (rowsFlushed > 0) { return ReturnOrAwait.runAgain(); diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java index 4bf1cb92a610..46a39ad4cb52 100644 --- a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java +++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java @@ -21,7 +21,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; @@ -77,7 +76,6 @@ public static Pair getCursorFromSequence(Sequence r null, Intervals.ETERNITY, // Setting the interval to eternity ensures that we are iterating over all of the rows VirtualColumns.EMPTY, - Granularities.ALL, false, rowSignature != null ? rowSignature : RowSignature.empty() ); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java index 17aa2443544c..92d6d594cb8e 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java @@ -112,17 +112,18 @@ private static RowsAndColumns materialize(StorageAdapter as) Collections.emptyList() ); - final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory); - while (!cursor.isDoneOrInterrupted()) { - writer.addSelection(); - cursor.advance(); - } - - if (writer == null) { - return new EmptyRowsAndColumns(); - } else { - final byte[] bytes = writer.toByteArray(); - return new ColumnBasedFrameRowsAndColumns(Frame.wrap(bytes), rowSignature); + try(final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + while (!cursor.isDoneOrInterrupted()) { + writer.addSelection(); + cursor.advance(); + } + + if (writer == null) { + return new EmptyRowsAndColumns(); + } else { + final byte[] bytes = writer.toByteArray(); + return new ColumnBasedFrameRowsAndColumns(Frame.wrap(bytes), rowSignature); + } } } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index 076747666ccc..d086d61c7519 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -55,8 +55,8 @@ public TopNQueryEngine(NonBlockingPool bufferPool) /** * Do the thing - process a {@link StorageAdapter} into a {@link Sequence} of {@link TopNResultValue}, with one of the * fine {@link TopNAlgorithm} available chosen based on the type of column being aggregated. The algorithm provides a - * mapping function to process rows from the adapter {@link org.apache.druid.segment.Cursor} to apply - * {@link AggregatorFactory} and create or update {@link TopNResultValue} + * mapping function to process rows from the adapter {@link Cursor} to apply {@link AggregatorFactory} and create or + * update {@link TopNResultValue} */ public Sequence> query( final TopNQuery query, diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 9bfda8a20cd3..846b087bd6c9 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -53,7 +53,6 @@ import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.vector.VectorOffset; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -67,6 +66,7 @@ public class QueryableIndexCursorMaker implements CursorMaker private static final Logger log = new Logger(QueryableIndexCursorMaker.class); private final QueryableIndex index; private final Interval interval; + @SuppressWarnings("unused") private final Granularity gran; private final VirtualColumns virtualColumns; @@ -206,9 +206,9 @@ public Cursor makeCursor() descending ); final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); - return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, DateTimes.utc(timeStart)); + return new QueryableIndexCursor(filteredOffset, columnSelectorFactory); } else { - return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, DateTimes.utc(timeStart)); + return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory); } } @@ -487,13 +487,11 @@ private static class QueryableIndexCursor implements HistoricalCursor { private final Offset cursorOffset; private final ColumnSelectorFactory columnSelectorFactory; - private final DateTime bucketStart; - QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory, DateTime bucketStart) + QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory) { this.cursorOffset = cursorOffset; this.columnSelectorFactory = columnSelectorFactory; - this.bucketStart = bucketStart; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index d591e0cd11d2..7f5040a8ae13 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -19,13 +19,11 @@ package org.apache.druid.segment; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.ValueMatchers; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -42,7 +40,6 @@ public class RowBasedCursor implements Cursor private final ToLongFunction timestampFunction; private final Interval interval; private final boolean descending; - private final DateTime cursorTime; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher valueMatcher; @@ -54,7 +51,6 @@ public RowBasedCursor( @Nullable final Filter filter, final Interval interval, final VirtualColumns virtualColumns, - final Granularity gran, final boolean descending, final RowSignature rowSignature ) @@ -63,7 +59,6 @@ public RowBasedCursor( this.timestampFunction = rowAdapter.timestampFunction(); this.interval = interval; this.descending = descending; - this.cursorTime = gran.toDateTime(interval.getStartMillis()); this.columnSelectorFactory = virtualColumns.wrap( new RowBasedColumnSelectorFactory<>( rowWalker::currentRow, diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index 2092b17f428d..412877dcc1f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -203,7 +203,6 @@ public Cursor makeCursor() spec.getFilter(), actualInterval, spec.getVirtualColumns(), - gran, spec.isDescending(), rowSignature ); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java index 442666f084dd..372804a9c085 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.incremental; import com.google.common.collect.Iterators; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -30,7 +29,6 @@ import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.filter.ValueMatchers; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -80,8 +78,7 @@ public Cursor makeCursor() builder.getVirtualColumns(), builder.isDescending(), builder.getFilter(), - actualInterval, - builder.getGranularity() + actualInterval ); } @@ -94,7 +91,6 @@ static class IncrementalIndexCursor implements Cursor private Iterator baseIter; private Iterable cursorIterable; private boolean emptyRange; - private final DateTime time; private int numAdvanced; private boolean done; @@ -104,8 +100,7 @@ static class IncrementalIndexCursor implements Cursor VirtualColumns virtualColumns, boolean descending, @Nullable Filter filter, - Interval actualInterval, - Granularity gran + Interval actualInterval ) { currEntry = new IncrementalIndexRowHolder(); @@ -125,7 +120,6 @@ static class IncrementalIndexCursor implements Cursor actualInterval.getEndMillis() ); emptyRange = !cursorIterable.iterator().hasNext(); - time = gran.toDateTime(actualInterval.getStartMillis()); reset(); } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java index bb2826498c54..05a9927fc12b 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java @@ -32,13 +32,13 @@ public StorageAdapterRowsAndColumnsTest() super(StorageAdapterRowsAndColumns.class); } - public static Function MAKER = input -> { - return buildFrame(input); - }; + public static Function MAKER = + StorageAdapterRowsAndColumnsTest::buildFrame; private static StorageAdapterRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) { - ColumnBasedFrameRowsAndColumns fRAC = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input); - return new StorageAdapterRowsAndColumns(fRAC.as(StorageAdapter.class)); + try(ColumnBasedFrameRowsAndColumns fRAC = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input)) { + return new StorageAdapterRowsAndColumns(fRAC.as(StorageAdapter.class)); + } } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 9521257c25b1..4f6a7bca28a9 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -84,7 +84,6 @@ import java.util.Collections; import java.util.List; import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; /** */ @@ -437,7 +436,6 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception .build(); try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { Cursor cursor = maker.makeCursor(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); DimensionSelector dimSelector = cursor .getColumnSelectorFactory() .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); @@ -496,7 +494,6 @@ public void testCursorDictionaryRaceConditionFix() throws Exception .build(); try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { Cursor cursor = maker.makeCursor(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); DimensionSelector dimSelector = cursor .getColumnSelectorFactory() .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); @@ -537,7 +534,6 @@ public void testCursoringAndSnapshot() throws Exception .build(); try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { Cursor cursor = maker.makeCursor(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); DimensionSelector dimSelector1A = cursor .getColumnSelectorFactory() 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 8a7067015e33..2b9f23afad6a 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 @@ -242,6 +242,7 @@ public static void sanityTestVectorizedExpressionSelectors( .build(); try (final CursorMaker maker = storageAdapter.asCursorMaker(buildSpec)) { final VectorCursor cursor = maker.makeVectorCursor(); + Assert.assertNotNull(cursor); ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); @@ -273,6 +274,7 @@ public static void sanityTestVectorizedExpressionSelectors( boolean[] nulls; switch (outputType.getType()) { case LONG: + Assert.assertNotNull(selector); nulls = selector.getNullVector(); long[] longs = selector.getLongVector(); for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { @@ -280,6 +282,7 @@ public static void sanityTestVectorizedExpressionSelectors( } break; case DOUBLE: + Assert.assertNotNull(selector); // special case to test floats just to get coverage on getFloatVector if ("float2".equals(expression)) { nulls = selector.getNullVector(); @@ -295,7 +298,8 @@ public static void sanityTestVectorizedExpressionSelectors( } } break; - case STRING: + default: + Assert.assertNotNull(objectSelector); Object[] objects = objectSelector.getObjectVector(); for (int i = 0; i < objectSelector.getCurrentVectorSize(); i++, rowCount++) { resultsVector.add(objects[i]); @@ -312,7 +316,7 @@ public static void sanityTestVectorizedExpressionSelectors( final Cursor nonVectorized = maker.makeCursor(); - + Assert.assertNotNull(nonVectorized); final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory() .makeColumnValueSelector("v"); int rows = 0; From bde77bac8163d29b608886ec1194b5d0364cbee5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 26 Jul 2024 21:32:23 -0700 Subject: [PATCH 13/74] fix test, style --- .../indexing/common/task/CompactionTaskRunTest.java | 9 +++++---- .../query/rowsandcols/StorageAdapterRowsAndColumns.java | 2 +- .../rowsandcols/StorageAdapterRowsAndColumnsTest.java | 2 +- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index fbfc2401a162..f009e0609cbe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -1708,7 +1708,7 @@ public void testRunWithSpatialDimensions() throws Exception .build(); try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { final Cursor cursor = maker.makeCursor(); - + Assert.assertNotNull(cursor); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); Assert.assertTrue(factory.getColumnCapabilities("spatial").hasSpatialIndexes()); @@ -1735,8 +1735,8 @@ public void testRunWithSpatialDimensions() throws Exception cursor.advance(); } } - Assert.assertEquals(spatialrows, rowsFromSegment); } + Assert.assertEquals(spatialrows, rowsFromSegment); } @Test @@ -1840,6 +1840,7 @@ public void testRunWithAutoCastDimensions() throws Exception .build(); try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { final Cursor cursor = maker.makeCursor(); + Assert.assertNotNull(cursor); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); Assert.assertEquals(ColumnType.STRING, factory.getColumnCapabilities("ts").toColumnType()); @@ -1868,8 +1869,8 @@ public void testRunWithAutoCastDimensions() throws Exception cursor.advance(); } } - Assert.assertEquals(rows, rowsFromSegment); } + Assert.assertEquals(rows, rowsFromSegment); } private Pair runIndexTask() throws Exception @@ -2068,7 +2069,7 @@ private List getCSVFormatRowsFromSegments(List segments) th .build(); try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { final Cursor cursor = maker.makeCursor(); - + Assert.assertNotNull(cursor); cursor.reset(); while (!cursor.isDone()) { final DimensionSelector selector1 = cursor.getColumnSelectorFactory() diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java index 92d6d594cb8e..b996c5213ba0 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java @@ -112,7 +112,7 @@ private static RowsAndColumns materialize(StorageAdapter as) Collections.emptyList() ); - try(final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + try (final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { while (!cursor.isDoneOrInterrupted()) { writer.addSelection(); cursor.advance(); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java index 05a9927fc12b..4395ce9dd5b5 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java @@ -37,7 +37,7 @@ public StorageAdapterRowsAndColumnsTest() private static StorageAdapterRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) { - try(ColumnBasedFrameRowsAndColumns fRAC = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input)) { + try (ColumnBasedFrameRowsAndColumns fRAC = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input)) { return new StorageAdapterRowsAndColumns(fRAC.as(StorageAdapter.class)); } } From 920a86e2426704766e6ceeea782d67b0ca4ea094 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 12:12:58 -0700 Subject: [PATCH 14/74] add mark/resetMark to cursor to improve topN performance --- .../druid/frame/segment/FrameCursor.java | 15 +++++ .../druid/query/CursorGranularizer.java | 20 ------- .../druid/query/topn/BaseTopNAlgorithm.java | 4 +- .../druid/query/topn/TopNQueryEngine.java | 1 + .../java/org/apache/druid/segment/Cursor.java | 4 ++ .../apache/druid/segment/CursorFactory.java | 49 ----------------- .../segment/QueryableIndexCursorMaker.java | 19 +++++++ .../apache/druid/segment/RowBasedCursor.java | 23 +++++++- .../apache/druid/segment/StorageAdapter.java | 55 +++++++++++++++++++ .../UnnestColumnValueSelectorCursor.java | 21 +++++++ .../druid/segment/UnnestDimensionCursor.java | 21 +++++++ .../IncrementalIndexCursorMaker.java | 39 ++++++++++++- .../druid/segment/join/HashJoinEngine.java | 19 +++++++ .../druid/segment/join/PostJoinCursor.java | 14 +++++ .../druid/frame/TestArrayStorageAdapter.java | 13 +++++ .../testutil/RowNumberUpdatingCursor.java | 17 ++++++ .../org/apache/druid/segment/ListCursor.java | 15 +++++ .../segment/join/PostJoinCursorTest.java | 15 ++++- .../selector/TestColumnValueSelector.java | 21 +++++++ 19 files changed, 311 insertions(+), 74 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 63f87f7efad7..67169a302f16 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -24,6 +24,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; +import org.joda.time.DateTime; /** * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorMakerFactory} @@ -36,6 +37,7 @@ public class FrameCursor implements Cursor { private final SimpleSettableOffset offset; private final ColumnSelectorFactory columnSelectorFactory; + private int markOffset = 0; public FrameCursor( SimpleSettableOffset offset, @@ -77,9 +79,22 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } + @Override + public void mark(DateTime mark) + { + markOffset = offset.getOffset(); + } + + @Override + public void resetMark() + { + offset.setCurrentOffset(markOffset); + } + @Override public void reset() { + markOffset = 0; offset.reset(); } diff --git a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java index ddaebf8aec38..44915c98cd04 100644 --- a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java @@ -136,26 +136,6 @@ public boolean advanceToBucket(final Interval bucketInterval) return currentBucketStart <= currentTime && currentTime < currentBucketEnd; } - public void resetBucket() - { - cursor.reset(); - if (timeSelector == null) { - return; - } - long currentTime = timeSelector.getLong(); - if (descending) { - while (currentTime >= currentBucketEnd && !cursor.isDone()) { - cursor.advance(); - currentTime = timeSelector.getLong(); - } - } else { - while (currentTime < currentBucketStart && !cursor.isDone()) { - cursor.advance(); - currentTime = timeSelector.getLong(); - } - } - } - public boolean advanceCursorWithinBucket() { if (cursor.isDone()) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java index 7236a294197c..f1cf009b4249 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java @@ -123,7 +123,7 @@ private void runWithCardinalityKnown( resetAggregators(aggregatesStore); numProcessed += numToProcess; - params.getGranularizer().resetBucket(); + params.getCursor().resetMark(); } if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); @@ -152,7 +152,7 @@ private void runWithCardinalityUnknown( long processedRows = scanAndAggregate(params, null, aggregatesStore); updateResults(params, null, aggregatesStore, resultBuilder); resetAggregators(aggregatesStore); - params.getGranularizer().resetBucket(); + params.getCursor().resetMark(); if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); queryMetrics.stopRecordingScanTime(); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index d086d61c7519..4b2761c9d658 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -96,6 +96,7 @@ public Sequence> query( Sequences.simple(granularizer.getBucketIterable()) .map(bucketInterval -> { granularizer.advanceToBucket(bucketInterval); + cursor.mark(buildSpec.isDescending() ? bucketInterval.getEnd() : bucketInterval.getStart()); return mapFn.apply(cursor, granularizer, queryMetrics); }), Predicates.notNull() diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index d2bddba99fa2..2459edb4f606 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -19,6 +19,8 @@ package org.apache.druid.segment; +import org.joda.time.DateTime; + /** * Cursor is an interface for iteration over a range of data points, used during query execution. {@link * QueryableIndexCursorMaker.QueryableIndexCursor} is an implementation for historical segments, and {@link @@ -41,5 +43,7 @@ public interface Cursor boolean isDone(); boolean isDoneOrInterrupted(); + void mark(DateTime mark); + void resetMark(); void reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index 68363e740fad..31cc3a49236f 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -19,9 +19,7 @@ package org.apache.druid.segment; -import com.google.common.collect.Iterables; import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.QueryMetrics; @@ -38,53 +36,6 @@ */ public interface CursorFactory extends CursorMakerFactory { - @Override - default CursorMaker asCursorMaker(CursorBuildSpec spec) - { - return new CursorMaker() - { - @Override - public boolean canVectorize() - { - return CursorFactory.this.canVectorize(spec.getFilter(), spec.getVirtualColumns(), spec.isDescending()); - } - - @Override - public Cursor makeCursor() - { - return Iterables.getOnlyElement( - CursorFactory.this.makeCursors( - spec.getFilter(), - spec.getInterval(), - spec.getVirtualColumns(), - Granularities.ALL, - spec.isDescending(), - spec.getQueryMetrics() - ).toList() - ); - } - - @Override - public VectorCursor makeVectorCursor() - { - return CursorFactory.this.makeVectorCursor( - spec.getFilter(), - spec.getInterval(), - spec.getVirtualColumns(), - spec.isDescending(), - spec.getQueryContext().getVectorSize(), - spec.getQueryMetrics() - ); - } - - @Override - public void close() - { - // consuming sequences of CursorFactory are expected to close themselves. - } - }; - } - /** * Returns true if the provided combination of parameters can be handled by "makeVectorCursor". * diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 846b087bd6c9..c36b38c2074e 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -53,6 +53,7 @@ import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.vector.VectorOffset; import org.apache.druid.utils.CloseableUtils; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -487,6 +488,7 @@ private static class QueryableIndexCursor implements HistoricalCursor { private final Offset cursorOffset; private final ColumnSelectorFactory columnSelectorFactory; + private int markOffset = 0; QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory) { @@ -536,10 +538,27 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } + @Override + public void mark(DateTime mark) + { + markOffset = cursorOffset.getOffset(); + } + + @Override + public void resetMark() + { + cursorOffset.reset(); + // todo (clint): this kind of sucks, add a seek to offset to cursor... + while (markOffset > cursorOffset.getOffset()) { + advance(); + } + } + @Override public void reset() { cursorOffset.reset(); + markOffset = cursorOffset.getOffset(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index 7f5040a8ae13..59e35fa67c86 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -24,6 +24,7 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.ValueMatchers; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -44,6 +45,8 @@ public class RowBasedCursor implements Cursor private final ValueMatcher valueMatcher; private long rowId = 0; + private long markId = 0; + private DateTime markDate; public RowBasedCursor( final RowWalker rowWalker, @@ -113,12 +116,30 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } + @Override + public void mark(DateTime mark) + { + markId = rowId; + markDate = mark; + } + + @Override + public void resetMark() + { + rowId = markId; + rowWalker.reset(); + rowWalker.skipToDateTime(markDate, descending); + advanceToMatchingRow(); + } + @Override public void reset() { rowId = 0; + markId = 0; + markDate = descending ? interval.getEnd().minus(1) : interval.getStart(); rowWalker.reset(); - rowWalker.skipToDateTime(descending ? interval.getEnd().minus(1) : interval.getStart(), descending); + rowWalker.skipToDateTime(markDate, descending); advanceToMatchingRow(); } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index 2d3fc6a50bbb..dfe8bb14b249 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -21,9 +21,11 @@ import com.google.common.collect.Iterables; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -35,6 +37,59 @@ @PublicApi public interface StorageAdapter extends CursorFactory, ColumnInspector { + + /** + * Build a {@link CursorMaker} which can provide {@link Cursor} and {@link VectorCursor} (if capable) which allows + * scanning segments and creating {@link ColumnSelectorFactory} and + * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} respectively to read row values. + */ + @Override + default CursorMaker asCursorMaker(CursorBuildSpec spec) + { + return new CursorMaker() + { + @Override + public boolean canVectorize() + { + return StorageAdapter.this.canVectorize(spec.getFilter(), spec.getVirtualColumns(), spec.isDescending()); + } + + @Override + public Cursor makeCursor() + { + return Iterables.getOnlyElement( + StorageAdapter.this.makeCursors( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + Granularities.ALL, + spec.isDescending(), + spec.getQueryMetrics() + ).toList() + ); + } + + @Override + public VectorCursor makeVectorCursor() + { + return StorageAdapter.this.makeVectorCursor( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + spec.isDescending(), + spec.getQueryContext().getVectorSize(), + spec.getQueryMetrics() + ); + } + + @Override + public void close() + { + // consuming sequences of CursorFactory are expected to close themselves. + } + }; + } + Interval getInterval(); Indexed getAvailableDimensions(); Iterable getAvailableMetrics(); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index 8d3ab329e032..d10fdca7bf58 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -24,6 +24,7 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Arrays; @@ -63,6 +64,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor private Object currentVal; private List unnestListForCurrentRow; private boolean needInitialization; + private int markIndex = 0; public UnnestColumnValueSelectorCursor( @@ -240,10 +242,29 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } + @Override + public void mark(DateTime mark) + { + markIndex = index; + baseCursor.mark(mark); + } + + @Override + public void resetMark() + { + index = markIndex; + baseCursor.resetMark(); + getNextRow(); + if (unnestListForCurrentRow.isEmpty()) { + moveToNextNonEmptyRow(); + } + } + @Override public void reset() { index = 0; + markIndex = 0; needInitialization = true; baseCursor.reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 86874a5140a1..4d3e5e402ba3 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -29,6 +29,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -76,6 +77,7 @@ public class UnnestDimensionCursor implements Cursor private SingleIndexInts indexIntsForRow; private final int nullId; private final int idOffset; + private int markIndex = 0; public UnnestDimensionCursor( Cursor cursor, @@ -289,10 +291,29 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } + @Override + public void mark(DateTime mark) + { + markIndex = index; + baseCursor.mark(mark); + } + + @Override + public void resetMark() + { + index = markIndex; + baseCursor.resetMark(); + if (!baseCursor.isDone()) { + indexIntsForRow = new SingleIndexInts(); + indexedIntsForCurrentRow = dimSelector.getRow(); + } + } + @Override public void reset() { index = 0; + markIndex = 0; needInitialization = true; baseCursor.reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java index 372804a9c085..e5f58aaf1cb4 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java @@ -29,6 +29,7 @@ import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.filter.ValueMatchers; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -88,11 +89,16 @@ static class IncrementalIndexCursor implements Cursor private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher filterMatcher; private final int maxRowIndex; + private final IncrementalIndex.FactsHolder facts; + private final Interval interval; + private final boolean isDescending; private Iterator baseIter; private Iterable cursorIterable; private boolean emptyRange; private int numAdvanced; private boolean done; + private DateTime markDate; + private int markAdvanced = 0; IncrementalIndexCursor( IncrementalIndexStorageAdapter storageAdapter, @@ -114,7 +120,11 @@ static class IncrementalIndexCursor implements Cursor maxRowIndex = index.getLastRowIndex(); filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; - cursorIterable = index.getFacts().timeRangeIterable( + facts = index.getFacts(); + interval = actualInterval; + isDescending = descending; + markDate = isDescending ? interval.getEnd() : interval.getStart(); + cursorIterable = facts.timeRangeIterable( descending, actualInterval.getStartMillis(), actualInterval.getEndMillis() @@ -196,9 +206,31 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } + @Override + public void mark(DateTime mark) + { + markDate = mark; + markAdvanced = numAdvanced; + } + + @Override + public void resetMark() + { + numAdvanced = markAdvanced; + baseIter = facts.timeRangeIterable( + isDescending, + isDescending ? interval.getStartMillis() : markDate.getMillis(), + isDescending ? markDate.getMillis() : interval.getEndMillis() + ).iterator(); + + seekNextOffset(); + } + @Override public void reset() { + markAdvanced = 0; + markDate = isDescending ? interval.getEnd() : interval.getStart(); baseIter = cursorIterable.iterator(); if (numAdvanced == -1) { @@ -207,6 +239,11 @@ public void reset() Iterators.advance(baseIter, numAdvanced); } + seekNextOffset(); + } + + private void seekNextOffset() + { BaseQuery.checkInterrupted(); boolean foundMatched = false; diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index a478169633b6..8f5aa2a953f4 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowIdSupplier; import org.apache.druid.segment.column.ColumnCapabilities; +import org.joda.time.DateTime; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -147,6 +148,8 @@ void resetRowId() class JoinCursor implements Cursor { + private long joinMarkId = 0; + public void initialize() { matchCurrentPosition(); @@ -242,12 +245,28 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } + @Override + public void mark(DateTime mark) + { + leftCursor.mark(mark); + joinMarkId = joinColumnSelectorFactory.getRowId(); + } + + @Override + public void resetMark() + { + leftCursor.resetMark(); + joinMatcher.reset(); + joinColumnSelectorFactory.rowId = joinMarkId; + } + @Override public void reset() { leftCursor.reset(); joinMatcher.reset(); joinColumnSelectorFactory.resetRowId(); + joinMarkId = joinColumnSelectorFactory.getRowId(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 81cd032bc3f9..ad429c67182b 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -26,6 +26,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; +import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -139,6 +140,19 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } + @Override + public void mark(DateTime mark) + { + baseCursor.mark(mark); + } + + @Override + public void resetMark() + { + baseCursor.resetMark(); + advanceToMatch(); + } + @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index 07e9714cb981..b1012e9efcb9 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; +import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -208,6 +209,18 @@ public boolean isDoneOrInterrupted() return cursor.isDoneOrInterrupted(); } + @Override + public void mark(DateTime mark) + { + cursor.mark(mark); + } + + @Override + public void resetMark() + { + cursor.resetMark(); + } + @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 34cdab25a48b..96e0d60661b5 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -22,6 +22,7 @@ import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.joda.time.DateTime; /** * Used by {@link FrameTestUtil#readRowsFromAdapter} and {@link FrameTestUtil#readRowsFromCursor}. @@ -30,6 +31,7 @@ public class RowNumberUpdatingCursor implements Cursor { private final Cursor baseCursor; private final SettableLongVirtualColumn rowNumberVirtualColumn; + private long markRowNumber = 0; RowNumberUpdatingCursor(Cursor baseCursor, SettableLongVirtualColumn rowNumberVirtualColumn) { @@ -69,9 +71,24 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } + @Override + public void mark(DateTime mark) + { + baseCursor.mark(mark); + markRowNumber = rowNumberVirtualColumn.getValue(); + } + + @Override + public void resetMark() + { + rowNumberVirtualColumn.setValue(markRowNumber); + baseCursor.resetMark(); + } + @Override public void reset() { + markRowNumber = 0; rowNumberVirtualColumn.setValue(0); baseCursor.reset(); } diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index 2ed9e8431501..11fd52d5a3d6 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -26,6 +26,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; +import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.List; @@ -42,6 +43,7 @@ public class ListCursor implements Cursor { private final List baseList; private int index; + private int markIndex = 0; public ListCursor(List inputList) { @@ -213,9 +215,22 @@ public boolean isDoneOrInterrupted() return false; } + @Override + public void resetMark() + { + index = markIndex; + } + + @Override + public void mark(DateTime mark) + { + markIndex = index; + } + @Override public void reset() { index = 0; + markIndex = 0; } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index e99e70f909c1..d5d89f96e1a6 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -34,6 +34,7 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; import org.junit.Test; import javax.annotation.Nullable; @@ -134,9 +135,21 @@ public boolean isDoneOrInterrupted() } @Override - public void reset() + public void mark(DateTime mark) { + cursor.mark(mark); + } + @Override + public void resetMark() + { + cursor.resetMark(); + } + + @Override + public void reset() + { + cursor.reset(); } } } diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index 7bec5f1fa33d..10d7c20dc592 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -42,6 +42,8 @@ public class TestColumnValueSelector implements ColumnValueSelector, private Iterator iterator; private Object value; + private int rowCounter = 0; + private int markCounter = 0; public static TestColumnValueSelector of(Class clazz, Collection collection, DateTime time) { @@ -91,6 +93,7 @@ public ColumnCapabilities getColumnCapabilities(String column) public void advance() { value = iterator.next(); + rowCounter++; } @Override @@ -111,9 +114,27 @@ public boolean isDoneOrInterrupted() return isDone(); } + @Override + public void mark(DateTime mark) + { + markCounter = rowCounter; + } + + @Override + public void resetMark() + { + rowCounter = markCounter; + iterator = iteratorSupplier.get(); + for (int i = 0; i < markCounter; i++) { + iterator.next(); + } + } + @Override public void reset() { + rowCounter = 0; + markCounter = 0; iterator = iteratorSupplier.get(); } From 414a42014ae6a8c965fd3d2f31c54ecdc3f64631 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 17:16:18 -0700 Subject: [PATCH 15/74] VectorCursor doesnt need to be Closeable anymore --- .../ExpressionVectorSelectorBenchmark.java | 1 - ...bjectVectorColumnProcessorFactoryTest.java | 10 +- .../segment/columnar/FrameVectorCursor.java | 13 -- .../vector/VectorGroupByEngine.java | 72 ++++----- .../timeseries/TimeseriesQueryEngine.java | 2 - .../segment/QueryableIndexCursorMaker.java | 146 +++++++----------- .../druid/segment/vector/VectorCursor.java | 10 +- .../segment/FrameStorageAdapterTest.java | 4 +- .../druid/frame/testutil/FrameTestUtil.java | 51 +++--- .../VectorGroupByEngineIteratorTest.java | 1 + .../QueryableIndexStorageAdapterTest.java | 4 +- .../druid/segment/filter/BaseFilterTest.java | 16 +- .../NestedFieldColumnSelectorsTest.java | 3 +- .../ExpressionVectorSelectorsCastTest.java | 2 +- .../ExpressionVectorSelectorsTest.java | 1 - 15 files changed, 127 insertions(+), 209 deletions(-) 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 074caf85b861..9bd2e628511d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -174,7 +174,6 @@ public void scan(Blackhole blackhole) cursor.advance(); } } - closer.register(cursor); } } else { final Cursor cursor = cursorMaker.makeCursor(); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index 68b2b82e8c6d..0f8e4a0d76c8 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -57,9 +57,8 @@ public void setUp() @Test public void testRead() { - try (final CursorMaker maker = makeCursor(); - final VectorCursor cursor = maker.makeVectorCursor() - ) { + try (final CursorMaker maker = makeCursor()) { + final VectorCursor cursor = maker.makeVectorCursor(); final Supplier qualitySupplier = ColumnProcessors.makeVectorProcessor( "quality", ToObjectVectorColumnProcessorFactory.INSTANCE, @@ -186,9 +185,8 @@ private CursorMaker makeCursor() private List readColumn(final String column, final int limit) { - try (final CursorMaker maker = makeCursor(); - final VectorCursor cursor = maker.makeVectorCursor() - ) { + try (final CursorMaker maker = makeCursor()) { + final VectorCursor cursor = maker.makeVectorCursor(); final Supplier supplier = ColumnProcessors.makeVectorProcessor( column, ToObjectVectorColumnProcessorFactory.INSTANCE, diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java index da1cbf9b0cea..7524a9fb45f5 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java @@ -26,8 +26,6 @@ import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.vector.VectorOffset; -import java.io.IOException; - /** * A {@link VectorCursor} that is based on a {@link Frame}. * @@ -75,17 +73,6 @@ public void reset() offset.reset(); } - @Override - public void close() - { - try { - closer.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - @Override public int getMaxVectorSize() { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index f15be94f8695..51aee0c56ddd 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -120,48 +120,37 @@ public void close() }; } - try { - final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - final List dimensions = query.getDimensions().stream().map( - dimensionSpec -> { - if (dimensionSpec instanceof DefaultDimensionSpec) { - // Delegate creation of GroupByVectorColumnSelector to the column selector factory, so that - // virtual columns (like ExpressionVirtualColumn) can control their own grouping behavior. - return columnSelectorFactory.makeGroupByVectorColumnSelector( - dimensionSpec.getDimension(), - config.getDeferExpressionDimensions() - ); - } else { - return ColumnProcessors.makeVectorProcessor( - dimensionSpec, - GroupByVectorColumnProcessorFactory.instance(), - columnSelectorFactory - ); - } + final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final List dimensions = query.getDimensions().stream().map( + dimensionSpec -> { + if (dimensionSpec instanceof DefaultDimensionSpec) { + // Delegate creation of GroupByVectorColumnSelector to the column selector factory, so that + // virtual columns (like ExpressionVirtualColumn) can control their own grouping behavior. + return columnSelectorFactory.makeGroupByVectorColumnSelector( + dimensionSpec.getDimension(), + config.getDeferExpressionDimensions() + ); + } else { + return ColumnProcessors.makeVectorProcessor( + dimensionSpec, + GroupByVectorColumnProcessorFactory.instance(), + columnSelectorFactory + ); } - ).collect(Collectors.toList()); - - return new VectorGroupByEngineIterator( - query, - config, - processingConfig, - storageAdapter, - cursor, - interval, - dimensions, - processingBuffer, - fudgeTimestamp - ); - } - catch (Throwable e) { - try { - cursor.close(); - } - catch (Throwable e2) { - e.addSuppressed(e2); - } - throw e; - } + } + ).collect(Collectors.toList()); + + return new VectorGroupByEngineIterator( + query, + config, + processingConfig, + storageAdapter, + cursor, + interval, + dimensions, + processingBuffer, + fudgeTimestamp + ); } @Override @@ -323,7 +312,6 @@ public void close() throws IOException if (delegate != null) { closer.register(delegate); } - closer.register(cursor); closer.close(); } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index fd450a6359b6..2eb2742012bb 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -130,8 +130,6 @@ private Sequence> processVectorized( } final Closer closer = Closer.create(); - closer.register(cursor); - try { final VectorCursorGranularizer granularizer = VectorCursorGranularizer.create( adapter, diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index c36b38c2074e..45c45635967d 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -24,7 +24,6 @@ import com.google.common.base.Suppliers; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; @@ -118,20 +117,15 @@ public boolean canVectorize() } final CursorResources resources = resourcesSupplier.get(); - try { - final FilterBundle filterBundle = resources.filterBundle; - if (filterBundle != null) { - if (!filterBundle.canVectorizeMatcher()) { - return false; - } + final FilterBundle filterBundle = resources.filterBundle; + if (filterBundle != null) { + if (!filterBundle.canVectorizeMatcher()) { + return false; } - - // vector cursors can't iterate backwards yet - return !descending; - } - catch (ISE rethrow) { - throw CloseableUtils.closeInCatch(rethrow, resources); } + + // vector cursors can't iterate backwards yet + return !descending; } @Override @@ -226,78 +220,62 @@ public VectorCursor makeVectorCursor() // Wrap the remainder of cursor setup in a try, so if an error is encountered while setting it up, we don't // leak columns in the ColumnCache. - try { + // sanity check + if (!canVectorize()) { + close(); + throw new IllegalStateException("canVectorize()"); + } + if (metrics != null) { + metrics.vectorized(true); + } - // sanity check - if (!canVectorize()) { - close(); - throw new IllegalStateException("canVectorize()"); - } - if (metrics != null) { - metrics.vectorized(true); - } + final int startOffset; + final int endOffset; - final int startOffset; - final int endOffset; + if (interval.getStartMillis() > minDataTimestamp) { + startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows()); + } else { + startOffset = 0; + } - if (interval.getStartMillis() > minDataTimestamp) { - startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows()); - } else { - startOffset = 0; - } + if (interval.getEndMillis() <= maxDataTimestamp) { + endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows()); + } else { + endOffset = index.getNumRows(); + } - if (interval.getEndMillis() <= maxDataTimestamp) { - endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows()); - } else { - endOffset = index.getNumRows(); - } + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter can use + // an index + final VectorOffset baseOffset = + filterBundle == null || filterBundle.getIndex() == null + ? new NoFilterVectorOffset(vectorSize, startOffset, endOffset) + : new BitmapVectorOffset(vectorSize, filterBundle.getIndex().getBitmap(), startOffset, endOffset); - // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter can use - // an index - final VectorOffset baseOffset = - filterBundle == null || filterBundle.getIndex() == null - ? new NoFilterVectorOffset(vectorSize, startOffset, endOffset) - : new BitmapVectorOffset(vectorSize, filterBundle.getIndex().getBitmap(), startOffset, endOffset); + // baseColumnSelectorFactory using baseOffset is the column selector for filtering. + final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( + columnCache, + baseOffset + ); - // baseColumnSelectorFactory using baseOffset is the column selector for filtering. - final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( - columnCache, - baseOffset + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter needs to use + // a value matcher + if (filterBundle != null && filterBundle.getMatcherBundle() != null) { + final VectorValueMatcher vectorValueMatcher = filterBundle.getMatcherBundle() + .vectorMatcher(baseColumnSelectorFactory, baseOffset); + final VectorOffset filteredOffset = FilteredVectorOffset.create( + baseOffset, + vectorValueMatcher ); - // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter needs to use - // a value matcher - if (filterBundle != null && filterBundle.getMatcherBundle() != null) { - final VectorValueMatcher vectorValueMatcher = filterBundle.getMatcherBundle() - .vectorMatcher(baseColumnSelectorFactory, baseOffset); - final VectorOffset filteredOffset = FilteredVectorOffset.create( - baseOffset, - vectorValueMatcher - ); - - // Now create the cursor and column selector that will be returned to the caller. - final VectorColumnSelectorFactory filteredColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( - columnCache, - filteredOffset - ); - return new QueryableIndexVectorCursor( - filteredColumnSelectorFactory, - filteredOffset, - vectorSize, - resources - ); - } else { - return new QueryableIndexVectorCursor( - baseColumnSelectorFactory, - baseOffset, - vectorSize, - resources - ); - } - } - catch (Throwable t) { - throw CloseableUtils.closeAndWrapInCatch(t, resources); + // Now create the cursor and column selector that will be returned to the caller. + final VectorColumnSelectorFactory filteredColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( + columnCache, + filteredOffset + ); + return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize); + } else { + return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize); } } @@ -417,7 +395,6 @@ static int timeSearch( private static class QueryableIndexVectorCursor implements VectorCursor { - private final Closeable closer; private final int vectorSize; private final VectorOffset offset; private final VectorColumnSelectorFactory columnSelectorFactory; @@ -425,14 +402,12 @@ private static class QueryableIndexVectorCursor implements VectorCursor public QueryableIndexVectorCursor( final VectorColumnSelectorFactory vectorColumnSelectorFactory, final VectorOffset offset, - final int vectorSize, - final Closeable closer + final int vectorSize ) { this.columnSelectorFactory = vectorColumnSelectorFactory; this.vectorSize = vectorSize; this.offset = offset; - this.closer = closer; } @Override @@ -471,17 +446,6 @@ public void reset() { offset.reset(); } - - @Override - public void close() - { - try { - closer.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } } private static class QueryableIndexCursor implements HistoricalCursor diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index 86b3e305f0d9..c3806fbccf34 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java @@ -22,8 +22,6 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndexCursorMaker; -import java.io.Closeable; - /** * Vectorized cursor used during query execution. VectorCursors are returned by * {@link org.apache.druid.segment.StorageAdapter#asCursorMaker(CursorBuildSpec)} and are created by @@ -48,7 +46,7 @@ * * @see org.apache.druid.segment.Cursor, the non-vectorized version. */ -public interface VectorCursor extends VectorSizeInspector, Closeable +public interface VectorCursor extends VectorSizeInspector { /** * Returns a vectorized column selector factory. @@ -70,10 +68,4 @@ public interface VectorCursor extends VectorSizeInspector, Closeable */ @SuppressWarnings("unused") /* Not currently used, but anticipated to be used by topN in the future. */ void reset(); - - /** - * Close the cursor and release its resources. - */ - @Override - void close(); } diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index 3fa224a62b5f..db2663c2b619 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -394,9 +394,9 @@ private void assertVectorCursorsMatch(final Function> queryableRows = - FrameTestUtil.readRowsFromVectorCursor(maker.makeVectorCursor(), signature); + FrameTestUtil.readRowsFromVectorCursor(maker.makeVectorCursor(), signature).withBaggage(maker); final Sequence> frameRows = - FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameMaker.makeVectorCursor()), signature); + FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameMaker.makeVectorCursor()), signature).withBaggage(frameMaker); FrameTestUtil.assertRowsEqual(queryableRows, frameRows); } else { maker.close(); 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 1ae8561cd602..4c3ca581f52d 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 @@ -338,46 +338,41 @@ public static Sequence> readRowsFromCursor(final Cursor cursor, fin public static Sequence> readRowsFromVectorCursor(final VectorCursor cursor, final RowSignature signature) { - try { - final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - final List> columnReaders = new ArrayList<>(); + final List> columnReaders = new ArrayList<>(); - for (int i = 0; i < signature.size(); i++) { - final String columnName = signature.getColumnName(i); - final Supplier columnReader = ColumnProcessors.makeVectorProcessor( - columnName, - RowReadingVectorColumnProcessorFactory.INSTANCE, - columnSelectorFactory - ); - - columnReaders.add(columnReader); - } + for (int i = 0; i < signature.size(); i++) { + final String columnName = signature.getColumnName(i); + final Supplier columnReader = ColumnProcessors.makeVectorProcessor( + columnName, + RowReadingVectorColumnProcessorFactory.INSTANCE, + columnSelectorFactory + ); - final List> retVal = new ArrayList<>(); + columnReaders.add(columnReader); + } - while (!cursor.isDone()) { - final int vectorSize = cursor.getCurrentVectorSize(); - final List columns = columnReaders.stream().map(Supplier::get).collect(Collectors.toList()); + final List> retVal = new ArrayList<>(); - for (int i = 0; i < vectorSize; i++) { - final List row = new ArrayList<>(); + while (!cursor.isDone()) { + final int vectorSize = cursor.getCurrentVectorSize(); + final List columns = columnReaders.stream().map(Supplier::get).collect(Collectors.toList()); - for (final Object[] column : columns) { - row.add(column[i]); - } + for (int i = 0; i < vectorSize; i++) { + final List row = new ArrayList<>(); - retVal.add(row); + for (final Object[] column : columns) { + row.add(column[i]); } - cursor.advance(); + retVal.add(row); } - return Sequences.simple(retVal); - } - finally { - cursor.close(); + cursor.advance(); } + + return Sequences.simple(retVal); } private static Supplier dimensionSelectorReader(final DimensionSelector selector) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java index 91a455ff83f5..8403131e5b43 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java @@ -94,5 +94,6 @@ VectorGrouper makeGrouper() }; iterator.close(); Mockito.verify(grouperCaptor.getValue()).close(); + maker.close(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java index f4b4d41145e7..fcba838fd0de 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java @@ -70,9 +70,7 @@ public void setUp() final QueryableIndex index = TestIndex.getMMappedTestIndex(); final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); if (vectorize) { - final VectorCursor cursor = closer.register( - adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeVectorCursor() - ); + final VectorCursor cursor = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)).makeVectorCursor(); final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); 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 f232b70b82ba..b8bbdee3b67a 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 @@ -870,8 +870,8 @@ private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFil ); - try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(null)); - final VectorCursor cursor = maker.makeVectorCursor()) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(null))) { + final VectorCursor cursor = maker.makeVectorCursor(); final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), maybeOptimize(dimFilter) @@ -997,8 +997,8 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(postFilteringFilter)); - final VectorCursor cursor = maker.makeVectorCursor()) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(postFilteringFilter))) { + final VectorCursor cursor = maker.makeVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1022,8 +1022,8 @@ private List selectColumnValuesMatchingFilterUsingVectorCursor( final String selectColumn ) { - try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter))); - final VectorCursor cursor = maker.makeVectorCursor()) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter)))) { + final VectorCursor cursor = maker.makeVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1049,8 +1049,8 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur ) { final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); - try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter))); - final VectorCursor cursor = maker.makeVectorCursor()) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter)))) { + final VectorCursor cursor = maker.makeVectorCursor(); final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); final List values = new ArrayList<>(); 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 46852491ce08..e66460788360 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 @@ -373,8 +373,7 @@ private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumn Assert.assertEquals(1, segments.size()); StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build(); - VectorCursor cursor = storageAdapter.asCursorMaker(buildSpec).makeVectorCursor(); - closer.register(cursor); + VectorCursor cursor = closer.register(storageAdapter.asCursorMaker(buildSpec)).makeVectorCursor(); return cursor.getColumnSelectorFactory(); } } 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 0caf9b0eab59..b96bf9ebbfbd 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 @@ -121,7 +121,7 @@ public static void testCast( .setVirtualColumns(virtualColumns) .setInterval(index.getDataInterval()) .build(); - VectorCursor cursor = closer.register(storageAdapter.asCursorMaker(buildSpec).makeVectorCursor()); + VectorCursor cursor = closer.register(storageAdapter.asCursorMaker(buildSpec)).makeVectorCursor(); ColumnCapabilities capabilities = INDEX.getColumnCapabilities(column); 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 2b9f23afad6a..dedd6ad51a65 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 @@ -312,7 +312,6 @@ public static void sanityTestVectorizedExpressionSelectors( cursor.advance(); } } - closer.register(cursor); final Cursor nonVectorized = maker.makeCursor(); From 38df2aa1136a513c32dc08962507a196ff84d8aa Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 17:57:56 -0700 Subject: [PATCH 16/74] add mark/resetMark to Offset --- .../frame/segment/FrameFilteredOffset.java | 13 +++++++++++ .../apache/druid/segment/BitmapOffset.java | 21 ++++++++++++++++++ .../apache/druid/segment/FilteredOffset.java | 13 +++++++++++ .../segment/QueryableIndexCursorMaker.java | 22 ++++++++++++------- .../druid/segment/SimpleAscendingOffset.java | 14 ++++++++++++ .../druid/segment/SimpleDescendingOffset.java | 14 ++++++++++++ .../org/apache/druid/segment/data/Offset.java | 4 ++++ 7 files changed, 93 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java index 3db294801c03..04b46e2fa606 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java @@ -81,6 +81,19 @@ public void setCurrentOffset(int currentOffset) } } + @Override + public void mark() + { + baseOffset.mark(); + } + + @Override + public void resetMark() + { + baseOffset.resetMark(); + incrementIfNeededOnCreationOrReset(); + } + @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index ab552f0d5a39..536eda57ef1d 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; @@ -30,6 +31,7 @@ import org.apache.druid.segment.data.ReadableOffset; import org.roaringbitmap.IntIterator; +import javax.annotation.Nullable; import java.util.Arrays; import java.util.HashSet; @@ -126,6 +128,9 @@ private static String factorizeFullness(long bitmapCardinality, long numRows) private final IntIterator iteratorForReset; private final int valueForReset; private int value; + @Nullable + private IntIterator iteratorForMark; + private int valueForMark; public static IntIterator getReverseBitmapOffsetIterator(ImmutableBitmap bitmapIndex) { @@ -176,6 +181,8 @@ private BitmapOffset(String fullness, IntIterator iterator, int value) this.iteratorForReset = safeClone(iterator); this.valueForReset = value; this.value = value; + this.iteratorForMark = null; + this.valueForMark = value; } @Override @@ -194,6 +201,20 @@ public boolean withinBounds() return value > INVALID_VALUE; } + @Override + public void mark() + { + iteratorForMark = safeClone(iterator); + valueForMark = value; + } + + @Override + public void resetMark() + { + iterator = Preconditions.checkNotNull(iteratorForMark); + value = valueForMark; + } + @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index 755a399f8e0d..20b2fa17299f 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -56,6 +56,19 @@ public boolean withinBounds() return baseOffset.withinBounds(); } + @Override + public void mark() + { + baseOffset.mark(); + } + + @Override + public void resetMark() + { + baseOffset.resetMark(); + incrementIfNeededOnCreationOrReset(); + } + @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 45c45635967d..21c877a9944b 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -452,7 +452,6 @@ private static class QueryableIndexCursor implements HistoricalCursor { private final Offset cursorOffset; private final ColumnSelectorFactory columnSelectorFactory; - private int markOffset = 0; QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory) { @@ -505,24 +504,19 @@ public boolean isDoneOrInterrupted() @Override public void mark(DateTime mark) { - markOffset = cursorOffset.getOffset(); + cursorOffset.mark(); } @Override public void resetMark() { - cursorOffset.reset(); - // todo (clint): this kind of sucks, add a seek to offset to cursor... - while (markOffset > cursorOffset.getOffset()) { - advance(); - } + cursorOffset.resetMark(); } @Override public void reset() { cursorOffset.reset(); - markOffset = cursorOffset.getOffset(); } } @@ -566,6 +560,18 @@ public boolean withinBounds() return timeInRange(timestamps.getLongSingleValueRow(baseOffset.getOffset())); } + @Override + public void mark() + { + baseOffset.mark(); + } + + @Override + public void resetMark() + { + baseOffset.resetMark(); + } + @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java index b4c6111804eb..07e107c206e7 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java @@ -28,6 +28,7 @@ public class SimpleAscendingOffset extends SimpleSettableOffset private final int rowCount; private final int initialOffset; private int currentOffset; + private int markOffset; public SimpleAscendingOffset(int rowCount) { @@ -38,6 +39,7 @@ private SimpleAscendingOffset(int initialOffset, int rowCount) { this.initialOffset = initialOffset; this.currentOffset = initialOffset; + this.markOffset = initialOffset; this.rowCount = rowCount; } @@ -59,6 +61,18 @@ public void setCurrentOffset(int currentOffset) this.currentOffset = currentOffset; } + @Override + public void mark() + { + markOffset = currentOffset; + } + + @Override + public void resetMark() + { + currentOffset = markOffset; + } + @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java index fe8aba36fb99..73dc678c4ee3 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java @@ -28,6 +28,7 @@ public class SimpleDescendingOffset extends SimpleSettableOffset private final int rowCount; private final int initialOffset; private int currentOffset; + private int markOffset; public SimpleDescendingOffset(int rowCount) { @@ -39,6 +40,7 @@ private SimpleDescendingOffset(int initialOffset, int rowCount) this.rowCount = rowCount; this.initialOffset = initialOffset; this.currentOffset = initialOffset; + this.markOffset = initialOffset; } @Override @@ -53,6 +55,18 @@ public boolean withinBounds() return currentOffset >= 0; } + @Override + public void mark() + { + markOffset = currentOffset; + } + + @Override + public void resetMark() + { + currentOffset = markOffset; + } + @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/data/Offset.java b/processing/src/main/java/org/apache/druid/segment/data/Offset.java index ea0c5e046e22..8d52a706ff82 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/Offset.java +++ b/processing/src/main/java/org/apache/druid/segment/data/Offset.java @@ -47,6 +47,10 @@ public abstract class Offset implements ReadableOffset, Cloneable @CalledFromHotLoop public abstract boolean withinBounds(); + public abstract void mark(); + + public abstract void resetMark(); + /** * Resets the Offset to the position it was created or cloned with. */ From d6130e070a035f4d194947fe3f10ea84a3c74c42 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 18:00:04 -0700 Subject: [PATCH 17/74] remove unused closer --- .../frame/segment/columnar/FrameCursorMakerFactory.java | 4 ++-- .../druid/frame/segment/columnar/FrameVectorCursor.java | 6 +----- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java index 81945b795be9..776dfe67d0fc 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java @@ -140,7 +140,7 @@ public VectorCursor makeVectorCursor() ); if (filterToUse == null) { - return new FrameVectorCursor(baseOffset, baseColumnSelectorFactory, closer); + return new FrameVectorCursor(baseOffset, baseColumnSelectorFactory); } else { final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseColumnSelectorFactory); final FilteredVectorOffset filteredOffset = FilteredVectorOffset.create( @@ -155,7 +155,7 @@ public VectorCursor makeVectorCursor() spec.getVirtualColumns() ); - return new FrameVectorCursor(filteredOffset, filteredColumnSelectorFactory, closer); + return new FrameVectorCursor(filteredOffset, filteredColumnSelectorFactory); } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java index 7524a9fb45f5..e7789ed03758 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java @@ -20,7 +20,6 @@ package org.apache.druid.frame.segment.columnar; import org.apache.druid.frame.Frame; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.BaseQuery; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; @@ -35,17 +34,14 @@ public class FrameVectorCursor implements VectorCursor { private final VectorOffset offset; private final VectorColumnSelectorFactory columnSelectorFactory; - private final Closer closer; FrameVectorCursor( final VectorOffset offset, - final VectorColumnSelectorFactory columnSelectorFactory, - final Closer closer + final VectorColumnSelectorFactory columnSelectorFactory ) { this.offset = offset; this.columnSelectorFactory = columnSelectorFactory; - this.closer = closer; } @Override From 460f6dfde87f883deab4b3ab0a04e3716ed371c0 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 23:12:34 -0700 Subject: [PATCH 18/74] fix offset mark/resetMark --- .../frame/segment/FrameFilteredOffset.java | 1 - .../apache/druid/segment/BitmapOffset.java | 34 +++++++---- .../apache/druid/segment/FilteredOffset.java | 1 - .../segment/QueryableIndexCursorMaker.java | 7 +++ .../druid/segment/BitmapOffsetTest.java | 57 +++++++++++++++++++ 5 files changed, 88 insertions(+), 12 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java index 04b46e2fa606..1ed10bf097c7 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java @@ -91,7 +91,6 @@ public void mark() public void resetMark() { baseOffset.resetMark(); - incrementIfNeededOnCreationOrReset(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index 536eda57ef1d..c7bc893f9a78 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -158,7 +158,7 @@ private BitmapOffset(ImmutableBitmap bitmapIndex, boolean descending, long numRo increment(); // It's important to set iteratorForReset and valueForReset after calling increment(), because only after that the // iterator and the value are in proper initial state. - this.iteratorForReset = safeClone(iterator); + this.iteratorForReset = iterator.clone(); this.valueForReset = value; } @@ -174,15 +174,21 @@ private IntIterator newIterator(ImmutableBitmap bitmapIndex, boolean descending) /** * Constructor for {@link #clone()}. */ - private BitmapOffset(String fullness, IntIterator iterator, int value) + private BitmapOffset( + String fullness, + IntIterator iterator, + int value, + @Nullable IntIterator iteratorForMark, + int markValue + ) { this.fullness = fullness; this.iterator = iterator; - this.iteratorForReset = safeClone(iterator); + this.iteratorForReset = iterator.clone(); this.valueForReset = value; this.value = value; - this.iteratorForMark = null; - this.valueForMark = value; + this.iteratorForMark = iteratorForMark; + this.valueForMark = markValue; } @Override @@ -204,22 +210,24 @@ public boolean withinBounds() @Override public void mark() { - iteratorForMark = safeClone(iterator); + iteratorForMark = iterator.clone(); valueForMark = value; } @Override public void resetMark() { - iterator = Preconditions.checkNotNull(iteratorForMark); + iterator = Preconditions.checkNotNull(iteratorForMark).clone(); value = valueForMark; } @Override public void reset() { - iterator = safeClone(iteratorForReset); + iterator = iteratorForReset.clone(); value = valueForReset; + valueForMark = valueForReset; + iteratorForMark = null; } @Override @@ -232,7 +240,13 @@ public ReadableOffset getBaseReadableOffset() @Override public Offset clone() { - return new BitmapOffset(fullness, safeClone(iterator), value); + return new BitmapOffset( + fullness, + iterator.clone(), + value, + iteratorForMark != null ? iteratorForMark.clone() : null, + valueForMark + ); } @Override @@ -252,6 +266,6 @@ private static IntIterator safeClone(IntIterator iterator) { // Calling clone() on empty iterators from RoaringBitmap library sometimes fails with NPE, // see https://github.com/apache/druid/issues/4709, https://github.com/RoaringBitmap/RoaringBitmap/issues/177 - return iterator.hasNext() ? iterator.clone() : EmptyIntIterator.instance(); + return iterator.clone(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index 20b2fa17299f..b053640581a1 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -66,7 +66,6 @@ public void mark() public void resetMark() { baseOffset.resetMark(); - incrementIfNeededOnCreationOrReset(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 21c877a9944b..9e460ecc473f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -452,6 +452,7 @@ private static class QueryableIndexCursor implements HistoricalCursor { private final Offset cursorOffset; private final ColumnSelectorFactory columnSelectorFactory; + private int markOffset = 0; QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory) { @@ -504,6 +505,7 @@ public boolean isDoneOrInterrupted() @Override public void mark(DateTime mark) { +// markOffset = cursorOffset.getOffset(); cursorOffset.mark(); } @@ -511,12 +513,17 @@ public void mark(DateTime mark) public void resetMark() { cursorOffset.resetMark(); +// cursorOffset.reset(); +// while (markOffset > cursorOffset.getOffset()) { +// advance(); +// } } @Override public void reset() { cursorOffset.reset(); +// markOffset = cursorOffset.getOffset(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java b/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java index ef892496bf1a..8715180a26b8 100644 --- a/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java +++ b/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java @@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.collections.bitmap.BitSetBitmapFactory; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ConciseBitmapFactory; @@ -36,6 +38,7 @@ import org.junit.runners.Parameterized; import java.util.List; +import java.util.concurrent.ThreadLocalRandom; /** */ @@ -103,4 +106,58 @@ public void testSanity() } Assert.assertEquals(count, expected.length); } + + @Test + public void testMarkResetMark() + { + MutableBitmap mutable = factory.makeEmptyMutableBitmap(); + IntList rows = new IntArrayList(); + for (int i = 0; i < 1_000_000; i++) { + if (ThreadLocalRandom.current().nextBoolean()) { + rows.add(i); + mutable.add(i); + } + } + + ImmutableBitmap bitmap = factory.makeImmutableBitmap(mutable); + final BitmapOffset offset = BitmapOffset.of(bitmap, descending, bitmap.size()); + final IntList expected; + if (descending) { + expected = new IntArrayList(rows.size()); + for (int i = rows.size() - 1; i >= 0; i--) { + expected.add(rows.getInt(i)); + } + } else { + expected = rows; + } + + int count = 0; + int mark = rows.size() / 2; + while (offset.withinBounds()) { + Assert.assertEquals(expected.getInt(count), offset.getOffset()); + + if (count == mark) { + offset.mark(); + } + + ++count; + offset.increment(); + } + + offset.resetMark(); + count = mark; + while (offset.withinBounds()) { + Assert.assertEquals(expected.getInt(count), offset.getOffset()); + + if (count == mark) { + offset.mark(); + } + + ++count; + offset.increment(); + } + + Assert.assertEquals(count, expected.size()); + } + } From d225b1665aeb45d2b2c75da28e7390177e3cb32d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 23:43:06 -0700 Subject: [PATCH 19/74] make IndexTaskTest.testNumShardsAndPartitionDimensionsProvided actually test stuff --- .../indexing/common/task/IndexTaskTest.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index c01ba7ddea9f..4619bdc5a1f7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -783,17 +783,19 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception final List hashes = new ArrayList<>(); final DimensionSelector selector = cursor.getColumnSelectorFactory() .makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); - final int hash = HashPartitionFunction.MURMUR3_32_ABS.hash( - HashBasedNumberedShardSpec.serializeGroupKey( - jsonMapper, - Collections.singletonList(selector.getObject()) - ), - hashBasedNumberedShardSpec.getNumBuckets() - ); - hashes.add(hash); - cursor.advance(); + while (!cursor.isDone()) { + final int hash = HashPartitionFunction.MURMUR3_32_ABS.hash( + HashBasedNumberedShardSpec.serializeGroupKey( + jsonMapper, + // list of list because partitioning extractKeys uses InputRow.getDimension which always returns a List + Collections.singletonList(Collections.singletonList(selector.getObject())) + ), + hashBasedNumberedShardSpec.getNumBuckets() + ); + hashes.add(hash); + cursor.advance(); + } - // todo (clint): wtf does this mean Assert.assertTrue(hashes.stream().allMatch(h -> h.intValue() == hashes.get(0))); } } From 815f0bcf651301e8e814c124c448021e9b983b08 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 29 Jul 2024 23:52:55 -0700 Subject: [PATCH 20/74] remove unused --- .../main/java/org/apache/druid/segment/BitmapOffset.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index c7bc893f9a78..54aca419a07e 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -24,7 +24,6 @@ import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.collections.bitmap.WrappedImmutableRoaringBitmap; -import org.apache.druid.extendedset.intset.EmptyIntIterator; import org.apache.druid.java.util.common.RE; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.Offset; @@ -261,11 +260,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) inspector.visit("iterator", iterator); inspector.visit("fullness", fullness); } - - private static IntIterator safeClone(IntIterator iterator) - { - // Calling clone() on empty iterators from RoaringBitmap library sometimes fails with NPE, - // see https://github.com/apache/druid/issues/4709, https://github.com/RoaringBitmap/RoaringBitmap/issues/177 - return iterator.clone(); - } } From 546ab37a61135a54fb05fb2af0c367851099cbc7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 30 Jul 2024 00:25:13 -0700 Subject: [PATCH 21/74] style --- .../apache/druid/segment/QueryableIndexCursorMaker.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 9e460ecc473f..21c877a9944b 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -452,7 +452,6 @@ private static class QueryableIndexCursor implements HistoricalCursor { private final Offset cursorOffset; private final ColumnSelectorFactory columnSelectorFactory; - private int markOffset = 0; QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory) { @@ -505,7 +504,6 @@ public boolean isDoneOrInterrupted() @Override public void mark(DateTime mark) { -// markOffset = cursorOffset.getOffset(); cursorOffset.mark(); } @@ -513,17 +511,12 @@ public void mark(DateTime mark) public void resetMark() { cursorOffset.resetMark(); -// cursorOffset.reset(); -// while (markOffset > cursorOffset.getOffset()) { -// advance(); -// } } @Override public void reset() { cursorOffset.reset(); -// markOffset = cursorOffset.getOffset(); } } From cdf3c3926a0fd26f54e1c664dd69683e57f155f7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 30 Jul 2024 01:58:56 -0700 Subject: [PATCH 22/74] rename resetMark to resetToMark, javadocs --- .../druid/frame/segment/FrameCursor.java | 2 +- .../frame/segment/FrameFilteredOffset.java | 4 +-- .../druid/query/topn/BaseTopNAlgorithm.java | 4 +-- .../apache/druid/segment/BitmapOffset.java | 2 +- .../java/org/apache/druid/segment/Cursor.java | 33 ++++++++++++++++++- .../apache/druid/segment/FilteredOffset.java | 4 +-- .../segment/QueryableIndexCursorMaker.java | 8 ++--- .../apache/druid/segment/RowBasedCursor.java | 2 +- .../druid/segment/SimpleAscendingOffset.java | 2 +- .../druid/segment/SimpleDescendingOffset.java | 2 +- .../apache/druid/segment/StorageAdapter.java | 3 +- .../UnnestColumnValueSelectorCursor.java | 4 +-- .../druid/segment/UnnestDimensionCursor.java | 4 +-- .../org/apache/druid/segment/data/Offset.java | 14 +++++++- .../IncrementalIndexCursorMaker.java | 2 +- .../druid/segment/join/HashJoinEngine.java | 4 +-- .../druid/segment/join/PostJoinCursor.java | 4 +-- .../druid/frame/TestArrayStorageAdapter.java | 4 +-- .../testutil/RowNumberUpdatingCursor.java | 4 +-- .../druid/segment/BitmapOffsetTest.java | 4 +-- .../org/apache/druid/segment/ListCursor.java | 2 +- .../segment/join/PostJoinCursorTest.java | 4 +-- .../selector/TestColumnValueSelector.java | 2 +- 23 files changed, 81 insertions(+), 37 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 67169a302f16..c67c2b0e671c 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -86,7 +86,7 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { offset.setCurrentOffset(markOffset); } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java index 1ed10bf097c7..6aa5b81aeb02 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java @@ -88,9 +88,9 @@ public void mark() } @Override - public void resetMark() + public void resetToMark() { - baseOffset.resetMark(); + baseOffset.resetToMark(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java index f1cf009b4249..16c6b498c996 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java @@ -123,7 +123,7 @@ private void runWithCardinalityKnown( resetAggregators(aggregatesStore); numProcessed += numToProcess; - params.getCursor().resetMark(); + params.getCursor().resetToMark(); } if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); @@ -152,7 +152,7 @@ private void runWithCardinalityUnknown( long processedRows = scanAndAggregate(params, null, aggregatesStore); updateResults(params, null, aggregatesStore, resultBuilder); resetAggregators(aggregatesStore); - params.getCursor().resetMark(); + params.getCursor().resetToMark(); if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); queryMetrics.stopRecordingScanTime(); diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index 54aca419a07e..a6f424ea5fbf 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -214,7 +214,7 @@ public void mark() } @Override - public void resetMark() + public void resetToMark() { iterator = Preconditions.checkNotNull(iteratorForMark).clone(); value = valueForMark; diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 2459edb4f606..01c34fca1f85 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import org.apache.druid.query.QueryInterruptedException; import org.joda.time.DateTime; /** @@ -35,15 +36,45 @@ */ public interface Cursor { + /** + * Get a {@link ColumnSelectorFactory} whose selectors will be backed by the row values at the current position of + * the cursor + */ ColumnSelectorFactory getColumnSelectorFactory(); + /** + * Advance to the next row in the cursor, checking if thread has been interrupted after advancing and possibly + * throwing {@link QueryInterruptedException} if so + */ void advance(); + + /** + * Advance to the next row in the cursor + */ void advanceUninterruptibly(); + /** + * Check if there are any additional rows in the cursor + */ boolean isDone(); + + /** + * Check if there are any additional rows in the cursor, or if the thread has been interrupted + */ boolean isDoneOrInterrupted(); + /** + * Mark a position on the cursor which can recalled with {@link #resetToMark()} + */ void mark(DateTime mark); - void resetMark(); + + /** + * Reset to position set by {@link #mark(DateTime)} + */ + void resetToMark(); + + /** + * Reset to start of cursor + */ void reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index b053640581a1..abbb26c417bb 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -63,9 +63,9 @@ public void mark() } @Override - public void resetMark() + public void resetToMark() { - baseOffset.resetMark(); + baseOffset.resetToMark(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index 21c877a9944b..79389b8bf09f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -508,9 +508,9 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { - cursorOffset.resetMark(); + cursorOffset.resetToMark(); } @Override @@ -567,9 +567,9 @@ public void mark() } @Override - public void resetMark() + public void resetToMark() { - baseOffset.resetMark(); + baseOffset.resetToMark(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index 59e35fa67c86..b9df2754b557 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -124,7 +124,7 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { rowId = markId; rowWalker.reset(); diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java index 07e107c206e7..b8b16ead19c8 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java @@ -68,7 +68,7 @@ public void mark() } @Override - public void resetMark() + public void resetToMark() { currentOffset = markOffset; } diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java index 73dc678c4ee3..3ba6f06f8a3b 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java @@ -62,7 +62,7 @@ public void mark() } @Override - public void resetMark() + public void resetToMark() { currentOffset = markOffset; } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index dfe8bb14b249..0607f9206bca 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -41,7 +41,8 @@ public interface StorageAdapter extends CursorFactory, ColumnInspector /** * Build a {@link CursorMaker} which can provide {@link Cursor} and {@link VectorCursor} (if capable) which allows * scanning segments and creating {@link ColumnSelectorFactory} and - * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} respectively to read row values. + * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} respectively to read row values at the cursor + * position. */ @Override default CursorMaker asCursorMaker(CursorBuildSpec spec) diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index d10fdca7bf58..13a3a701a2bf 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -250,10 +250,10 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { index = markIndex; - baseCursor.resetMark(); + baseCursor.resetToMark(); getNextRow(); if (unnestListForCurrentRow.isEmpty()) { moveToNextNonEmptyRow(); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 4d3e5e402ba3..e6773d9095f0 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -299,10 +299,10 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { index = markIndex; - baseCursor.resetMark(); + baseCursor.resetToMark(); if (!baseCursor.isDone()) { indexIntsForRow = new SingleIndexInts(); indexedIntsForCurrentRow = dimSelector.getRow(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/Offset.java b/processing/src/main/java/org/apache/druid/segment/data/Offset.java index 8d52a706ff82..1f521e9f4a59 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/Offset.java +++ b/processing/src/main/java/org/apache/druid/segment/data/Offset.java @@ -41,15 +41,27 @@ @SubclassesMustBePublic public abstract class Offset implements ReadableOffset, Cloneable { + /** + * Move to next offset + */ @CalledFromHotLoop public abstract void increment(); + /** + * Check if current offset position is within the valid range + */ @CalledFromHotLoop public abstract boolean withinBounds(); + /** + * Mark the current offset to recall later with {@link #resetToMark()} + */ public abstract void mark(); - public abstract void resetMark(); + /** + * Reset to the offset set by {@link #mark()} + */ + public abstract void resetToMark(); /** * Resets the Offset to the position it was created or cloned with. diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java index e5f58aaf1cb4..69b2f749f51b 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java @@ -214,7 +214,7 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { numAdvanced = markAdvanced; baseIter = facts.timeRangeIterable( diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index 8f5aa2a953f4..2666552ee974 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -253,9 +253,9 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { - leftCursor.resetMark(); + leftCursor.resetToMark(); joinMatcher.reset(); joinColumnSelectorFactory.rowId = joinMarkId; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index ad429c67182b..1c023ba26eb0 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -147,9 +147,9 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { - baseCursor.resetMark(); + baseCursor.resetToMark(); advanceToMatch(); } diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index b1012e9efcb9..9154c41b3345 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -216,9 +216,9 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { - cursor.resetMark(); + cursor.resetToMark(); } @Override diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 96e0d60661b5..27a0688032a2 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -79,10 +79,10 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { rowNumberVirtualColumn.setValue(markRowNumber); - baseCursor.resetMark(); + baseCursor.resetToMark(); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java b/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java index 8715180a26b8..7cb55e62ac0f 100644 --- a/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java +++ b/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java @@ -108,7 +108,7 @@ public void testSanity() } @Test - public void testMarkResetMark() + public void testMarkResetToMark() { MutableBitmap mutable = factory.makeEmptyMutableBitmap(); IntList rows = new IntArrayList(); @@ -144,7 +144,7 @@ public void testMarkResetMark() offset.increment(); } - offset.resetMark(); + offset.resetToMark(); count = mark; while (offset.withinBounds()) { Assert.assertEquals(expected.getInt(count), offset.getOffset()); diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index 11fd52d5a3d6..d781f5893158 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -216,7 +216,7 @@ public boolean isDoneOrInterrupted() } @Override - public void resetMark() + public void resetToMark() { index = markIndex; } diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index d5d89f96e1a6..9e7ee660fa51 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -141,9 +141,9 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { - cursor.resetMark(); + cursor.resetToMark(); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index 10d7c20dc592..84be585155f9 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -121,7 +121,7 @@ public void mark(DateTime mark) } @Override - public void resetMark() + public void resetToMark() { rowCounter = markCounter; iterator = iteratorSupplier.get(); From 47462129f3be8314c261dd09fe06ac4aeaeffcd2 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 30 Jul 2024 18:34:48 -0700 Subject: [PATCH 23/74] update benchmarks --- .../benchmark/query/GroupByBenchmark.java | 12 ++++++------ .../benchmark/query/TimeseriesBenchmark.java | 19 +++++++++++-------- .../druid/benchmark/query/TopNBenchmark.java | 17 ++++++++++------- 3 files changed, 27 insertions(+), 21 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 346b8e4a7f7b..9fb4343af3a1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -121,12 +121,12 @@ import java.util.concurrent.TimeUnit; @State(Scope.Benchmark) -@Fork(value = 2) -@Warmup(iterations = 10) -@Measurement(iterations = 25) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 15) public class GroupByBenchmark { - @Param({"2", "4"}) + @Param({"4"}) private int numProcessingThreads; @Param({"-1"}) @@ -141,7 +141,7 @@ public class GroupByBenchmark @Param({"all", "day"}) private String queryGranularity; - @Param({"force", "false"}) + @Param({"false", "true"}) private String vectorize; private static final Logger log = new Logger(GroupByBenchmark.class); @@ -514,7 +514,7 @@ public String getFormatString() @State(Scope.Benchmark) public static class IncrementalIndexState { - @Param({"onheap", "offheap"}) + @Param({"onheap"}) private String indexType; IncrementalIndex incIndex; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index 97d20ca68ae8..5ea82211091f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -26,7 +26,7 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.Druids; @@ -101,8 +101,8 @@ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 10) -@Measurement(iterations = 25) +@Warmup(iterations = 5) +@Measurement(iterations = 15) public class TimeseriesBenchmark { @Param({"750000"}) @@ -114,6 +114,9 @@ public class TimeseriesBenchmark @Param({"true", "false"}) private boolean descending; + @Param({"all", "hour"}) + private String queryGranularity; + private static final Logger log = new Logger(TimeseriesBenchmark.class); private static final int RNG_SEED = 9999; private static final IndexMergerV9 INDEX_MERGER_V9; @@ -162,7 +165,7 @@ private void setupQueries() TimeseriesQuery queryA = Druids.newTimeseriesQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .intervals(intervalSpec) .aggregators(queryAggs) .descending(descending) @@ -182,7 +185,7 @@ private void setupQueries() TimeseriesQuery timeFilterQuery = Druids.newTimeseriesQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .intervals(intervalSpec) .aggregators(queryAggs) .descending(descending) @@ -202,7 +205,7 @@ private void setupQueries() TimeseriesQuery timeFilterQuery = Druids.newTimeseriesQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .intervals(intervalSpec) .aggregators(queryAggs) .descending(descending) @@ -219,7 +222,7 @@ private void setupQueries() TimeseriesQuery timeFilterQuery = Druids.newTimeseriesQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .intervals(intervalSpec) .aggregators(queryAggs) .descending(descending) @@ -271,7 +274,7 @@ public void setup() @State(Scope.Benchmark) public static class IncrementalIndexState { - @Param({"onheap", "offheap"}) + @Param({"onheap"}) private String indexType; IncrementalIndex incIndex; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java index e69bd15a5f0c..655b0d2a423f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -26,7 +26,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.offheap.OffheapBufferGenerator; @@ -98,8 +98,8 @@ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 10) -@Measurement(iterations = 25) +@Warmup(iterations = 5) +@Measurement(iterations = 15) public class TopNBenchmark { @Param({"750000"}) @@ -111,6 +111,9 @@ public class TopNBenchmark @Param({"10"}) private int threshold; + @Param({"all", "hour"}) + private String queryGranularity; + private static final Logger log = new Logger(TopNBenchmark.class); private static final int RNG_SEED = 9999; private static final IndexMergerV9 INDEX_MERGER_V9; @@ -159,7 +162,7 @@ private void setupQueries() TopNQueryBuilder queryBuilderA = new TopNQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .dimension("dimSequential") .metric("sumFloatNormal") .intervals(intervalSpec) @@ -175,7 +178,7 @@ private void setupQueries() TopNQueryBuilder queryBuilderA = new TopNQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .dimension("dimUniform") .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .intervals(intervalSpec) @@ -191,7 +194,7 @@ private void setupQueries() TopNQueryBuilder queryBuilderA = new TopNQueryBuilder() .dataSource("blah") - .granularity(Granularities.ALL) + .granularity(Granularity.fromString(queryGranularity)) .dimension("dimUniform") .metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC)) .intervals(intervalSpec) @@ -249,7 +252,7 @@ public void setup() @State(Scope.Benchmark) public static class IncrementalIndexState { - @Param({"onheap", "offheap"}) + @Param({"onheap"}) private String indexType; IncrementalIndex incIndex; From 39eee3025b31315aad66f721515328df55722e88 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 30 Jul 2024 18:35:44 -0700 Subject: [PATCH 24/74] group by benchmark force instead of true --- .../java/org/apache/druid/benchmark/query/GroupByBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 9fb4343af3a1..e7220cc286d9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -141,7 +141,7 @@ public class GroupByBenchmark @Param({"all", "day"}) private String queryGranularity; - @Param({"false", "true"}) + @Param({"false", "force"}) private String vectorize; private static final Logger log = new Logger(GroupByBenchmark.class); From 246dd70290d871ff725cc2a3f768023c1dcfb601 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 31 Jul 2024 15:40:34 -0700 Subject: [PATCH 25/74] cleanup from self review --- .../druid/query/CursorGranularizer.java | 2 - .../query/FrameBasedInlineDataSource.java | 3 + .../epinephelinae/GroupByQueryEngine.java | 158 +++++++++--------- .../timeseries/TimeseriesQueryEngine.java | 45 ----- .../join/table/FrameBasedIndexedTable.java | 2 + .../frame/testutil/FrameSequenceBuilder.java | 9 +- .../druid/frame/testutil/FrameTestUtil.java | 31 +++- 7 files changed, 109 insertions(+), 141 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java index 44915c98cd04..694d4b092010 100644 --- a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java @@ -145,8 +145,6 @@ public boolean advanceCursorWithinBucket() return currentOffsetWithinBucket(); } - /** - */ public boolean advanceCursorWithinBucketUninterruptedly() { if (cursor.isDone()) { diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index c75e44025722..afdf67fdac35 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -87,6 +87,9 @@ public Sequence getRowsAsSequence() Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); + // currently FrameStorageAdapter cursor maker doesn't have any resources which need closed, but + // if this changes we need to register this cursor maker with a closer as the baggage of the + // sequence return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( CursorBuildSpec.FULL_SCAN ).makeCursor(); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index 0191a0f4fa8c..1d7a249c8afe 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -145,46 +145,17 @@ public static Sequence process( ); if (query.getGranularity().equals(Granularities.ALL)) { - return new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public GroupByEngineIterator make() - { - if (cardinalityForArrayAggregation >= 0) { - return new ArrayAggregateIterator( - query, - querySpecificConfig, - processingConfig, - cursor, - granularizer, - processingBuffer, - fudgeTimestamp, - dims, - hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()), - cardinalityForArrayAggregation - ); - } else { - return new HashAggregateIterator( - query, - querySpecificConfig, - processingConfig, - cursor, - granularizer, - processingBuffer, - fudgeTimestamp, - dims, - hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()) - ); - } - } - - @Override - public void cleanup(GroupByEngineIterator iterFromMake) - { - iterFromMake.close(); - } - } + return makeEngineIteratorSequence( + query, + processingBuffer, + fudgeTimestamp, + querySpecificConfig, + processingConfig, + cardinalityForArrayAggregation, + cursor, + granularizer, + dims, + columnSelectorFactory ); } @@ -193,50 +164,77 @@ public void cleanup(GroupByEngineIterator iterFromMake) if (!granularizer.advanceToBucket(bucketInterval)) { return Sequences.empty(); } - return new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public GroupByEngineIterator make() - { - if (cardinalityForArrayAggregation >= 0) { - return new ArrayAggregateIterator( - query, - querySpecificConfig, - processingConfig, - cursor, - granularizer, - processingBuffer, - fudgeTimestamp, - dims, - hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()), - cardinalityForArrayAggregation - ); - } else { - return new HashAggregateIterator( - query, - querySpecificConfig, - processingConfig, - cursor, - granularizer, - processingBuffer, - fudgeTimestamp, - dims, - hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()) - ); - } - } - - @Override - public void cleanup(GroupByEngineIterator iterFromMake) - { - iterFromMake.close(); - } - } + return makeEngineIteratorSequence( + query, + processingBuffer, + fudgeTimestamp, + querySpecificConfig, + processingConfig, + cardinalityForArrayAggregation, + cursor, + granularizer, + dims, + columnSelectorFactory ); }); } + private static Sequence makeEngineIteratorSequence( + GroupByQuery query, + ByteBuffer processingBuffer, + @Nullable DateTime fudgeTimestamp, + GroupByQueryConfig querySpecificConfig, + DruidProcessingConfig processingConfig, + int cardinalityForArrayAggregation, + Cursor cursor, + CursorGranularizer granularizer, + GroupByColumnSelectorPlus[] dims, + ColumnSelectorFactory columnSelectorFactory + ) + { + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public GroupByEngineIterator make() + { + if (cardinalityForArrayAggregation >= 0) { + return new ArrayAggregateIterator( + query, + querySpecificConfig, + processingConfig, + cursor, + granularizer, + processingBuffer, + fudgeTimestamp, + dims, + hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()), + cardinalityForArrayAggregation + ); + } else { + return new HashAggregateIterator( + query, + querySpecificConfig, + processingConfig, + cursor, + granularizer, + processingBuffer, + fudgeTimestamp, + dims, + hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()) + ); + } + } + + @Override + public void cleanup(GroupByEngineIterator iterFromMake) + { + iterFromMake.close(); + } + } + ); + } + /** * check if a column will operate correctly with {@link LimitedBufferHashGrouper} for query limit pushdown */ diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index 2eb2742012bb..f12da0a23900 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -306,50 +306,5 @@ private Sequence> processNonVectorized( } ) .filter(Objects::nonNull); - - /* - return Sequences.filter( - Sequences.map( - cursorMaker.makeCursors(), - cursor -> { - if (skipEmptyBuckets && cursor.isDone()) { - return null; - } - - Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()]; - String[] aggregatorNames = new String[aggregatorSpecs.size()]; - - for (int i = 0; i < aggregatorSpecs.size(); i++) { - aggregators[i] = aggregatorSpecs.get(i).factorize(cursor.getColumnSelectorFactory()); - aggregatorNames[i] = aggregatorSpecs.get(i).getName(); - } - - try { - while (!cursor.isDone()) { - for (Aggregator aggregator : aggregators) { - aggregator.aggregate(); - } - cursor.advance(); - } - - TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); - - for (int i = 0; i < aggregatorSpecs.size(); i++) { - bob.addMetric(aggregatorNames[i], aggregators[i].get()); - } - - return bob.build(); - } - finally { - // cleanup - for (Aggregator agg : aggregators) { - agg.close(); - } - } - } - ), - Objects::nonNull - ); - */ } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index 3eaf8a463f52..1824f4a810d5 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -122,6 +122,8 @@ public FrameBasedIndexedTable( RowSignature rowSignature = frameSignaturePair.getRowSignature(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); + // currently FrameStorageAdapter cursor maker doesn't have any resources which need closed, but if this + // changes we need to register this cursor maker with a closer as the baggage of the sequence return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursor(); }) .collect(Collectors.toList()) diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java index 7102883bb904..84f7c649e6a0 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java @@ -30,16 +30,15 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import java.io.Closeable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -140,8 +139,8 @@ public Sequence frames() throw DruidException.defensive("Unrecognized frame type"); } - final Pair cursorAndCloseable = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber); - final Cursor cursor = cursorAndCloseable.lhs; + final CursorMaker maker = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber); + final Cursor cursor = maker.makeCursor(); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -199,6 +198,6 @@ public void cleanup(Iterator iterFromMake) // Nothing to do. } } - ).withBaggage(cursorAndCloseable.rhs); + ).withBaggage(maker); } } 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 4c3ca581f52d..c16197b0e0f3 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 @@ -33,7 +33,6 @@ import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -57,7 +56,6 @@ import org.junit.Assert; import javax.annotation.Nullable; -import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -263,19 +261,23 @@ public static Sequence> readRowsFromAdapter( ) { final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature; - final Pair cursorAndCloseable = makeCursorForAdapter(adapter, populateRowNumber); - return readRowsFromCursor(cursorAndCloseable.lhs, signatureToUse).withBaggage(cursorAndCloseable.rhs); + final CursorMaker maker = makeCursorForAdapter(adapter, populateRowNumber); + final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return Sequences.withBaggage(Sequences.empty(), maker); + } + return readRowsFromCursor(cursor, signatureToUse).withBaggage(maker); } /** - * Creates a single-Cursor Sequence from a storage adapter. + * Creates a Cursor and from a storage adapter. * * If {@param populateRowNumber} is set, the row number will be populated into {@link #ROW_NUMBER_COLUMN}. * * @param adapter the adapter * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static Pair makeCursorForAdapter( + public static CursorMaker makeCursorForAdapter( final StorageAdapter adapter, final boolean populateRowNumber ) @@ -297,11 +299,22 @@ public static Pair makeCursorForAdapter( .build(); final CursorMaker maker = adapter.asCursorMaker(buildSpec); - final Cursor cursor = maker.makeCursor(); if (populateRowNumber) { - return new Pair<>(new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn), maker); + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return null; + } + return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn); + } + }; } else { - return new Pair<>(cursor, maker); + return maker; } } From ab10354ddc5b235a74dd1b1336f0f09c27ca3f6f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 31 Jul 2024 15:56:55 -0700 Subject: [PATCH 26/74] fixup --- .../query/FrameBasedInlineDataSource.java | 17 ++++++++++------- .../join/HashJoinSegmentStorageAdapter.java | 4 +--- .../join/table/FrameBasedIndexedTable.java | 8 ++++---- .../druid/frame/testutil/FrameTestUtil.java | 18 +++++++++--------- 4 files changed, 24 insertions(+), 23 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index afdf67fdac35..6c073c36234a 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -26,11 +26,13 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.RowSignature; @@ -80,6 +82,7 @@ public RowSignature getRowSignature() public Sequence getRowsAsSequence() { + final Closer closer = Closer.create(); final Sequence cursorSequence = Sequences.simple(frames) .map( @@ -87,14 +90,14 @@ public Sequence getRowsAsSequence() Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); - // currently FrameStorageAdapter cursor maker doesn't have any resources which need closed, but - // if this changes we need to register this cursor maker with a closer as the baggage of the - // sequence - return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( - CursorBuildSpec.FULL_SCAN - ).makeCursor(); + final CursorMaker maker = closer.register( + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( + CursorBuildSpec.FULL_SCAN + ) + ); + return maker.makeCursor(); } - ); + ).withBaggage(closer); return cursorSequence.flatMap( (cursor) -> { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 9f87694c0be1..143831a232db 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -229,9 +229,7 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) if (clauses.isEmpty()) { - // HashJoinEngine isn't vectorized yet. - // However, we can still vectorize if there are no clauses, since that means all we need to do is apply - // a base filter. That's easy enough! + // if there are no clauses, we can just use the base cursor directly if we apply the combined filter final CursorBuildSpec newSpec = cursorBuildSpecBuilder.setFilter(combinedFilter) .setVirtualColumns(spec.getVirtualColumns()) .build(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index 1824f4a810d5..e47d786629de 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.FrameSignaturePair; @@ -113,6 +114,7 @@ public FrameBasedIndexedTable( indexBuilders.add(m); } + final Closer closer = Closer.create(); final Sequence cursors = Sequences.simple( frameBasedInlineDataSource .getFrames() @@ -122,12 +124,10 @@ public FrameBasedIndexedTable( RowSignature rowSignature = frameSignaturePair.getRowSignature(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); - // currently FrameStorageAdapter cursor maker doesn't have any resources which need closed, but if this - // changes we need to register this cursor maker with a closer as the baggage of the sequence - return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursor(); + return closer.register(frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)).makeCursor(); }) .collect(Collectors.toList()) - ); + ).withBaggage(closer); final Sequence sequence = Sequences.map( cursors, 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 c16197b0e0f3..188186bcbc83 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 @@ -232,15 +232,15 @@ public static Sequence> readRowsFromFrameChannel( { return new FrameChannelSequence(channel) .flatMap( - frame -> - readRowsFromCursor( - // if FrameStorageAdapter.asCursorMaker ever needs closing.. this needs to change to add to a - // closer that is tied to baggage of this sequence... - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursor(), - frameReader.signature() - ) + frame -> { + final CursorMaker maker = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( + CursorBuildSpec.FULL_SCAN + ); + return readRowsFromCursor( + maker.makeCursor(), + frameReader.signature() + ).withBaggage(maker); + } ); } From 02a3f28fe9851b199974a43f7b98c042feff245f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 31 Jul 2024 21:25:03 -0700 Subject: [PATCH 27/74] fix leaky leak in test method --- .../druid/frame/processor/FrameProcessors.java | 3 ++- .../druid/frame/testutil/FrameTestUtil.java | 15 +++++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index f789f79e57d6..e081944e341b 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -121,7 +121,8 @@ public static FrameCursor makeCursor( .build(); // Safe to never close the Sequence that the FrameCursor comes from, because it does not need to be closed. // Refer to FrameStorageAdapter#makeCursors. - return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec).makeCursor(); + return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec) + .makeCursor(); } /** 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 188186bcbc83..8f68324bf50f 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 @@ -236,10 +236,11 @@ public static Sequence> readRowsFromFrameChannel( final CursorMaker maker = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( CursorBuildSpec.FULL_SCAN ); - return readRowsFromCursor( - maker.makeCursor(), - frameReader.signature() - ).withBaggage(maker); + final Cursor cursor = maker.makeCursor(); + if (cursor == null) { + return Sequences.withBaggage(Sequences.empty(), maker); + } + return readRowsFromCursor(cursor, frameReader.signature()).withBaggage(maker); } ); } @@ -312,6 +313,12 @@ public Cursor makeCursor() } return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn); } + + @Override + public void close() + { + maker.close(); + } }; } else { return maker; From 1760e6759578799de3061d5b2f55e25f2ee4053b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 1 Aug 2024 02:20:30 -0700 Subject: [PATCH 28/74] cleanup --- .../frame/processor/FrameProcessors.java | 7 +- .../apache/druid/segment/CursorFactory.java | 2 +- .../druid/segment/UnnestStorageAdapter.java | 6 +- .../join/HashJoinSegmentStorageAdapter.java | 9 ++- .../segment/FrameStorageAdapterTest.java | 6 +- .../druid/query/lookup/LookupSegmentTest.java | 4 +- .../segment/RowBasedStorageAdapterTest.java | 28 +++---- .../HashJoinSegmentStorageAdapterTest.java | 80 +++++++++---------- .../sql/calcite/CalciteJoinQueryTest.java | 2 +- 9 files changed, 74 insertions(+), 70 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index e081944e341b..3813a6711271 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -119,8 +119,11 @@ public static FrameCursor makeCursor( .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - // Safe to never close the Sequence that the FrameCursor comes from, because it does not need to be closed. - // Refer to FrameStorageAdapter#makeCursors. + // Despite appearances of columnar FrameCursorMakerFactory with its closers, it is currently safe to never close + // the CursorMaker that the FrameCursor comes from because it really does nothing. The row based + // FrameCursorMakerFactory has no closer stuff at all and is totally safe. If this ever changes, this method will + // probably need to wrap the cursor in something closeable, or be reworked to just return the CursorMaker so that + // callers can deal with closing the stuff. return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec) .makeCursor(); } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index 31cc3a49236f..de7a790af80e 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -71,7 +71,7 @@ default boolean canVectorize( * .isDescending(descending) * .setQueryMetrics(queryMetrics) * .build(); - * return asCursorMaker(buildSpec).makeCursors(); + * return asCursorMaker(buildSpec).makeCursor(); * * This method is no longer implemented by any built-in factories. */ diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index 4f8c9ac2ebfb..d0e4fd768879 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -269,9 +269,9 @@ public VirtualColumn getUnnestColumn() /** * Split queryFilter into pre- and post-correlate filters. * - * @param queryFilter query filter passed to makeCursors + * @param queryFilter query filter from {@link CursorBuildSpec} * @param unnestFilter filter on unnested column passed to PostUnnestCursor - * @param queryVirtualColumns query virtual columns passed to makeCursors + * @param queryVirtualColumns query virtual columns from {@link CursorBuildSpec} * @param inputColumn input column to unnest if it's a direct access; otherwise null * @param inputColumnCapabilites input column capabilities if known; otherwise null * @return pair of pre- and post-unnest filters @@ -467,7 +467,7 @@ public int getPreFilterCount() * while in case B, due to presence of the expression virtual column expressionVirtualColumn("j0.unnest", "array(\"dim1\",\"dim2\")", ColumnType.STRING_ARRAY) * the filters on d12 cannot be pushed to the pre filters * - * @param queryFilter query filter passed to makeCursors + * @param queryFilter query filter from {@link CursorBuildSpec} * @param inputColumn input column to unnest if it's a direct access; otherwise null * @param inputColumnCapabilites input column capabilities if known; otherwise null */ diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 143831a232db..b29a9b6ddd68 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -243,8 +243,8 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) @Override public Cursor makeCursor() { - // Filter pre-analysis key implied by the call to "makeCursors". We need to sanity-check that it matches - // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursors" call (it + // Filter pre-analysis key implied by the call to "makeCursor". We need to sanity-check that it matches + // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursor" call (it // requires access to the query context) so we'll need to skip sanity-checking it, by re-using the one present // in the cached key.) final JoinFilterPreAnalysisKey keyIn = @@ -258,7 +258,8 @@ public Cursor makeCursor() final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey(); final JoinFilterPreAnalysis preAnalysis; if (keyIn.equals(keyCached)) { - // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursors call (keyIn). + // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursor call + // (keyIn). preAnalysis = joinFilterPreAnalysis; } else { // Less common case: key differs. Re-analyze the filter. This case can happen when an unnest datasource is @@ -351,7 +352,7 @@ public boolean isBaseColumn(final String column) */ private Optional getClauseForColumn(final String column) { - // Check clauses in reverse, since "makeCursors" creates the cursor in such a way that the last clause + // Check clauses in reverse, since "makeCursor" creates the cursor in such a way that the last clause // gets first dibs to claim a column. return Lists.reverse(clauses) .stream() diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index db2663c2b619..a1f051421f19 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -71,7 +71,7 @@ public class FrameStorageAdapterTest { /** - * Basic tests: everything except makeCursors, makeVectorCursor. + * Basic tests: everything except makeCursor, makeVectorCursor. */ @RunWith(Parameterized.class) public static class BasicTests extends InitializedNullHandlingTest @@ -231,7 +231,7 @@ public void test_getMetadata() } /** - * CursorTests: matrix of tests of makeCursors, makeVectorCursor + * CursorTests: matrix of tests of makeCursor, makeVectorCursor */ @RunWith(Parameterized.class) public static class CursorTests extends InitializedNullHandlingTest @@ -363,7 +363,7 @@ public void tearDown() } @Test - public void test_makeCursors() + public void test_makeCursor() { assertCursorMatch(adapter -> adapter.asCursorMaker(buildSpec)); } diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java index 486ef7e20293..8e2fbad18ccb 100644 --- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java @@ -190,7 +190,7 @@ public void test_asStorageAdapter_getDimensionCardinalityV() } @Test - public void test_asStorageAdapter_makeCursors() + public void test_asStorageAdapter_makeCursor() { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970/PT1H")) @@ -222,7 +222,7 @@ public void test_asStorageAdapter_makeCursors() @Test public void test_asStorageAdapter_isRowBasedAdapter() { - // This allows us to assume that RowBasedStorageAdapterTest is further exercising makeCursors and verifying misc. + // This allows us to assume that RowBasedStorageAdapterTest is further exercising makeCursor and verifying misc. // methods like getMinTime, getMaxTime, getMetadata, etc, without checking them explicitly in _this_ test class. Assert.assertThat(LOOKUP_SEGMENT.asStorageAdapter(), CoreMatchers.instanceOf(RowBasedStorageAdapter.class)); } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index d12ca72d638f..3ddc7841711e 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -261,7 +261,7 @@ public void test_getDimensionCardinality_knownColumns() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursors is called). + // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursor is called). for (String column : ROW_SIGNATURE.getColumnNames()) { Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality(column)); } @@ -330,7 +330,7 @@ public void test_getCapabilities() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursors is called). + // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursor is called). for (String column : ROW_SIGNATURE.getColumnNames()) { Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality(column)); } @@ -450,7 +450,7 @@ public void test_getMetadata() } @Test - public void test_makeCursors_filterOnLong() + public void test_makeCursor_filterOnLong() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); @@ -474,7 +474,7 @@ public void test_makeCursors_filterOnLong() } @Test - public void test_makeCursors_filterOnNonexistentColumnEqualsNull() + public void test_makeCursor_filterOnNonexistentColumnEqualsNull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); @@ -498,7 +498,7 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNull() } @Test - public void test_makeCursors_filterOnVirtualColumn() + public void test_makeCursor_filterOnVirtualColumn() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); @@ -533,7 +533,7 @@ public void test_makeCursors_filterOnVirtualColumn() } @Test - public void test_makeCursors_descending() + public void test_makeCursor_descending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); @@ -554,7 +554,7 @@ public void test_makeCursors_descending() } @Test - public void test_makeCursors_intervalDoesNotMatch() + public void test_makeCursor_intervalDoesNotMatch() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); @@ -574,7 +574,7 @@ public void test_makeCursors_intervalDoesNotMatch() } @Test - public void test_makeCursors_intervalPartiallyMatches() + public void test_makeCursor_intervalPartiallyMatches() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); @@ -596,7 +596,7 @@ public void test_makeCursors_intervalPartiallyMatches() } @Test - public void test_makeCursors_hourGranularity() + public void test_makeCursor_hourGranularity() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); @@ -622,7 +622,7 @@ public void test_makeCursors_hourGranularity() } @Test - public void test_makeCursors_hourGranularityWithInterval() + public void test_makeCursor_hourGranularityWithInterval() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); @@ -648,7 +648,7 @@ public void test_makeCursors_hourGranularityWithInterval() } @Test - public void test_makeCursors_hourGranularityWithIntervalDescending() + public void test_makeCursor_hourGranularityWithIntervalDescending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); @@ -674,7 +674,7 @@ public void test_makeCursors_hourGranularityWithIntervalDescending() } @Test - public void test_makeCursors_allProcessors() + public void test_makeCursor_allProcessors() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); @@ -779,7 +779,7 @@ public void test_makeCursors_allProcessors() } @Test - public void test_makeCursors_filterOnNonexistentColumnEqualsNonnull() + public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); @@ -800,7 +800,7 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNonnull() } @Test - public void test_makeCursors_eternityIntervalWithMonthGranularity() + public void test_makeCursor_eternityIntervalWithMonthGranularity() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); Assert.assertThrows(IAE.class, () -> { diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index 24becbd4a298..ab5f3636f5e6 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -297,7 +297,7 @@ public void test_getMetadata_factToCountry() } @Test - public void test_makeCursors_factToCountryLeft() + public void test_makeCursor_factToCountryLeft() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); @@ -354,7 +354,7 @@ public void test_makeCursors_factToCountryLeft() } @Test - public void test_makeCursors_factToCountryLeftUsingLookup() + public void test_makeCursor_factToCountryLeftUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); @@ -410,7 +410,7 @@ public void test_makeCursors_factToCountryLeftUsingLookup() } @Test - public void test_makeCursors_factToCountryInner() + public void test_makeCursor_factToCountryInner() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)); JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis( @@ -460,7 +460,7 @@ public void test_makeCursors_factToCountryInner() } @Test - public void test_makeCursors_factToCountryInnerUsingLookup() + public void test_makeCursor_factToCountryInnerUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)); JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis( @@ -509,7 +509,7 @@ public void test_makeCursors_factToCountryInnerUsingLookup() } @Test - public void test_makeCursors_factToCountryInnerUsingCountryNumber() + public void test_makeCursor_factToCountryInnerUsingCountryNumber() { // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). @@ -569,7 +569,7 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() } @Test - public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() + public void test_makeCursor_factToCountryInnerUsingCountryNumberUsingLookup() { // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). @@ -627,7 +627,7 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() } @Test - public void test_makeCursors_factToCountryLeftWithFilterOnFacts() + public void test_makeCursor_factToCountryLeftWithFilterOnFacts() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); @@ -658,7 +658,7 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() } @Test - public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() + public void test_makeCursor_factToCountryLeftWithFilterOnFactsUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); @@ -688,7 +688,7 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() } @Test - public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() + public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); @@ -721,7 +721,7 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() } @Test - public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup() + public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); @@ -753,7 +753,7 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup } @Test - public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() + public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); @@ -786,7 +786,7 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() } @Test - public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup() + public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); @@ -818,7 +818,7 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( } @Test - public void test_makeCursors_factToCountryRightWithFilterOnJoinable() + public void test_makeCursor_factToCountryRightWithFilterOnJoinable() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter filter = new SelectorDimFilter( @@ -855,7 +855,7 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() } @Test - public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() + public void test_makeCursor_factToCountryRightWithFilterOnJoinableUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.RIGHT)); Filter filter = new SelectorDimFilter( @@ -891,7 +891,7 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() } @Test - public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() + public void test_makeCursor_factToCountryLeftWithFilterOnJoinable() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); @@ -930,7 +930,7 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() } @Test - public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() + public void test_makeCursor_factToCountryLeftWithFilterOnJoinableUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); Filter filter = new OrDimFilter( @@ -965,7 +965,7 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() } @Test - public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondition() + public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinCondition() { // Join condition => always true. // Filter => Fact to countries on countryIsoCode. @@ -1037,7 +1037,7 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit } @Test - public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinConditionUsingLookup() + public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditionUsingLookup() { // Join condition => always true. // Filter => Fact to countries on countryIsoCode. @@ -1107,7 +1107,7 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit } @Test - public void test_makeCursors_factToRegionToCountryLeft() + public void test_makeCursor_factToRegionToCountryLeft() { List joinableClauses = ImmutableList.of( factToRegion(JoinType.LEFT), @@ -1164,7 +1164,7 @@ public void test_makeCursors_factToRegionToCountryLeft() } @Test - public void test_makeCursors_factToRegionToCountryInnerIncludeNull() + public void test_makeCursor_factToRegionToCountryInnerIncludeNull() { List joinableClauses = ImmutableList.of( factToRegionIncludeNull(JoinType.INNER), @@ -1220,7 +1220,7 @@ public void test_makeCursors_factToRegionToCountryInnerIncludeNull() } @Test - public void test_makeCursors_factToCountryAlwaysTrue() + public void test_makeCursor_factToCountryAlwaysTrue() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1277,7 +1277,7 @@ public void test_makeCursors_factToCountryAlwaysTrue() } @Test - public void test_makeCursors_factToCountryAlwaysFalse() + public void test_makeCursor_factToCountryAlwaysFalse() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1319,7 +1319,7 @@ public void test_makeCursors_factToCountryAlwaysFalse() } @Test - public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() + public void test_makeCursor_factToCountryAlwaysTrueUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1377,7 +1377,7 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() } @Test - public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() + public void test_makeCursor_factToCountryAlwaysFalseUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1419,7 +1419,7 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() } @Test - public void test_makeCursors_factToCountryUsingVirtualColumn() + public void test_makeCursor_factToCountryUsingVirtualColumn() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1470,7 +1470,7 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() } @Test - public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() + public void test_makeCursor_factToCountryUsingVirtualColumnUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1521,7 +1521,7 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() } @Test - public void test_makeCursors_factToCountryUsingExpression() + public void test_makeCursor_factToCountryUsingExpression() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1566,7 +1566,7 @@ public void test_makeCursors_factToCountryUsingExpression() } @Test - public void test_makeCursors_factToCountryUsingExpressionUsingLookup() + public void test_makeCursor_factToCountryUsingExpressionUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1611,7 +1611,7 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() } @Test - public void test_makeCursors_factToRegionTheWrongWay() + public void test_makeCursor_factToRegionTheWrongWay() { // Joins using only regionIsoCode, which is wrong since they are not unique internationally. List joinableClauses = ImmutableList.of( @@ -1663,7 +1663,7 @@ public void test_makeCursors_factToRegionTheWrongWay() } @Test - public void test_makeCursors_errorOnNonEquiJoin() + public void test_makeCursor_errorOnNonEquiJoin() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: x == y"); @@ -1698,7 +1698,7 @@ public void test_makeCursors_errorOnNonEquiJoin() } @Test - public void test_makeCursors_errorOnNonEquiJoinUsingLookup() + public void test_makeCursor_errorOnNonEquiJoinUsingLookup() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot join lookup with non-equi condition: x == y"); @@ -1733,7 +1733,7 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() } @Test - public void test_makeCursors_errorOnNonKeyBasedJoin() + public void test_makeCursor_errorOnNonKeyBasedJoin() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-key-based condition: " @@ -1768,7 +1768,7 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() } @Test - public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() + public void test_makeCursor_errorOnNonKeyBasedJoinUsingLookup() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage( @@ -1803,7 +1803,7 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() } @Test - public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() + public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRows() { Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); @@ -1833,7 +1833,7 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() } @Test - public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLookup() + public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRowsUsingLookup() { Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); @@ -1861,7 +1861,7 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo } @Test - public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE() + public void test_makeCursor_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); Filter filter = new SelectorFilter("page", "this matches nothing"); @@ -1881,7 +1881,7 @@ public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowIS } @Test - public void test_makeCursors_factToCountryLeftWithBaseFilter() + public void test_makeCursor_factToCountryLeftWithBaseFilter() { final Filter baseFilter = Filters.or(Arrays.asList( new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(), @@ -1919,7 +1919,7 @@ public void test_makeCursors_factToCountryLeftWithBaseFilter() } @Test - public void test_makeCursors_factToCountryInnerWithBaseFilter() + public void test_makeCursor_factToCountryInnerWithBaseFilter() { final Filter baseFilter = Filters.or(Arrays.asList( new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(), @@ -1955,7 +1955,7 @@ public void test_makeCursors_factToCountryInnerWithBaseFilter() } @Test - public void test_makeCursors_factToCountryRightWithBaseFilter() + public void test_makeCursor_factToCountryRightWithBaseFilter() { final Filter baseFilter = Filters.or(Arrays.asList( new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(), @@ -2008,7 +2008,7 @@ public void test_makeCursors_factToCountryRightWithBaseFilter() } @Test - public void test_makeCursors_factToCountryFullWithBaseFilter() + public void test_makeCursor_factToCountryFullWithBaseFilter() { final Filter baseFilter = Filters.or(Arrays.asList( new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 264ebf6890ef..37a1a82f0df4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -5044,7 +5044,7 @@ public void testGroupByJoinAsNativeQueryWithUnoptimizedFilter(Map Date: Fri, 2 Aug 2024 18:42:26 -0700 Subject: [PATCH 29/74] partial review stuff --- .../ExpressionAggregationBenchmark.java | 6 +- .../benchmark/ExpressionFilterBenchmark.java | 10 +- .../ExpressionSelectorBenchmark.java | 62 ++++----- .../ExpressionVectorSelectorBenchmark.java | 10 +- .../benchmark/FilterPartitionBenchmark.java | 62 ++++----- .../IndexedTableJoinCursorBenchmark.java | 14 +- .../benchmark/JoinAndLookupBenchmark.java | 68 ++++++---- .../IncrementalIndexReadBenchmark.java | 14 +- ...bjectVectorColumnProcessorFactoryTest.java | 14 +- .../SegmentGeneratorFrameProcessor.java | 6 +- .../results/ExportResultsFrameProcessor.java | 6 +- .../scan/ScanQueryFrameProcessor.java | 16 ++- .../hadoop/DatasourceRecordReader.java | 8 +- .../indexing/input/DruidSegmentReader.java | 10 +- .../common/task/CompactionTaskRunTest.java | 14 +- .../indexing/common/task/IndexTaskTest.java | 10 +- .../frame/processor/FrameProcessors.java | 8 +- .../frame/segment/FrameStorageAdapter.java | 6 +- .../columnar/FrameCursorMakerFactory.java | 10 +- .../segment/row/FrameCursorMakerFactory.java | 4 +- .../druid/query/CursorGranularizer.java | 8 +- .../query/FrameBasedInlineDataSource.java | 17 +-- .../java/org/apache/druid/query/Query.java | 18 +-- .../apache/druid/query/filter/DimFilter.java | 2 +- .../druid/query/groupby/GroupByQuery.java | 17 +-- .../druid/query/groupby/GroupingEngine.java | 10 +- .../epinephelinae/GroupByQueryEngine.java | 6 +- .../vector/VectorGroupByEngine.java | 6 +- .../druid/query/metadata/SegmentAnalyzer.java | 6 +- .../LazilyDecoratedRowsAndColumns.java | 6 +- .../StorageAdapterRowsAndColumns.java | 6 +- .../apache/druid/query/scan/ScanQuery.java | 17 +-- .../druid/query/scan/ScanQueryEngine.java | 8 +- .../query/search/CursorOnlyStrategy.java | 6 +- .../TimeBoundaryQueryRunnerFactory.java | 8 +- .../query/timeseries/TimeseriesQuery.java | 17 +-- .../timeseries/TimeseriesQueryEngine.java | 22 +-- .../apache/druid/query/topn/TopNQuery.java | 17 +-- .../druid/query/topn/TopNQueryEngine.java | 12 +- .../vector/VectorCursorGranularizer.java | 7 +- .../druid/segment/ArrayListSegment.java | 2 +- .../java/org/apache/druid/segment/Cursor.java | 41 ++++-- .../apache/druid/segment/CursorBuildSpec.java | 6 +- .../apache/druid/segment/CursorFactory.java | 50 +++---- .../{CursorMaker.java => CursorHolder.java} | 15 ++- .../druid/segment/CursorMakerFactory.java | 2 +- .../druid/segment/FilteredStorageAdapter.java | 4 +- ...r.java => QueryableIndexCursorHolder.java} | 119 +++++++++-------- .../segment/QueryableIndexStorageAdapter.java | 6 +- .../apache/druid/segment/RowBasedSegment.java | 2 +- .../druid/segment/RowBasedStorageAdapter.java | 6 +- .../apache/druid/segment/SegmentWrangler.java | 2 +- .../apache/druid/segment/StorageAdapter.java | 12 +- .../druid/segment/UnnestStorageAdapter.java | 10 +- ...java => IncrementalIndexCursorHolder.java} | 8 +- .../IncrementalIndexStorageAdapter.java | 6 +- .../join/HashJoinSegmentStorageAdapter.java | 20 +-- .../druid/segment/join/PostJoinCursor.java | 2 +- .../table/BroadcastSegmentIndexedTable.java | 6 +- .../join/table/FrameBasedIndexedTable.java | 2 +- .../druid/segment/vector/VectorCursor.java | 16 +-- .../druid/frame/TestArrayStorageAdapter.java | 12 +- .../processor/OutputChannelFactoryTest.java | 10 +- .../segment/FrameStorageAdapterTest.java | 33 ++--- .../frame/testutil/FrameSequenceBuilder.java | 8 +- .../druid/frame/testutil/FrameTestUtil.java | 37 +++--- .../druid/frame/write/FrameWriterTest.java | 7 +- .../druid/query/CursorGranularizerTest.java | 10 +- .../groupby/NestedDataGroupByQueryTest.java | 10 +- .../VectorGroupByEngineIteratorTest.java | 8 +- .../druid/query/lookup/LookupSegmentTest.java | 6 +- .../semantic/RowsAndColumnsDecoratorTest.java | 6 +- .../NestedDataTimeseriesQueryTest.java | 10 +- .../topn/TopNMetricSpecOptimizationsTest.java | 8 +- .../segment/AutoTypeColumnIndexerTest.java | 28 ++-- .../NestedDataColumnIndexerV4Test.java | 24 ++-- ...va => QueryableIndexCursorHolderTest.java} | 22 +-- .../QueryableIndexStorageAdapterTest.java | 10 +- .../segment/RowBasedStorageAdapterTest.java | 48 +++---- .../TombstoneSegmentStorageAdapterTest.java | 6 +- .../segment/UnnestStorageAdapterTest.java | 32 ++--- .../druid/segment/filter/BaseFilterTest.java | 32 ++--- .../IncrementalIndexStorageAdapterTest.java | 125 +++++++++--------- ...BaseHashJoinSegmentStorageAdapterTest.java | 2 +- .../HashJoinSegmentStorageAdapterTest.java | 80 +++++------ .../segment/join/JoinFilterAnalyzerTest.java | 68 +++++----- .../druid/segment/join/JoinTestHelper.java | 12 +- .../segment/join/PostJoinCursorTest.java | 16 +-- .../NestedFieldColumnSelectorsTest.java | 8 +- .../virtual/ExpressionSelectorsTest.java | 22 +-- .../ExpressionVectorSelectorsCastTest.java | 2 +- .../ExpressionVectorSelectorsTest.java | 8 +- .../apache/druid/server/TestSegmentUtils.java | 8 +- .../org/apache/druid/cli/DumpSegment.java | 6 +- 94 files changed, 846 insertions(+), 789 deletions(-) rename processing/src/main/java/org/apache/druid/segment/{CursorMaker.java => CursorHolder.java} (84%) rename processing/src/main/java/org/apache/druid/segment/{QueryableIndexCursorMaker.java => QueryableIndexCursorHolder.java} (88%) rename processing/src/main/java/org/apache/druid/segment/incremental/{IncrementalIndexCursorMaker.java => IncrementalIndexCursorHolder.java} (97%) rename processing/src/test/java/org/apache/druid/segment/{QueryableIndexCursorMakerTest.java => QueryableIndexCursorHolderTest.java} (75%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java index 0c873bb4246c..f338a0605be3 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.ValueType; @@ -167,8 +167,8 @@ private double compute(final Function a .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory()); bufferAggregator.init(aggregationBuffer, 0); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java index bd0ed02b24d9..db6ebb5e7a15 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -33,7 +33,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.ValueType; @@ -150,8 +150,8 @@ public void expressionFilter(Blackhole blackhole) .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); @@ -170,8 +170,8 @@ public void nativeFilter(Blackhole blackhole) .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); while (!cursor.isDone()) { blackhole.consume(selector.getObject()); 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 ea11639addfd..627cb4ca9457 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -160,8 +160,8 @@ public void timeFloorUsingExpression(Blackhole blackhole) ) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -175,8 +175,8 @@ public void timeFloorUsingExtractionFn(Blackhole blackhole) .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -199,8 +199,8 @@ public void timeFloorUsingCursor(Blackhole blackhole) .setGranularity(Granularities.HOUR) .build(); final StorageAdapter adapter = new QueryableIndexStorageAdapter(index); - try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final CursorGranularizer granularizer = CursorGranularizer.create( adapter, cursor, @@ -251,8 +251,8 @@ public void timeFormatUsingExpression(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector( DefaultDimensionSpec.of("v") ); @@ -267,8 +267,8 @@ public void timeFormatUsingExtractionFn(Blackhole blackhole) .setInterval(index.getDataInterval()) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() .makeDimensionSelector( @@ -300,8 +300,8 @@ public void strlenUsingExpressionAsLong(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -325,8 +325,8 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() .makeDimensionSelector(new DefaultDimensionSpec("v", "v", ColumnType.STRING)); @@ -343,8 +343,8 @@ public void strlenUsingExtractionFn(Blackhole blackhole) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() .makeDimensionSelector(new ExtractionDimensionSpec("x", "v", StrlenExtractionFn.instance())); @@ -371,8 +371,8 @@ public void arithmeticOnLong(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -395,8 +395,8 @@ public void stringConcatAndCompareOnLong(Blackhole blackhole) ) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -419,8 +419,8 @@ public void caseSearched1(Blackhole blackhole) ) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -443,8 +443,8 @@ public void caseSearched2(Blackhole blackhole) ) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -480,8 +480,8 @@ public void caseSearched100(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -511,8 +511,8 @@ public void caseSearchedWithLookup(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } @@ -550,8 +550,8 @@ public void caseSearchedWithLookup2(Blackhole blackhole) ) .build(); - try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); } 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 9bd2e628511d..a6179a9b56b6 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -33,7 +33,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -156,9 +156,11 @@ public void scan(Blackhole blackhole) .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - final CursorMaker cursorMaker = closer.register(new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)); + final CursorHolder cursorHolder = closer.register( + new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec) + ); if (vectorize) { - VectorCursor cursor = cursorMaker.makeVectorCursor(); + VectorCursor cursor = cursorHolder.asVectorCursor(); if (outputType.isNumeric()) { VectorValueSelector selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); if (outputType.is(ExprType.DOUBLE)) { @@ -176,7 +178,7 @@ public void scan(Blackhole blackhole) } } } else { - final Cursor cursor = cursorMaker.makeCursor(); + final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); int rowCount = 0; if (outputType.isNumeric()) { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 3e4cb6bef234..c4ee7ee842e9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -47,7 +47,7 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -233,8 +233,8 @@ private IncrementalIndex makeIncIndex() public void stringRead(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, null)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -245,8 +245,8 @@ public void stringRead(Blackhole blackhole) public void longRead(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, null)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } } @@ -257,8 +257,8 @@ public void longRead(Blackhole blackhole) public void timeFilterNone(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (CursorMaker maker = makeCursorMaker(sa, timeFilterNone)) { - final Cursor cursor = maker.makeCursor(); + try (CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterNone)) { + final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } } @@ -269,8 +269,8 @@ public void timeFilterNone(Blackhole blackhole) public void timeFilterHalf(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, timeFilterHalf)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterHalf)) { + final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } } @@ -281,8 +281,8 @@ public void timeFilterHalf(Blackhole blackhole) public void timeFilterAll(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, timeFilterAll)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterAll)) { + final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } } @@ -295,8 +295,8 @@ public void readWithPreFilter(Blackhole blackhole) Filter filter = new SelectorFilter("dimSequential", "199"); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, filter)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -309,8 +309,8 @@ public void readWithPostFilter(Blackhole blackhole) Filter filter = new NoBitmapSelectorFilter("dimSequential", "199"); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, filter)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -323,8 +323,8 @@ public void readWithExFnPreFilter(Blackhole blackhole) Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, filter)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -337,8 +337,8 @@ public void readWithExFnPostFilter(Blackhole blackhole) Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, filter)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -356,8 +356,8 @@ public void readAndFilter(Blackhole blackhole) ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, andFilter)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, andFilter)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -372,8 +372,8 @@ public void readOrFilter(Blackhole blackhole) Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, orFilter)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, orFilter)) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -388,8 +388,8 @@ public void readOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosionExcept Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, Filters.toCnf(orFilter))) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(orFilter))) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -427,8 +427,8 @@ public void readComplexOrFilter(Blackhole blackhole) ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, dimFilter3.toFilter())) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, dimFilter3.toFilter())) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } @@ -466,15 +466,15 @@ public void readComplexOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosio ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorMaker maker = makeCursorMaker(sa, Filters.toCnf(dimFilter3.toFilter()))) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(dimFilter3.toFilter()))) { + final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } - private CursorMaker makeCursorMaker(StorageAdapter sa, Filter filter) + private CursorHolder makeCursorHolder(StorageAdapter sa, Filter filter) { - return sa.asCursorMaker( + return sa.makeCursorHolder( CursorBuildSpec.builder() .setFilter(filter) .setInterval(schemaInfo.getDataInterval()) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java index d1fbe9ee2e70..952e9b188fe9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java @@ -37,7 +37,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -217,8 +217,8 @@ public void tearDown() throws IOException @OutputTimeUnit(TimeUnit.MILLISECONDS) public void hashJoinCursorColumnValueSelectors(Blackhole blackhole) { - try (final CursorMaker maker = makeCursorMaker()) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder()) { + final Cursor cursor = cursorHolder.asCursor(); int rowCount = processRowsValueSelector(blackhole, cursor, projectionColumns); blackhole.consume(rowCount); } @@ -229,16 +229,16 @@ public void hashJoinCursorColumnValueSelectors(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void hashJoinCursorDimensionSelectors(Blackhole blackhole) { - try (final CursorMaker maker = makeCursorMaker()) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder()) { + final Cursor cursor = cursorHolder.asCursor(); int rowCount = processRowsDimensionSelectors(blackhole, cursor, projectionColumns); blackhole.consume(rowCount); } } - private CursorMaker makeCursorMaker() + private CursorHolder makeCursorHolder() { - return hashJoinSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN); + return hashJoinSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN); } 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 d93e7da592a2..e4596cb36c4e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -35,7 +35,7 @@ import org.apache.druid.query.lookup.MapLookupExtractorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -370,8 +370,8 @@ private static String getLastValue(final Cursor cursor, final String dimension) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void baseSegment(Blackhole blackhole) { - try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "countryIsoCode")); } } @@ -386,8 +386,8 @@ public void baseSegmentWithFilter(Blackhole blackhole) .setGranularity(Granularities.ALL) .setFilter(filter) .build(); - try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "countryIsoCode")); } } @@ -397,8 +397,9 @@ public void baseSegmentWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupStringKey(Blackhole blackhole) { - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); } } @@ -413,8 +414,9 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole) .setGranularity(Granularities.ALL) .setFilter(filter) .build(); - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); } } @@ -424,8 +426,9 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupLongKey(Blackhole blackhole) { - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); } } @@ -440,8 +443,9 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole) .setGranularity(Granularities.ALL) .setFilter(filter) .build(); - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); } } @@ -451,8 +455,9 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableLongKey(Blackhole blackhole) { - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); } } @@ -467,8 +472,9 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) .setGranularity(Granularities.ALL) .setFilter(filter) .build(); - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); } } @@ -478,8 +484,9 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableStringKey(Blackhole blackhole) { - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); } } @@ -494,8 +501,9 @@ public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) .setGranularity(Granularities.ALL) .setFilter(filter) .build(); - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); } } @@ -509,8 +517,9 @@ public void lookupVirtualColumnStringKey(Blackhole blackhole) .setVirtualColumns(lookupVirtualColumns) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); } } @@ -526,8 +535,9 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) .setFilter(filter) .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorMaker maker = hashJoinLookupStringKeySegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); } } @@ -541,8 +551,8 @@ public void lookupVirtualColumnLongKey(Blackhole blackhole) .setVirtualColumns(lookupVirtualColumns) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } } @@ -558,8 +568,8 @@ public void lookupVirtualColumnLongKeyWithFilter(Blackhole blackhole) .setVirtualColumns(lookupVirtualColumns) .setFilter(filter) .build(); - try (final CursorMaker maker = baseSegment.asStorageAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 96b6feaa4c6d..f71271dd1ebc 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -36,7 +36,7 @@ import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.generator.DataGenerator; @@ -149,8 +149,8 @@ private IncrementalIndex makeIncIndex() public void read(Blackhole blackhole) { IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - try (final CursorMaker maker = makeCursor(sa, null)) { - Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursor(sa, null)) { + Cursor cursor = cursorHolder.asCursor(); List selectors = new ArrayList<>(); selectors.add(makeDimensionSelector(cursor, "dimSequential")); @@ -185,8 +185,8 @@ public void readWithFilters(Blackhole blackhole) ); IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - try (final CursorMaker maker = makeCursor(sa, filter)) { - Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = makeCursor(sa, filter)) { + Cursor cursor = cursorHolder.asCursor(); List selectors = new ArrayList<>(); selectors.add(makeDimensionSelector(cursor, "dimSequential")); @@ -205,7 +205,7 @@ public void readWithFilters(Blackhole blackhole) } } - private CursorMaker makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter) + private CursorHolder makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter) { CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() .setGranularity(Granularities.ALL) @@ -213,7 +213,7 @@ private CursorMaker makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filt if (filter != null) { builder.setFilter(filter.toFilter()); } - return sa.asCursorMaker(builder.build()); + return sa.makeCursorHolder(builder.build()); } private static DimensionSelector makeDimensionSelector(Cursor cursor, String name) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index 0f8e4a0d76c8..b69b1fe88615 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -26,7 +26,7 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; @@ -57,8 +57,8 @@ public void setUp() @Test public void testRead() { - try (final CursorMaker maker = makeCursor()) { - final VectorCursor cursor = maker.makeVectorCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder()) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); final Supplier qualitySupplier = ColumnProcessors.makeVectorProcessor( "quality", ToObjectVectorColumnProcessorFactory.INSTANCE, @@ -171,7 +171,7 @@ public void testComplexSketch() Assert.assertThat(sketch, CoreMatchers.instanceOf(HyperLogLogCollector.class)); } - private CursorMaker makeCursor() + private CursorHolder makeCursorHolder() { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setQueryContext( @@ -180,13 +180,13 @@ private CursorMaker makeCursor() ) ) .build(); - return adapter.asCursorMaker(buildSpec); + return adapter.makeCursorHolder(buildSpec); } private List readColumn(final String column, final int limit) { - try (final CursorMaker maker = makeCursor()) { - final VectorCursor cursor = maker.makeVectorCursor(); + try (final CursorHolder cursorHolder = makeCursorHolder()) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); final Supplier supplier = ColumnProcessors.makeVectorProcessor( column, ToObjectVectorColumnProcessorFactory.INSTANCE, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index e865137d7263..a4b52abee58b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java @@ -46,7 +46,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; @@ -182,8 +182,8 @@ private void addFrame(final Frame frame) final MSQInputRow inputRow = new MSQInputRow(); final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); - try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java index 52fe8a9d6c3c..665cfab89eed 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java @@ -39,7 +39,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.ColumnMapping; @@ -152,8 +152,8 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx private void exportFrame(final Frame frame) { final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); - try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { exportWriter.writeRowEnd(); return; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 3ffd3f8dd01b..fa2a56ecb1f2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -63,7 +63,7 @@ import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; @@ -252,12 +252,13 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment ); } - final CursorMaker maker = closer.register(adapter.asCursorMaker(query.asCursorBuildSpec(null))); - final Cursor nextCursor = maker.makeCursor(); + final CursorHolder cursorHolder = closer.register( + adapter.makeCursorHolder(query.asCursorBuildSpec(null)) + ); + final Cursor nextCursor = cursorHolder.asCursor(); if (nextCursor == null) { // No cursors! - maker.close(); return ReturnOrAwait.returnObject(Unit.instance()); } else { final long rowsFlushed = setNextCursor(nextCursor, segmentHolder.get()); @@ -296,12 +297,13 @@ protected ReturnOrAwait runWithInputChannel( ); } - final CursorMaker maker = closer.register(adapter.asCursorMaker(query.asCursorBuildSpec(null))); - final Cursor nextCursor = maker.makeCursor(); + final CursorHolder cursorHolder = closer.register( + adapter.makeCursorHolder(query.asCursorBuildSpec(null)) + ); + final Cursor nextCursor = cursorHolder.asCursor(); if (nextCursor == null) { // no cursor - maker.close(); return ReturnOrAwait.returnObject(Unit.instance()); } final long rowsFlushed = setNextCursor(nextCursor, frameSegment); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index 5ddbef8842bf..a006d76727a2 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -45,7 +45,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -220,8 +220,8 @@ public Sequence apply(WindowedStorageAdapter adapter) .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec); - final Cursor cursor = maker.makeCursor(); + final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.empty(); } @@ -307,7 +307,7 @@ public void remove() }; } } - ).withBaggage(maker); + ).withBaggage(cursorHolder); } } ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index f04d2eb8f7d4..91493fc9520d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -50,7 +50,7 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -131,10 +131,10 @@ protected CloseableIterator> intermediateRowIterator() throw .setGranularity(Granularities.ALL) .build(); - final CursorMaker maker = storageAdapter.getAdapter().asCursorMaker(cursorBuildSpec); - final Cursor cursor = maker.makeCursor(); + final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { - return CloseableIterators.wrap(Collections.emptyIterator(), maker); + return CloseableIterators.wrap(Collections.emptyIterator(), cursorHolder); } // Retain order of columns from the original segments. Useful for preserving dimension order if we're in @@ -146,7 +146,7 @@ protected CloseableIterator> intermediateRowIterator() throw ) ); - final Sequence> sequence = cursorToSequence(cursor, columnsToRead).withBaggage(maker); + final Sequence> sequence = cursorToSequence(cursor, columnsToRead).withBaggage(cursorHolder); return makeCloseableIteratorFromSequenceAndSegmentFile(sequence, segmentFile); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index f009e0609cbe..8347048af4cc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -72,7 +72,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; @@ -1706,8 +1706,8 @@ public void testRunWithSpatialDimensions() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -1838,8 +1838,8 @@ public void testRunWithAutoCastDimensions() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -2067,8 +2067,8 @@ private List getCSVFormatRowsFromSegments(List segments) th .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); while (!cursor.isDone()) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 4619bdc5a1f7..740aec8f8b26 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -68,7 +68,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; @@ -544,8 +544,8 @@ public void testTransformSpec() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final List> transforms = new ArrayList<>(); final DimensionSelector selector1 = cursor.getColumnSelectorFactory() @@ -778,8 +778,8 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception .setGranularity(Granularities.ALL) .setInterval(segment.getInterval()) .build(); - try (final CursorMaker maker = adapter.getAdapter().asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final List hashes = new ArrayList<>(); final DimensionSelector selector = cursor.getColumnSelectorFactory() .makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index 3813a6711271..5faa0cef5b8a 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -120,12 +120,12 @@ public static FrameCursor makeCursor( .setVirtualColumns(virtualColumns) .build(); // Despite appearances of columnar FrameCursorMakerFactory with its closers, it is currently safe to never close - // the CursorMaker that the FrameCursor comes from because it really does nothing. The row based + // the CursorHolder that the FrameCursor comes from because it really does nothing. The row based // FrameCursorMakerFactory has no closer stuff at all and is totally safe. If this ever changes, this method will - // probably need to wrap the cursor in something closeable, or be reworked to just return the CursorMaker so that + // probably need to wrap the cursor in something closeable, or be reworked to just return the CursorHolder so that // callers can deal with closing the stuff. - return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(cursorBuildSpec) - .makeCursor(); + return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder(cursorBuildSpec) + .asCursor(); } /** diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java index fcc8cfdaa4f4..28162bf1028f 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java @@ -22,7 +22,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.Metadata; @@ -142,8 +142,8 @@ public Metadata getMetadata() } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return cursorFactory.asCursorMaker(spec); + return cursorFactory.makeCursorHolder(spec); } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java index 776dfe67d0fc..cbc53dd7e805 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; @@ -79,10 +79,10 @@ public FrameCursorMakerFactory( } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { final Closer closer = Closer.create(); - return new CursorMaker() + return new CursorHolder() { @Override public boolean canVectorize() @@ -93,7 +93,7 @@ public boolean canVectorize() } @Override - public Cursor makeCursor() + public Cursor asCursor() { final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); @@ -116,7 +116,7 @@ public Cursor makeCursor() @Nullable @Override - public VectorCursor makeVectorCursor() + public VectorCursor asVectorCursor() { if (!canVectorize()) { throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java index 56e84387b35c..65426ddaec0d 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java @@ -31,7 +31,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; @@ -64,7 +64,7 @@ public FrameCursorMakerFactory( } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { if (!Granularities.ALL.equals(spec.getGranularity())) { // Not currently needed for the intended use cases of frame-based cursors. diff --git a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java index 694d4b092010..d54be0455a5e 100644 --- a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java @@ -33,6 +33,11 @@ import javax.annotation.Nullable; +/** + * Class that helps non-vectorized query engines handle "granularity" parameters. Given a set of intervals + * + * @see org.apache.druid.query.vector.VectorCursorGranularizer for vectorized query engines. + */ public class CursorGranularizer { @Nullable @@ -73,13 +78,12 @@ public static CursorGranularizer create( return new CursorGranularizer(cursor, bucketIterable, timeSelector, descending); } - // And a cursor that has been made from it. private final Cursor cursor; // Iterable that iterates over time buckets. private final Iterable bucketIterable; - // Vector selector for the "__time" column. + // Selector for the "__time" column. @Nullable private final ColumnValueSelector timeSelector; private final boolean descending; diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index 6c073c36234a..38a8b1e91d5b 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -26,13 +26,12 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.RowSignature; @@ -82,22 +81,20 @@ public RowSignature getRowSignature() public Sequence getRowsAsSequence() { - final Closer closer = Closer.create(); final Sequence cursorSequence = Sequences.simple(frames) - .map( + .flatMap( frameSignaturePair -> { Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); - final CursorMaker maker = closer.register( - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker( + final CursorHolder holder = + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder( CursorBuildSpec.FULL_SCAN - ) - ); - return maker.makeCursor(); + ); + return Sequences.simple(Collections.singletonList(holder.asCursor())).withBaggage(holder); } - ).withBaggage(closer); + ); return cursorSequence.flatMap( (cursor) -> { diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index dd3b0463dadb..1a75d45de2ed 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.ExtensionPoint; @@ -45,6 +44,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Interval; @@ -278,15 +278,15 @@ default Set getRequiredColumns() default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final List intervals = getIntervals(); - if (intervals.size() > 1) { - throw DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - intervals - ); - } + final Interval interval = CollectionUtils.getOnlyElement( + getIntervals(), + (i) -> DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + i + ) + ); return CursorBuildSpec.builder() - .setInterval(Iterables.getOnlyElement(intervals)) + .setInterval(interval) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setVirtualColumns(getVirtualColumns()) diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index 60c175c9f916..955553d2dd3d 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java @@ -77,7 +77,7 @@ public interface DimFilter extends Cacheable * * The Filter returned by this method across multiple calls must be the same object: parts of the query stack * compare Filters, and returning the same object allows these checks to avoid deep comparisons. - * (see {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter#asCursorMaker(CursorBuildSpec)} + * (see {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter#makeCursorHolder(CursorBuildSpec)} * for an example} * * @param mayIncludeUnknown whether the optimized filter may need to operate in "includeUnknown" mode. 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 5868c4856b4c..a758b8560e55 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 @@ -73,6 +73,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -811,15 +812,15 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final List intervals = getIntervals(); - if (intervals.size() > 1) { - throw DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - intervals - ); - } + final Interval interval = CollectionUtils.getOnlyElement( + getIntervals(), + (i) -> DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + i + ) + ); return CursorBuildSpec.builder() - .setInterval(Iterables.getOnlyElement(intervals)) + .setInterval(interval) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setGroupingColumns(groupingColumns) diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 58599cda1c0b..64eec9f153c9 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -76,7 +76,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -502,10 +502,10 @@ public Sequence process( // group by specific vectorization check: final CursorBuildSpec buildSpec = query.asCursorBuildSpec(groupByQueryMetrics); - final CursorMaker cursorMaker = closer.register(storageAdapter.asCursorMaker(buildSpec)); + final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); - final boolean canVectorize = cursorMaker.canVectorize() && + final boolean canVectorize = cursorHolder.canVectorize() && VectorGroupByEngine.canVectorizeDimensions(inspector, query.getDimensions()); final boolean shouldVectorize = query.context().getVectorize().shouldVectorize(canVectorize); final Sequence result; @@ -513,7 +513,7 @@ public Sequence process( result = VectorGroupByEngine.process( query, storageAdapter, - cursorMaker, + cursorHolder, bufferHolder.get(), fudgeTimestamp, buildSpec.getInterval(), @@ -524,7 +524,7 @@ public Sequence process( result = GroupByQueryEngine.process( query, storageAdapter, - cursorMaker, + cursorHolder, buildSpec, bufferHolder.get(), fudgeTimestamp, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index 1d7a249c8afe..3cc503ae8f6f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -51,7 +51,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; @@ -97,7 +97,7 @@ private GroupByQueryEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, - final CursorMaker cursorMaker, + final CursorHolder cursorHolder, final CursorBuildSpec buildSpec, final ByteBuffer processingBuffer, @Nullable final DateTime fudgeTimestamp, @@ -105,7 +105,7 @@ public static Sequence process( final DruidProcessingConfig processingConfig ) { - final Cursor cursor = cursorMaker.makeCursor(); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.empty(); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index 51aee0c56ddd..4d0a30547c26 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -44,7 +44,7 @@ import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnProcessors; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -80,7 +80,7 @@ private VectorGroupByEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, - final CursorMaker cursorMaker, + final CursorHolder cursorHolder, final ByteBuffer processingBuffer, @Nullable final DateTime fudgeTimestamp, final Interval interval, @@ -94,7 +94,7 @@ public static Sequence process( @Override public CloseableIterator make() { - final VectorCursor cursor = cursorMaker.makeVectorCursor(); + final VectorCursor cursor = cursorHolder.asVectorCursor(); if (cursor == null) { // Return empty iterator. diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index 8a0fe376f8c3..be8d5627c9fe 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -30,7 +30,7 @@ import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; @@ -279,8 +279,8 @@ private ColumnAnalysis analyzeStringColumn( .setInterval(new Interval(start, end)) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = storageAdapter.asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor != null) { final DimensionSelector selector = diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 19a093bdf98d..f6e265664c0e 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -45,7 +45,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -228,8 +228,8 @@ private Pair materializeStorageAdapter(StorageAdapter as) if (virtualColumns != null) { builder.setVirtualColumns(virtualColumns); } - try (final CursorMaker maker = as.asCursorMaker(builder.build())) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = as.makeCursorHolder(builder.build())) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return null; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java index b996c5213ba0..2d49cc324400 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java @@ -30,7 +30,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; @@ -95,8 +95,8 @@ protected RowsAndColumns getRealRAC() @Nonnull private static RowsAndColumns materialize(StorageAdapter as) { - try (final CursorMaker maker = as.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = as.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return new EmptyRowsAndColumns(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 3d8621c4643e..2caf1771a3d6 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature.Builder; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -498,15 +499,15 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final List intervals = getIntervals(); - if (intervals.size() > 1) { - throw DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - intervals - ); - } + final Interval interval = CollectionUtils.getOnlyElement( + getIntervals(), + (i) -> DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + i + ) + ); return CursorBuildSpec.builder() - .setInterval(Iterables.getOnlyElement(intervals)) + .setInterval(interval) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setVirtualColumns(getVirtualColumns()) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 4dc5841c78a8..c648e99d05ff 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; @@ -114,14 +114,14 @@ public Sequence process( // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(queryMetrics)); + final CursorHolder cursorHolder = adapter.makeCursorHolder(query.asCursorBuildSpec(queryMetrics)); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override public Iterator make() { - final Cursor cursor = maker.makeCursor(); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Collections.emptyIterator(); } @@ -222,7 +222,7 @@ public void cleanup(Iterator iterFromMake) { } } - ).withBaggage(maker); + ).withBaggage(cursorHolder); } /** diff --git a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index 53ac9f679c77..af13c6cb8887 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -26,7 +26,7 @@ import org.apache.druid.query.search.SearchQueryRunner.SearchColumnSelectorStrategy; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; @@ -75,8 +75,8 @@ public Object2IntRBTreeMap execute(final int limit) { final StorageAdapter adapter = segment.asStorageAdapter(); final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); - try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); final Object2IntRBTreeMap retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator()); retVal.defaultReturnValue(0); diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 4d561a1c712f..5ac83e2aebd2 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -40,7 +40,7 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; @@ -118,8 +118,8 @@ private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legac final CursorBuildSpec buildSpec = CursorBuildSpec.builder(legacyQuery.asCursorBuildSpec(null)) .isDescending(descending) .build(); - try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return null; } @@ -195,7 +195,7 @@ public void cleanup(Iterator> toClean) /** * Whether a particular {@link TimeBoundaryQuery} can use {@link StorageAdapter#getMinTime()} and/or - * {@link StorageAdapter#getMaxTime()}. If false, must use {@link StorageAdapter#asCursorMaker(CursorBuildSpec)}. + * {@link StorageAdapter#getMaxTime()}. If false, must use {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)}. */ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, final StorageAdapter adapter) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index fd8531c20d5f..ddb74d6c9d71 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -47,6 +47,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -205,15 +206,15 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final List intervals = getIntervals(); - if (intervals.size() > 1) { - throw DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - intervals - ); - } + final Interval interval = CollectionUtils.getOnlyElement( + getIntervals(), + (i) -> DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + i + ) + ); return CursorBuildSpec.builder() - .setInterval(Iterables.getOnlyElement(intervals)) + .setInterval(interval) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setVirtualColumns(getVirtualColumns()) diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index f12da0a23900..6461bd223043 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -39,7 +39,7 @@ import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -95,27 +95,27 @@ public Sequence> process( final Granularity gran = query.getGranularity(); - final CursorMaker cursorMaker = adapter.asCursorMaker(query.asCursorBuildSpec(timeseriesQueryMetrics)); + final CursorHolder cursorHolder = adapter.makeCursorHolder(query.asCursorBuildSpec(timeseriesQueryMetrics)); final Sequence> result; - if (query.context().getVectorize().shouldVectorize(cursorMaker.canVectorize(), cursorMaker::close)) { - result = processVectorized(query, adapter, cursorMaker, interval, gran); + if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize(), cursorHolder::close)) { + result = processVectorized(query, adapter, cursorHolder, interval, gran); } else { - result = processNonVectorized(query, adapter, cursorMaker, interval, gran); + result = processNonVectorized(query, adapter, cursorHolder, interval, gran); } final int limit = query.getLimit(); if (limit < Integer.MAX_VALUE) { - return result.limit(limit).withBaggage(cursorMaker); + return result.limit(limit).withBaggage(cursorHolder); } else { - return result.withBaggage(cursorMaker); + return result.withBaggage(cursorHolder); } } private Sequence> processVectorized( final TimeseriesQuery query, final StorageAdapter adapter, - final CursorMaker cursorMaker, + final CursorHolder cursorHolder, final Interval queryInterval, final Granularity gran ) @@ -123,7 +123,7 @@ private Sequence> processVectorized( final boolean skipEmptyBuckets = query.isSkipEmptyBuckets(); final List aggregatorSpecs = query.getAggregatorSpecs(); - final VectorCursor cursor = cursorMaker.makeVectorCursor(); + final VectorCursor cursor = cursorHolder.asVectorCursor(); if (cursor == null) { return Sequences.empty(); @@ -231,14 +231,14 @@ private Sequence> processVectorized( private Sequence> processNonVectorized( final TimeseriesQuery query, final StorageAdapter adapter, - final CursorMaker cursorMaker, + final CursorHolder cursorHolder, final Interval queryInterval, final Granularity gran ) { final boolean skipEmptyBuckets = query.isSkipEmptyBuckets(); final List aggregatorSpecs = query.getAggregatorSpecs(); - final Cursor cursor = cursorMaker.makeCursor(); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.empty(); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 015833c2e1c1..f58a0b0b8d75 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -43,6 +43,7 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -187,15 +188,15 @@ public Set getRequiredColumns() @Override public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) { - final List intervals = getIntervals(); - if (intervals.size() > 1) { - throw DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - intervals - ); - } + final Interval interval = CollectionUtils.getOnlyElement( + getIntervals(), + (i) -> DruidException.defensive( + "This method can only be called after query is reduced to a single segment interval, got [%s]", + i + ) + ); return CursorBuildSpec.builder() - .setInterval(Iterables.getOnlyElement(intervals)) + .setInterval(interval) .setGranularity(getGranularity()) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setGroupingColumns(Collections.singletonList(dimensionSpec.getDimension())) diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index 4b2761c9d658..ba695eeae668 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -29,7 +29,7 @@ import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -73,10 +73,10 @@ public Sequence> query( final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); final CursorBuildSpec buildSpec = query.asCursorBuildSpec(queryMetrics); - final CursorMaker maker = adapter.asCursorMaker(buildSpec); - final Cursor cursor = maker.makeCursor(); + final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { - return Sequences.withBaggage(Sequences.empty(), maker); + return Sequences.withBaggage(Sequences.empty(), cursorHolder); } final CursorGranularizer granularizer = CursorGranularizer.create( adapter, @@ -86,7 +86,7 @@ public Sequence> query( buildSpec.isDescending() ); if (granularizer == null) { - return Sequences.withBaggage(Sequences.empty(), maker); + return Sequences.withBaggage(Sequences.empty(), cursorHolder); } if (queryMetrics != null) { @@ -100,7 +100,7 @@ public Sequence> query( return mapFn.apply(cursor, granularizer, queryMetrics); }), Predicates.notNull() - ).withBaggage(maker); + ).withBaggage(cursorHolder); } /** diff --git a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java index d1f3d7e3c98c..38288dc7dc3f 100644 --- a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java @@ -31,9 +31,9 @@ import javax.annotation.Nullable; /** - * Class that helps vectorized query engines handle "granularity" parameters. Nonvectorized engines have it handled - * for them by the StorageAdapter. Vectorized engines don't, because they can get efficiency gains by pushing - * granularity handling into the engine layer. + * Class that helps vectorized query engines handle "granularity" parameters. + * + * @see org.apache.druid.query.CursorGranularizer for non-vectorized query engines. */ public class VectorCursorGranularizer { @@ -71,7 +71,6 @@ public static VectorCursorGranularizer create( return new VectorCursorGranularizer(cursor, bucketIterable, timeSelector); } - // And a cursor that has been made from it. private final VectorCursor cursor; // Iterable that iterates over time buckets. diff --git a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java index fd413f913527..4fa8d3b45b3b 100644 --- a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java @@ -46,7 +46,7 @@ public class ArrayListSegment implements Segment * Create a list-based segment. *

* The provided List must be in time-order according to the provided {@link RowAdapter#timestampFunction()}. - * The cursor returned by {@link RowBasedStorageAdapter#asCursorMaker(CursorBuildSpec)} makes no attempt to verify + * The cursor returned by {@link RowBasedStorageAdapter#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify * this, and callers will expect it. *

* The provided "rowSignature" will be used for reporting available columns and their capabilities to users of diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 01c34fca1f85..170502808f22 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -20,12 +20,31 @@ package org.apache.druid.segment; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.segment.incremental.IncrementalIndexCursorHolder; import org.joda.time.DateTime; /** - * Cursor is an interface for iteration over a range of data points, used during query execution. {@link - * QueryableIndexCursorMaker.QueryableIndexCursor} is an implementation for historical segments, and {@link - * org.apache.druid.segment.incremental.IncrementalIndexCursorMaker.IncrementalIndexCursor} is an implementation for + * Cursor is an interface for iteration over a range of data points, used during query execution. Cursors are available + * from {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. + * + * A typical usage pattern might look something like this: + *

+ *   try (CursorHolder cursorHolder = adapter.makeCursorHolder(...)) {
+ *     Cursor cursor = cursorHolder.asCursor();
+ *     ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
+ *     ColumnValueSelector timeSelector = factory.makeColumnValueSelector("__time");
+ *     // ...
+ *     while (!cursor.isDone()) {}
+ *       long time = timeSelector.getLong();
+ *       // do stuff with column values
+ *       // ...
+ *       cursor.advance();
+ *     }
+ *   }
+ * 
+ * + * {@link QueryableIndexCursorHolder.QueryableIndexCursor} is an implementation for historical segments, and {@link + * IncrementalIndexCursorHolder.IncrementalIndexCursor} is an implementation for * {@link org.apache.druid.segment.incremental.IncrementalIndex}. * * Cursor is conceptually similar to {@link TimeAndDimsPointer}, but the latter is used for historical segment creation @@ -44,7 +63,7 @@ public interface Cursor /** * Advance to the next row in the cursor, checking if thread has been interrupted after advancing and possibly - * throwing {@link QueryInterruptedException} if so + * throwing {@link QueryInterruptedException} if so. */ void advance(); @@ -54,17 +73,23 @@ public interface Cursor void advanceUninterruptibly(); /** - * Check if there are any additional rows in the cursor + * Check if the current cursor position is valid. If true, any selectors created via + * {@link #getColumnSelectorFactory()} will no longer produce values. */ boolean isDone(); /** - * Check if there are any additional rows in the cursor, or if the thread has been interrupted + * Check if the current cursor position is valid, or if the thread has been interrupted. + * + * @see #isDone() */ boolean isDoneOrInterrupted(); /** - * Mark a position on the cursor which can recalled with {@link #resetToMark()} + * Mark a position on the cursor which can recalled with {@link #resetToMark()}. This method is used by + * {@link org.apache.druid.query.topn.TopNQueryEngine} when computing results in query granularity buckets. The + * {@link DateTime} parameter supplied to this method corresponds to the start of the bucket interval. Cursor + * implementations may use this if useful, or ignore it and mark the current offset if not. */ void mark(DateTime mark); @@ -74,7 +99,7 @@ public interface Cursor void resetToMark(); /** - * Reset to start of cursor + * Reset to start of cursor and discard mark. */ void reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index 0383f43df6aa..f26569fde392 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -141,10 +141,10 @@ public static class CursorBuildSpecBuilder private Granularity granularity = Granularities.NONE; @Nullable - private List groupingColumns = null; + private List groupingColumns; private VirtualColumns virtualColumns = VirtualColumns.EMPTY; @Nullable - private List aggregators = null; + private List aggregators; private boolean descending = false; private QueryContext queryContext = QueryContext.empty(); @@ -153,7 +153,7 @@ public static class CursorBuildSpecBuilder private CursorBuildSpecBuilder() { - // + // initialize with defaults } private CursorBuildSpecBuilder(CursorBuildSpec buildSpec) diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index de7a790af80e..7a259d83d488 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -33,8 +33,13 @@ * Interface extended by {@link StorageAdapter}, which gives them the power to create cursors. * * @see StorageAdapter + * + * @deprecated This interface is deprecated and no longer implemented by any built-in {@link StorageAdapter}. Callers + * should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. Implementors should implement + * {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. */ -public interface CursorFactory extends CursorMakerFactory +@Deprecated +public interface CursorFactory { /** * Returns true if the provided combination of parameters can be handled by "makeVectorCursor". @@ -42,9 +47,10 @@ public interface CursorFactory extends CursorMakerFactory * Query engines should use this before running in vectorized mode, and be prepared to fall back to non-vectorized * mode if this method returns false. * - * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#canVectorize()}. - * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. This method is no longer - * implemented by any built-in factories. + * @deprecated Callers should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} and call + * {@link CursorHolder#canVectorize()}. + * Implementors should implement {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is + * no longer implemented by any built-in factories. */ @Deprecated default boolean canVectorize( @@ -53,26 +59,17 @@ default boolean canVectorize( boolean descending ) { - throw DruidException.defensive("canVectorize is no longer supported, use asCursorMaker instead"); + throw DruidException.defensive( + "CursorFactory.canVectorize is no longer supported, use CursorMakerFactory.makeCursorHolder instead" + ); } /** * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity). * - * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call {@link CursorMaker#makeCursor()}. - * Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} instead. Recommend for implementors to fill - * this method in with: - *
-   *     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-   *                                                      .setFilter(filter)
-   *                                                      .setInterval(interval)
-   *                                                      .setGranularity(gran)
-   *                                                      .setVirtualColumns(virtualColumns)
-   *                                                      .isDescending(descending)
-   *                                                      .setQueryMetrics(queryMetrics)
-   *                                                      .build();
-   *     return asCursorMaker(buildSpec).makeCursor();
-   * 
+ * @deprecated Callers should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} and call + * {@link CursorHolder#asCursor()}. + * Implementors should implement {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. * This method is no longer implemented by any built-in factories. */ @Deprecated @@ -85,7 +82,9 @@ default Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { - throw DruidException.defensive("makeCursors is no longer supported, use asCursorMaker instead"); + throw DruidException.defensive( + "CursorFactory.makeCursors is no longer supported, use CursorMakerFactory.makeCursorHolder instead" + ); } /** @@ -96,9 +95,10 @@ default Sequence makeCursors( * Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval * of this segment). * - * @deprecated Callers should use {@link #asCursorMaker(CursorBuildSpec)} and call - * {@link CursorMaker#makeVectorCursor()}. Implementors should implement {@link #asCursorMaker(CursorBuildSpec)} - * instead. This method is no longer implemented by any built-in factories. + * @deprecated Callers should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} and call + * {@link CursorHolder#asVectorCursor()}. Implementors should implement + * {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is no longer implemented by any + * built-in factories. */ @Deprecated @Nullable @@ -111,6 +111,8 @@ default VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - throw DruidException.defensive("makeVectorCursor is no longer supported, use asCursorMaker instead"); + throw DruidException.defensive( + "CursorFactory.makeVectorCursor is no longer supported, use CursorMakerFactory.makeCursorHolder instead" + ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java similarity index 84% rename from processing/src/main/java/org/apache/druid/segment/CursorMaker.java rename to processing/src/main/java/org/apache/druid/segment/CursorHolder.java index 871ecc8f0ffa..aa6ea6b6ea2a 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java @@ -25,25 +25,25 @@ import javax.annotation.Nullable; import java.io.Closeable; -public interface CursorMaker extends Closeable +public interface CursorHolder extends Closeable { /** * Create a {@link Cursor} for use with non-vectorized query engines. */ @Nullable - Cursor makeCursor(); + Cursor asCursor(); /** * Create a {@link VectorCursor} for use with vectorized query engines. */ @Nullable - default VectorCursor makeVectorCursor() + default VectorCursor asVectorCursor() { throw new UOE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor' on %s.", this.getClass().getName()); } /** - * Returns true if this {@link CursorMaker} supports calling {@link #makeVectorCursor()}. + * Returns true if this {@link CursorHolder} supports calling {@link #asVectorCursor()}. */ default boolean canVectorize() { @@ -59,7 +59,7 @@ default void close() // nothing to close } - CursorMaker EMPTY = new CursorMaker() + CursorHolder EMPTY = new CursorHolder() { @Override public boolean canVectorize() @@ -67,15 +67,16 @@ public boolean canVectorize() return true; } + @Nullable @Override - public Cursor makeCursor() + public Cursor asCursor() { return null; } @Nullable @Override - public VectorCursor makeVectorCursor() + public VectorCursor asVectorCursor() { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java index 31f6be54f4c7..89eb0f88e87d 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java @@ -21,5 +21,5 @@ public interface CursorMakerFactory { - CursorMaker asCursorMaker(CursorBuildSpec spec); + CursorHolder makeCursorHolder(CursorBuildSpec spec); } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index 4974c0f8833a..b1b19b6a032b 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -42,7 +42,7 @@ public FilteredStorageAdapter(final StorageAdapter adapter, final DimFilter filt } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { final CursorBuildSpec.CursorBuildSpecBuilder buildSpecBuilder = CursorBuildSpec.builder(spec); final Filter newFilter; @@ -56,7 +56,7 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter())); } buildSpecBuilder.setFilter(newFilter); - return baseStorageAdapter.asCursorMaker(buildSpecBuilder.build()); + return baseStorageAdapter.makeCursorHolder(buildSpecBuilder.build()); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java similarity index 88% rename from processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java rename to processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 79389b8bf09f..8553caa0c3f5 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -61,9 +61,9 @@ import java.util.List; import java.util.concurrent.TimeUnit; -public class QueryableIndexCursorMaker implements CursorMaker +public class QueryableIndexCursorHolder implements CursorHolder { - private static final Logger log = new Logger(QueryableIndexCursorMaker.class); + private static final Logger log = new Logger(QueryableIndexCursorHolder.class); private final QueryableIndex index; private final Interval interval; @SuppressWarnings("unused") @@ -81,7 +81,7 @@ public class QueryableIndexCursorMaker implements CursorMaker private final int vectorSize; private final Supplier resourcesSupplier; - public QueryableIndexCursorMaker( + public QueryableIndexCursorHolder( QueryableIndex index, CursorBuildSpec cursorBuildSpec ) @@ -96,7 +96,7 @@ public QueryableIndexCursorMaker( this.queryContext = cursorBuildSpec.getQueryContext(); this.vectorSize = cursorBuildSpec.getQueryContext().getVectorSize(); this.metrics = cursorBuildSpec.getQueryMetrics(); - this.resourcesSupplier = Suppliers.memoize(CursorResources::new); + this.resourcesSupplier = Suppliers.memoize(() -> new CursorResources(index, virtualColumns, filter, metrics)); } @Override @@ -129,7 +129,7 @@ public boolean canVectorize() } @Override - public Cursor makeCursor() + public Cursor asCursor() { if (metrics != null) { metrics.vectorized(false); @@ -209,7 +209,7 @@ public Cursor makeCursor() @Nullable @Override - public VectorCursor makeVectorCursor() + public VectorCursor asVectorCursor() { final CursorResources resources = resourcesSupplier.get(); final FilterBundle filterBundle = resources.filterBundle; @@ -285,51 +285,8 @@ public void close() CloseableUtils.closeAndWrapExceptions(resourcesSupplier.get()); } - @Nullable - private FilterBundle makeFilterBundle( - ColumnSelectorColumnIndexSelector bitmapIndexSelector, - int numRows - ) - { - final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory(); - final BitmapResultFactory bitmapResultFactory; - if (metrics != null) { - bitmapResultFactory = metrics.makeBitmapResultFactory(bitmapFactory); - metrics.reportSegmentRows(numRows); - } else { - bitmapResultFactory = new DefaultBitmapResultFactory(bitmapFactory); - } - if (filter == null) { - return null; - } - final long bitmapConstructionStartNs = System.nanoTime(); - final FilterBundle filterBundle = filter.makeFilterBundle( - bitmapIndexSelector, - bitmapResultFactory, - numRows, - numRows, - false - ); - if (metrics != null) { - final long buildTime = System.nanoTime() - bitmapConstructionStartNs; - metrics.reportBitmapConstructionTime(buildTime); - final FilterBundle.BundleInfo info = filterBundle.getInfo(); - metrics.filterBundle(info); - log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); - if (filterBundle.getIndex() != null) { - metrics.reportPreFilteredRows(filterBundle.getIndex().getBitmap().size()); - } else { - metrics.reportPreFilteredRows(0); - } - } else if (log.isDebugEnabled()) { - final FilterBundle.BundleInfo info = filterBundle.getInfo(); - final long buildTime = System.nanoTime() - bitmapConstructionStartNs; - log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); - } - return filterBundle; - } - VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( + private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( ColumnCache columnCache, VectorOffset baseOffset ) @@ -477,7 +434,7 @@ public void advance() cursorOffset.increment(); // Must call BaseQuery.checkInterrupted() after cursorOffset.increment(), not before, because // FilteredOffset.increment() is a potentially long, not an "instant" operation (unlike to all other subclasses - // of Offset) and it returns early on interruption, leaving itself in an illegal We should not let + // of Offset) and it returns early on interruption, leaving itself in an illegal state. We should not let // aggregators, etc. access this illegal state and throw a QueryInterruptedException by calling // BaseQuery.checkInterrupted(). BaseQuery.checkInterrupted(); @@ -520,7 +477,6 @@ public void reset() } } - public abstract static class TimestampCheckingOffset extends Offset { final Offset baseOffset; @@ -675,20 +631,31 @@ public Offset clone() } } - private final class CursorResources implements Closeable + private static final class CursorResources implements Closeable { private final Closer closer; private final long minDataTimestamp; private final long maxDataTimestamp; private final int numRows; @Nullable + private final Filter filter; + @Nullable private final FilterBundle filterBundle; private final NumericColumn timestamps; private final ColumnCache columnCache; + @Nullable + private final QueryMetrics> metrics; - private CursorResources() + private CursorResources( + QueryableIndex index, + VirtualColumns virtualColumns, + @Nullable Filter filter, + @Nullable QueryMetrics> metrics + ) { this.closer = Closer.create(); + this.filter = filter; + this.metrics = metrics; this.columnCache = new ColumnCache(index, closer); final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector( index.getBitmapFactoryForDimensions(), @@ -712,5 +679,49 @@ public void close() throws IOException { closer.close(); } + + @Nullable + private FilterBundle makeFilterBundle( + ColumnSelectorColumnIndexSelector bitmapIndexSelector, + int numRows + ) + { + final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory(); + final BitmapResultFactory bitmapResultFactory; + if (metrics != null) { + bitmapResultFactory =metrics.makeBitmapResultFactory(bitmapFactory); + metrics.reportSegmentRows(numRows); + } else { + bitmapResultFactory = new DefaultBitmapResultFactory(bitmapFactory); + } + if (filter == null) { + return null; + } + final long bitmapConstructionStartNs = System.nanoTime(); + final FilterBundle filterBundle = filter.makeFilterBundle( + bitmapIndexSelector, + bitmapResultFactory, + numRows, + numRows, + false + ); + if (metrics != null) { + final long buildTime = System.nanoTime() - bitmapConstructionStartNs; + metrics.reportBitmapConstructionTime(buildTime); + final FilterBundle.BundleInfo info = filterBundle.getInfo(); + metrics.filterBundle(info); + log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + if (filterBundle.getIndex() != null) { + metrics.reportPreFilteredRows(filterBundle.getIndex().getBitmap().size()); + } else { + metrics.reportPreFilteredRows(0); + } + } else if (log.isDebugEnabled()) { + final FilterBundle.BundleInfo info = filterBundle.getInfo(); + final long buildTime = System.nanoTime() - bitmapConstructionStartNs; + log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + } + return filterBundle; + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 7d13e57e6531..de16b37d461d 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -171,15 +171,15 @@ public DateTime getMaxIngestedEventTime() } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { final Interval actualInterval = computeCursorInterval(spec.getGranularity(), spec.getInterval()); if (actualInterval == null) { - return CursorMaker.EMPTY; + return CursorHolder.EMPTY; } - return new QueryableIndexCursorMaker( + return new QueryableIndexCursorHolder( index, CursorBuildSpec.builder(spec).setInterval(actualInterval).build() ); diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java index 0d273adc7573..26ff6594bbbc 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java @@ -40,7 +40,7 @@ public class RowBasedSegment implements Segment * Create a row-based segment. * * The provided "rowIterable" must be in time-order according to the provided {@link RowAdapter#timestampFunction()}. - * The cursor returned by {@link RowBasedStorageAdapter#asCursorMaker(CursorBuildSpec)} makes no attempt to verify + * The cursor returned by {@link RowBasedStorageAdapter#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify * this, and callers will expect it. * * The provided "rowSignature" will be used for reporting available columns and their capabilities to users of diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index 412877dcc1f3..4e880c4e4ce6 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -167,14 +167,14 @@ public Metadata getMetadata() } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return new CursorMaker() + return new CursorHolder() { final Closer closer = Closer.create(); @Override - public Cursor makeCursor() + public Cursor asCursor() { final Granularity gran = spec.getGranularity(); final Interval actualInterval = spec.getInterval() diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java index 7e756ffb616a..4e39ca00f18f 100644 --- a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java +++ b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java @@ -40,7 +40,7 @@ public interface SegmentWrangler * @return Segments that, collectively, contain data for dataSource. May be empty if dataSource does not exist or * has no data in the provided intervals. May contain data outside the provided intervals, so callers should * filter it down further, e.g. through the "interval" parameter of {@link CursorBuildSpec} for - * {@link StorageAdapter#asCursorMaker(CursorBuildSpec)} + * {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)} */ Iterable getSegmentsForIntervals(DataSource dataSource, Iterable intervals); } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index 0607f9206bca..fdb5cdc2fd8e 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -35,19 +35,19 @@ /** */ @PublicApi -public interface StorageAdapter extends CursorFactory, ColumnInspector +public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorMakerFactory { /** - * Build a {@link CursorMaker} which can provide {@link Cursor} and {@link VectorCursor} (if capable) which allows + * Build a {@link CursorHolder} which can provide {@link Cursor} and {@link VectorCursor} (if capable) which allows * scanning segments and creating {@link ColumnSelectorFactory} and * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} respectively to read row values at the cursor * position. */ @Override - default CursorMaker asCursorMaker(CursorBuildSpec spec) + default CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return new CursorMaker() + return new CursorHolder() { @Override public boolean canVectorize() @@ -56,7 +56,7 @@ public boolean canVectorize() } @Override - public Cursor makeCursor() + public Cursor asCursor() { return Iterables.getOnlyElement( StorageAdapter.this.makeCursors( @@ -71,7 +71,7 @@ public Cursor makeCursor() } @Override - public VectorCursor makeVectorCursor() + public VectorCursor asVectorCursor() { return StorageAdapter.this.makeVectorCursor( spec.getFilter(), diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index d0e4fd768879..e15b4d34b3c6 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -89,7 +89,7 @@ public UnnestStorageAdapter( } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { final String input = getUnnestInputIfDirectAccess(unnestColumn); final Pair filterPair = computeBaseAndPostUnnestFilters( @@ -105,14 +105,14 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) .setVirtualColumns(VirtualColumns.create(Collections.singletonList(unnestColumn))) .build(); - return new CursorMaker() + return new CursorHolder() { final Closer closer = Closer.create(); @Override - public Cursor makeCursor() + public Cursor asCursor() { - final CursorMaker maker = closer.register(baseAdapter.asCursorMaker(unnestBuildSpec)); - final Cursor cursor = maker.makeCursor(); + final CursorHolder cursorHolder = closer.register(baseAdapter.makeCursorHolder(unnestBuildSpec)); + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java similarity index 97% rename from processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java rename to processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 69b2f749f51b..efcfed15f268 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -26,7 +26,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.filter.ValueMatchers; import org.joda.time.DateTime; @@ -35,13 +35,13 @@ import javax.annotation.Nullable; import java.util.Iterator; -public class IncrementalIndexCursorMaker implements CursorMaker +public class IncrementalIndexCursorHolder implements CursorHolder { private final IncrementalIndexStorageAdapter storageAdapter; private final IncrementalIndex index; private final CursorBuildSpec builder; - public IncrementalIndexCursorMaker( + public IncrementalIndexCursorHolder( IncrementalIndexStorageAdapter storageAdapter, IncrementalIndex index, CursorBuildSpec builder @@ -53,7 +53,7 @@ public IncrementalIndexCursorMaker( } @Override - public Cursor makeCursor() + public Cursor asCursor() { if (index.isEmpty()) { return null; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index b4a9270ee582..ad55e33adaef 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.incremental; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.Metadata; @@ -243,9 +243,9 @@ public DateTime getMaxIngestedEventTime() } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return new IncrementalIndexCursorMaker(this, index, spec); + return new IncrementalIndexCursorHolder(this, index, spec); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index b29a9b6ddd68..3eae756f2bc2 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -27,7 +27,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -216,7 +216,7 @@ public boolean hasBuiltInFilters() } @Override - public CursorMaker asCursorMaker(CursorBuildSpec spec) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { final CursorBuildSpec.CursorBuildSpecBuilder cursorBuildSpecBuilder = CursorBuildSpec.builder() @@ -233,15 +233,15 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) final CursorBuildSpec newSpec = cursorBuildSpecBuilder.setFilter(combinedFilter) .setVirtualColumns(spec.getVirtualColumns()) .build(); - return baseAdapter.asCursorMaker(newSpec); + return baseAdapter.makeCursorHolder(newSpec); } - return new CursorMaker() + return new CursorHolder() { final Closer joinablesCloser = Closer.create(); @Override - public Cursor makeCursor() + public Cursor asCursor() { // Filter pre-analysis key implied by the call to "makeCursor". We need to sanity-check that it matches // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursor" call (it @@ -287,8 +287,8 @@ public Cursor makeCursor() ); cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns); - final Cursor baseCursor = joinablesCloser.register(baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build())) - .makeCursor(); + final Cursor baseCursor = joinablesCloser.register(baseAdapter.makeCursorHolder(cursorBuildSpecBuilder.build())) + .asCursor(); if (baseCursor == null) { return null; @@ -316,14 +316,14 @@ public void close() @Override public boolean canVectorize() { - return CursorMaker.super.canVectorize(); + return CursorHolder.super.canVectorize(); } @Nullable @Override - public VectorCursor makeVectorCursor() + public VectorCursor asVectorCursor() { - return CursorMaker.super.makeVectorCursor(); + return CursorHolder.super.asVectorCursor(); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 1c023ba26eb0..4195dc4975bd 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -31,7 +31,7 @@ import javax.annotation.Nullable; /** - * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#asCursorMaker(CursorBuildSpec)} to add post-join + * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#makeCursorHolder(CursorBuildSpec)} to add post-join * virtual columns and filters. */ public class PostJoinCursor implements Cursor diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index d170b0578544..59ffbacca99b 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -30,7 +30,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; @@ -121,8 +121,8 @@ public BroadcastSegmentIndexedTable( ) .setGranularity(Granularities.ALL) .build(); - try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { - final Cursor cursor = maker.makeCursor(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { this.keyColumnsIndexes = Collections.emptyList(); return; diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index e47d786629de..970ba5fe7279 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -124,7 +124,7 @@ public FrameBasedIndexedTable( RowSignature rowSignature = frameSignaturePair.getRowSignature(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); - return closer.register(frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)).makeCursor(); + return closer.register(frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asCursor(); }) .collect(Collectors.toList()) ).withBaggage(closer); diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index c3806fbccf34..9ecfa77e54a5 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java @@ -20,22 +20,20 @@ package org.apache.druid.segment.vector; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.QueryableIndexCursorMaker; /** - * Vectorized cursor used during query execution. VectorCursors are returned by - * {@link org.apache.druid.segment.StorageAdapter#asCursorMaker(CursorBuildSpec)} and are created by - * {@link QueryableIndexCursorMaker#makeVectorCursor()}. + * Vectorized cursor used during query execution. VectorCursors are available from + * {@link org.apache.druid.segment.CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} via + * {@link org.apache.druid.segment.CursorHolder#asVectorCursor()}. * - * Unlike the non-vectorized version, VectorCursor does not have a getTime() method. This is because we are trying to - * avoid creating needlessly-small vectors when the time granularity is very fine. See - * {@link org.apache.druid.query.vector.VectorCursorGranularizer} for a helper that makes it easier for query engines to - * do their own time granularization. + * See {@link org.apache.druid.query.vector.VectorCursorGranularizer} for a helper that makes it easier for query + * engines to do time granularization. * * An example of how to use the methods in this class: * *
- *   try (VectorCursor cursor = adapter.makeVectorCursor(...)) {
+ *   try (CursorHolder cursorHolder = adapter.makeCursorHolder(...)) {
+ *     VectorCursor cursor = cursorHolder.asVectorCursor();
  *     // ProcessorClass is some vectorized processor class.
  *     ProcessorClass o = makeProcessor(cursor.getColumnSelectorFactory());
  *     for (; !cursor.isDone(); cursor.advance()) {
diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
index 9154c41b3345..ca2f33cca591 100644
--- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
+++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
@@ -27,7 +27,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.ObjectColumnSelector;
 import org.apache.druid.segment.QueryableIndex;
@@ -53,16 +53,16 @@ public TestArrayStorageAdapter(QueryableIndex index)
   }
 
   @Override
-  public CursorMaker asCursorMaker(CursorBuildSpec spec)
+  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
-    final CursorMaker delegate = super.asCursorMaker(spec);
-    return new CursorMaker()
+    final CursorHolder delegate = super.makeCursorHolder(spec);
+    return new CursorHolder()
     {
       @Nullable
       @Override
-      public Cursor makeCursor()
+      public Cursor asCursor()
       {
-        return new DecoratedCursor(delegate.makeCursor());
+        return new DecoratedCursor(delegate.asCursor());
       }
 
       @Override
diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java
index 582efbd9a27b..846d07ebc11b 100644
--- a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java
@@ -32,7 +32,7 @@
 import org.apache.druid.frame.testutil.FrameTestUtil;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.TestIndex;
 import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
@@ -129,10 +129,10 @@ protected void verifySingleFrameReadableChannel(
     readableFrameChannel.close();
 
     // build list of rows from written and read data to verify
-    try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN);) {
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN);) {
       List> writtenData =
           FrameTestUtil.readRowsFromCursor(
-              maker.makeCursor(),
+              cursorHolder.asCursor(),
               adapter.getRowSignature()
           ).toList();
 
@@ -141,9 +141,9 @@ protected void verifySingleFrameReadableChannel(
           FrameReader.create(adapter.getRowSignature()),
           Intervals.ETERNITY
       );
-      try (final CursorMaker frameMaker = frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) {
+      try (final CursorHolder frameMaker = frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
         List> readData = FrameTestUtil.readRowsFromCursor(
-            frameMaker.makeCursor(),
+            frameMaker.asCursor(),
             frameStorageAdapter.getRowSignature()
         ).toList();
 
diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
index a1f051421f19..d8a3ddbc0e48 100644
--- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
@@ -38,7 +38,7 @@
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.StorageAdapter;
@@ -365,42 +365,43 @@ public void tearDown()
     @Test
     public void test_makeCursor()
     {
-      assertCursorMatch(adapter -> adapter.asCursorMaker(buildSpec));
+      assertCursorMatch(adapter -> adapter.makeCursorHolder(buildSpec));
     }
 
     @Test
     public void test_makeVectorCursor()
     {
-      assertVectorCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec));
+      assertVectorCursorsMatch(adapter -> adapter.makeCursorHolder(buildSpec));
     }
 
-    private void assertCursorMatch(final Function call)
+    private void assertCursorMatch(final Function call)
     {
       final RowSignature signature = frameAdapter.getRowSignature();
-      try (final CursorMaker queryableMaker = call.apply(queryableAdapter);
-           final CursorMaker frameMaker = call.apply(frameAdapter)) {
+      try (final CursorHolder queryableMaker = call.apply(queryableAdapter);
+           final CursorHolder frameMaker = call.apply(frameAdapter)) {
         final Sequence> queryableRows =
-            FrameTestUtil.readRowsFromCursor(queryableMaker.makeCursor(), signature);
+            FrameTestUtil.readRowsFromCursor(queryableMaker.asCursor(), signature);
         final Sequence> frameRows =
-            FrameTestUtil.readRowsFromCursor(frameMaker.makeCursor(), signature);
+            FrameTestUtil.readRowsFromCursor(frameMaker.asCursor(), signature);
         FrameTestUtil.assertRowsEqual(queryableRows, frameRows);
       }
     }
 
-    private void assertVectorCursorsMatch(final Function call)
+    private void assertVectorCursorsMatch(final Function call)
     {
-      final CursorMaker maker = call.apply(queryableAdapter);
-      final CursorMaker frameMaker = call.apply(frameAdapter);
-      if (frameMaker.canVectorize()) {
+      final CursorHolder cursorHolder = call.apply(queryableAdapter);
+      final CursorHolder frameCursorHolder = call.apply(frameAdapter);
+      if (frameCursorHolder.canVectorize()) {
         final RowSignature signature = frameAdapter.getRowSignature();
         final Sequence> queryableRows =
-            FrameTestUtil.readRowsFromVectorCursor(maker.makeVectorCursor(), signature).withBaggage(maker);
+            FrameTestUtil.readRowsFromVectorCursor(cursorHolder.asVectorCursor(), signature).withBaggage(cursorHolder);
         final Sequence> frameRows =
-            FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameMaker.makeVectorCursor()), signature).withBaggage(frameMaker);
+            FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameCursorHolder.asVectorCursor()), signature)
+                         .withBaggage(frameCursorHolder);
         FrameTestUtil.assertRowsEqual(queryableRows, frameRows);
       } else {
-        maker.close();
-        frameMaker.close();
+        cursorHolder.close();
+        frameCursorHolder.close();
       }
     }
 
diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java
index 84f7c649e6a0..eb8162251967 100644
--- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java
+++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java
@@ -34,7 +34,7 @@
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
@@ -139,8 +139,8 @@ public Sequence frames()
       throw DruidException.defensive("Unrecognized frame type");
     }
 
-    final CursorMaker maker = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber);
-    final Cursor cursor = maker.makeCursor();
+    final CursorHolder cursorHolder = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber);
+    final Cursor cursor = cursorHolder.asCursor();
     return new BaseSequence<>(
         new BaseSequence.IteratorMaker>()
         {
@@ -198,6 +198,6 @@ public void cleanup(Iterator iterFromMake)
             // Nothing to do.
           }
         }
-    ).withBaggage(maker);
+    ).withBaggage(cursorHolder);
   }
 }
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 8f68324bf50f..b876af20c4d8 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
@@ -42,7 +42,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.VirtualColumns;
@@ -220,7 +220,7 @@ public static FrameSegment adapterToFrameSegment(
 
   /**
    * Reads a sequence of rows from a frame channel using a non-vectorized cursor from
-   * {@link FrameStorageAdapter#asCursorMaker(CursorBuildSpec)}.
+   * {@link FrameStorageAdapter#makeCursorHolder(CursorBuildSpec)}.
    *
    * @param channel     the channel
    * @param frameReader reader for this channel
@@ -233,14 +233,13 @@ public static Sequence> readRowsFromFrameChannel(
     return new FrameChannelSequence(channel)
         .flatMap(
             frame -> {
-              final CursorMaker maker = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).asCursorMaker(
-                  CursorBuildSpec.FULL_SCAN
-              );
-              final Cursor cursor = maker.makeCursor();
+              final CursorHolder cursorHolder = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)
+                  .makeCursorHolder(CursorBuildSpec.FULL_SCAN);
+              final Cursor cursor = cursorHolder.asCursor();
               if (cursor == null) {
-                return Sequences.withBaggage(Sequences.empty(), maker);
+                return Sequences.withBaggage(Sequences.empty(), cursorHolder);
               }
-              return readRowsFromCursor(cursor, frameReader.signature()).withBaggage(maker);
+              return readRowsFromCursor(cursor, frameReader.signature()).withBaggage(cursorHolder);
             }
         );
   }
@@ -262,12 +261,12 @@ public static Sequence> readRowsFromAdapter(
   )
   {
     final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature;
-    final CursorMaker maker = makeCursorForAdapter(adapter, populateRowNumber);
-    final Cursor cursor = maker.makeCursor();
+    final CursorHolder cursorHolder = makeCursorForAdapter(adapter, populateRowNumber);
+    final Cursor cursor = cursorHolder.asCursor();
     if (cursor == null) {
-      return Sequences.withBaggage(Sequences.empty(), maker);
+      return Sequences.withBaggage(Sequences.empty(), cursorHolder);
     }
-    return readRowsFromCursor(cursor, signatureToUse).withBaggage(maker);
+    return readRowsFromCursor(cursor, signatureToUse).withBaggage(cursorHolder);
   }
 
   /**
@@ -278,7 +277,7 @@ public static Sequence> readRowsFromAdapter(
    * @param adapter           the adapter
    * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN}
    */
-  public static CursorMaker makeCursorForAdapter(
+  public static CursorHolder makeCursorForAdapter(
       final StorageAdapter adapter,
       final boolean populateRowNumber
   )
@@ -299,15 +298,15 @@ public static CursorMaker makeCursorForAdapter(
                                                      .setVirtualColumns(virtualColumns)
                                                      .build();
 
-    final CursorMaker maker = adapter.asCursorMaker(buildSpec);
+    final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec);
     if (populateRowNumber) {
-      return new CursorMaker()
+      return new CursorHolder()
       {
         @Nullable
         @Override
-        public Cursor makeCursor()
+        public Cursor asCursor()
         {
-          final Cursor cursor = maker.makeCursor();
+          final Cursor cursor = cursorHolder.asCursor();
           if (cursor == null) {
             return null;
           }
@@ -317,11 +316,11 @@ public Cursor makeCursor()
         @Override
         public void close()
         {
-          maker.close();
+          cursorHolder.close();
         }
       };
     } else {
-      return maker;
+      return cursorHolder;
     }
   }
 
diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
index 9ea17a258027..9caaf40e0cbc 100644
--- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
@@ -50,7 +50,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.RowBasedSegment;
 import org.apache.druid.segment.RowIdSupplier;
@@ -577,8 +577,9 @@ private static Pair writeFrame(
       inputSegment = new FrameSegment(inputFrame, FrameReader.create(signature), SegmentId.dummy("xxx"));
     }
 
-    try (final CursorMaker maker = inputSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = inputSegment.asStorageAdapter()
+                                                       .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      final Cursor cursor = cursorHolder.asCursor();
 
       int numRows = 0;
       final FrameWriterFactory frameWriterFactory;
diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
index f708714cbcef..723ea11a49ff 100644
--- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
@@ -30,7 +30,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.StorageAdapter;
@@ -152,8 +152,8 @@ public void setup() throws IOException
   @Test
   public void testGranularizeFullScan()
   {
-    try (CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) {
-      final Cursor cursor = maker.makeCursor();
+    try (CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      final Cursor cursor = cursorHolder.asCursor();
       CursorGranularizer granularizer = CursorGranularizer.create(
           adapter,
           cursor,
@@ -206,8 +206,8 @@ public void testGranularizeFullScanDescending()
                                                       .setGranularity(Granularities.ALL)
                                                       .isDescending(true)
                                                       .build();
-    try (CursorMaker maker = adapter.asCursorMaker(descending)) {
-      final Cursor cursor = maker.makeCursor();
+    try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) {
+      final Cursor cursor = cursorHolder.asCursor();
       CursorGranularizer granularizer = CursorGranularizer.create(
           adapter,
           cursor,
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
index 79a0ae80875b..05d14613b5e8 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
@@ -37,7 +37,7 @@
 import org.apache.druid.query.filter.InDimFilter;
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
@@ -659,10 +659,10 @@ private void runResults(
     boolean allCanVectorize = segments.stream()
                                       .allMatch(
                                           s -> {
-                                            final CursorMaker maker = s.asStorageAdapter()
-                                                                       .asCursorMaker(spec);
-                                            final boolean canVectorize = maker.canVectorize();
-                                            maker.close();
+                                            final CursorHolder cursorHolder = s.asStorageAdapter()
+                                                                               .makeCursorHolder(spec);
+                                            final boolean canVectorize = cursorHolder.canVectorize();
+                                            cursorHolder.close();
                                             return canVectorize;
                                           });
 
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java
index 8403131e5b43..47e6dde69b0d 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java
@@ -30,7 +30,7 @@
 import org.apache.druid.query.groupby.epinephelinae.VectorGrouper;
 import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine.VectorGroupByEngineIterator;
 import org.apache.druid.segment.ColumnProcessors;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.TestIndex;
@@ -61,9 +61,9 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException
         .setAggregatorSpecs(factory)
         .build();
     final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex());
-    final CursorMaker maker = storageAdapter.asCursorMaker(query.asCursorBuildSpec(null));
+    final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(query.asCursorBuildSpec(null));
     final ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[4096]);
-    final VectorCursor cursor = maker.makeVectorCursor();
+    final VectorCursor cursor = cursorHolder.asVectorCursor();
     final List dimensions = query.getDimensions().stream().map(
         dimensionSpec ->
             ColumnProcessors.makeVectorProcessor(
@@ -94,6 +94,6 @@ VectorGrouper makeGrouper()
     };
     iterator.close();
     Mockito.verify(grouperCaptor.getValue()).close();
-    maker.close();
+    cursorHolder.close();
   }
 }
diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
index 8e2fbad18ccb..0ab09ed95a41 100644
--- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
+++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
@@ -29,7 +29,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.RowBasedStorageAdapter;
 import org.apache.druid.segment.column.ColumnCapabilities;
@@ -196,8 +196,8 @@ public void test_asStorageAdapter_makeCursor()
                                                      .setInterval(Intervals.of("1970/PT1H"))
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = LOOKUP_SEGMENT.asStorageAdapter().asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asStorageAdapter().makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
 
       final List> kvs = new ArrayList<>();
 
diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
index b9fc63db8808..1d707c68850d 100644
--- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
@@ -37,7 +37,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.column.TypeStrategy;
@@ -256,8 +256,8 @@ private void validateDecorated(
       if (interval != null) {
         builder.setInterval(interval);
       }
-      try (final CursorMaker maker = seggy.asStorageAdapter().asCursorMaker(builder.build())) {
-        final Cursor cursor = maker.makeCursor();
+      try (final CursorHolder cursorHolder = seggy.asStorageAdapter().makeCursorHolder(builder.build())) {
+        final Cursor cursor = cursorHolder.asCursor();
 
         vals = new ArrayList<>();
         final ColumnValueSelector idSupplier = cursor.getColumnSelectorFactory().makeColumnValueSelector("arrayIndex");
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
index fbac92a4adac..b56e87ad0969 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
@@ -41,7 +41,7 @@
 import org.apache.druid.query.filter.NullFilter;
 import org.apache.druid.query.filter.OrDimFilter;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.TestHelper;
 import org.apache.druid.segment.column.ColumnType;
@@ -666,10 +666,10 @@ private void runResults(
     boolean allCanVectorize = segments.stream()
                                       .allMatch(
                                           s -> {
-                                            final CursorMaker maker = s.asStorageAdapter()
-                                                                       .asCursorMaker(spec);
-                                            final boolean canVectorize = maker.canVectorize();
-                                            maker.close();
+                                            final CursorHolder cursorHolder = s.asStorageAdapter()
+                                                                               .makeCursorHolder(spec);
+                                            final boolean canVectorize = cursorHolder.canVectorize();
+                                            cursorHolder.close();
                                             return canVectorize;
                                           });
 
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java
index 16a7b7066af0..5c5cba6ece68 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java
@@ -33,7 +33,7 @@
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.IdLookup;
 import org.apache.druid.segment.Metadata;
@@ -318,13 +318,13 @@ public Metadata getMetadata()
       }
 
       @Override
-      public CursorMaker asCursorMaker(CursorBuildSpec spec)
+      public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       {
-        return new CursorMaker()
+        return new CursorHolder()
         {
           @Nullable
           @Override
-          public Cursor makeCursor()
+          public Cursor asCursor()
           {
             return null;
           }
diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java
index 2f5af7240e86..f1642760cacd 100644
--- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java
@@ -148,8 +148,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE
     index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -201,8 +201,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc
     index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -284,8 +284,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE
     index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -367,8 +367,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index
     index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(
           STRING_ARRAY_COL,
           STRING_ARRAY_COL,
@@ -415,8 +415,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize
     index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -459,8 +459,8 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee
     index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -518,8 +518,8 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz
     index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java
index ce18715d1c2c..f1076bd73d92 100644
--- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java
+++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java
@@ -148,8 +148,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
 
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -199,8 +199,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc
     index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -280,8 +280,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE
     index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -361,8 +361,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index
     index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(
           STRING_ARRAY_COL,
           STRING_ARRAY_COL,
@@ -404,8 +404,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize
     index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -447,8 +447,8 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee
     index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null));
 
     IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) {
+      Cursor cursor = cursorHolder.asCursor();
       final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING);
       ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorMakerTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorHolderTest.java
similarity index 75%
rename from processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorMakerTest.java
rename to processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorHolderTest.java
index 2a819636a418..349236a9dcf3 100644
--- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorMakerTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorHolderTest.java
@@ -25,7 +25,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
-public class QueryableIndexCursorMakerTest
+public class QueryableIndexCursorHolderTest
 {
   @Test
   public void testTimeSearch()
@@ -66,52 +66,52 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset)
 
     Assert.assertEquals(
         0,
-        QueryableIndexCursorMaker.timeSearch(column, 0, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 0, 0, values.length)
     );
 
     Assert.assertEquals(
         2,
-        QueryableIndexCursorMaker.timeSearch(column, 0, 2, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 0, 2, values.length)
     );
 
     Assert.assertEquals(
         0,
-        QueryableIndexCursorMaker.timeSearch(column, 0, 0, values.length / 2)
+        QueryableIndexCursorHolder.timeSearch(column, 0, 0, values.length / 2)
     );
 
     Assert.assertEquals(
         1,
-        QueryableIndexCursorMaker.timeSearch(column, 1, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 0, values.length)
     );
 
     Assert.assertEquals(
         2,
-        QueryableIndexCursorMaker.timeSearch(column, 1, 2, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 2, values.length)
     );
 
     Assert.assertEquals(
         1,
-        QueryableIndexCursorMaker.timeSearch(column, 1, 0, values.length / 2)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 0, values.length / 2)
     );
 
     Assert.assertEquals(
         1,
-        QueryableIndexCursorMaker.timeSearch(column, 1, 1, 8)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 1, 8)
     );
 
     Assert.assertEquals(
         8,
-        QueryableIndexCursorMaker.timeSearch(column, 2, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 2, 0, values.length)
     );
 
     Assert.assertEquals(
         10,
-        QueryableIndexCursorMaker.timeSearch(column, 10, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 10, 0, values.length)
     );
 
     Assert.assertEquals(
         11,
-        QueryableIndexCursorMaker.timeSearch(column, 15, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 15, 0, values.length)
     );
   }
 }
diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
index fcba838fd0de..4cde2a796585 100644
--- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
@@ -70,7 +70,7 @@ public void setUp()
       final QueryableIndex index = TestIndex.getMMappedTestIndex();
       final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index);
       if (vectorize) {
-        final VectorCursor cursor = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)).makeVectorCursor();
+        final VectorCursor cursor = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asVectorCursor();
 
         final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -81,8 +81,8 @@ public void setUp()
         partialNullSelector =
             columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column"));
       } else {
-        final CursorMaker maker = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN));
-        final Cursor cursor = maker.makeCursor();
+        final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN));
+        final Cursor cursor = cursorHolder.asCursor();
         final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
         qualitySelector =
@@ -226,8 +226,8 @@ public void setUp()
     {
       final QueryableIndex index = TestIndex.getMMappedTestIndex();
       final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index);
-      final CursorMaker maker = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN));
-      cursor = maker.makeCursor();
+      final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN));
+      cursor = cursorHolder.asCursor();
       columnSelectorFactory = cursor.getColumnSelectorFactory();
     }
 
diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
index 3ddc7841711e..1376e48789bc 100644
--- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
@@ -458,8 +458,8 @@ public void test_makeCursor_filterOnLong()
                                                      .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
 
       Assert.assertEquals(
           ImmutableList.of(
@@ -482,8 +482,8 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNull()
                                                      .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
 
       Assert.assertEquals(
           ImmutableList.of(
@@ -519,8 +519,8 @@ public void test_makeCursor_filterOnVirtualColumn()
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
 
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(
               ImmutableList.of("1")
@@ -538,8 +538,8 @@ public void test_makeCursor_descending()
     final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2);
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build();
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(
               ImmutableList.of("2"),
@@ -562,8 +562,8 @@ public void test_makeCursor_intervalDoesNotMatch()
                                                      .setInterval(Intervals.of("2000/P1D"))
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(),
           walkCursor(cursor, READ_STRING)
@@ -582,8 +582,8 @@ public void test_makeCursor_intervalPartiallyMatches()
                                                      .setInterval(Intervals.of("1970-01-01T01/PT1H"))
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(
               ImmutableList.of("1")
@@ -604,8 +604,8 @@ public void test_makeCursor_hourGranularity()
                                                      .setInterval(Intervals.of("1970/1971"))
                                                      .setGranularity(Granularities.HOUR)
                                                      .build();
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(
               ImmutableList.of(DateTimes.of("1970-01-01T00"), "0"),
@@ -631,8 +631,8 @@ public void test_makeCursor_hourGranularityWithInterval()
                                                      .setGranularity(Granularities.HOUR)
                                                      .build();
 
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
 
       Assert.assertEquals(
           ImmutableList.of(
@@ -658,8 +658,8 @@ public void test_makeCursor_hourGranularityWithIntervalDescending()
                                                      .isDescending(true)
                                                      .build();
 
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(
               ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"),
@@ -678,8 +678,8 @@ public void test_makeCursor_allProcessors()
   {
     final RowBasedStorageAdapter adapter = createIntAdapter(0, 1);
 
-    try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(
               Lists.newArrayList(
@@ -788,8 +788,8 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull()
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
 
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
           ImmutableList.of(),
           walkCursor(cursor, new ArrayList<>(PROCESSORS.values()))
@@ -804,11 +804,11 @@ public void test_makeCursor_eternityIntervalWithMonthGranularity()
   {
     final RowBasedStorageAdapter adapter = createIntAdapter(0, 1);
     Assert.assertThrows(IAE.class, () -> {
-      adapter.asCursorMaker(
+      adapter.makeCursorHolder(
           CursorBuildSpec.builder()
                          .setGranularity(Granularities.MONTH)
                          .build()
-      ).makeCursor();
+      ).asCursor();
     });
   }
 
diff --git a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java
index 7d51b9e46e69..8950693f14fa 100644
--- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java
@@ -37,13 +37,13 @@ public void testTombstoneDefaultInterface()
     StorageAdapter sa = new StorageAdapter()
     {
       @Override
-      public CursorMaker asCursorMaker(CursorBuildSpec spec)
+      public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       {
-        return new CursorMaker()
+        return new CursorHolder()
         {
           @Nullable
           @Override
-          public Cursor makeCursor()
+          public Cursor asCursor()
           {
             return null;
           }
diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
index 4667ffe0a36e..1e9246682990 100644
--- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
@@ -232,8 +232,8 @@ public void test_unnest_adapters_basic()
                                                      .setInterval(UNNEST_STORAGE_ADAPTER.getInterval())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = UNNEST_STORAGE_ADAPTER.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
 
       ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
 
@@ -262,8 +262,8 @@ public void test_two_levels_of_unnest_adapters()
                                                      .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = UNNEST_STORAGE_ADAPTER1.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
 
       DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME1));
@@ -320,8 +320,8 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest
                                                      .setInterval(unnestStorageAdapter.getInterval())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter();
       final Filter pushDownFilter = base.getPushDownFilter();
 
@@ -367,8 +367,8 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap
                                                      .setInterval(unnestStorageAdapter.getInterval())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter();
       final Filter pushDownFilter = base.getPushDownFilter();
 
@@ -647,8 +647,8 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters()
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
 
-    try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter();
       final Filter pushDownFilter = base.getPushDownFilter();
 
@@ -690,8 +690,8 @@ public void test_pushdown_filters_unnested_dimension_outside()
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
 
-    try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter();
       final Filter pushDownFilter = base.getPushDownFilter();
 
@@ -737,8 +737,8 @@ public void testUnnestValueMatcherValueDoesntExist()
                                                      .setInterval(withNullsStorageAdapter.getInterval())
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = withNullsStorageAdapter.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
 
       DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME));
@@ -832,9 +832,9 @@ public Filter getPushDownFilter()
   }
 
   @Override
-  public CursorMaker asCursorMaker(CursorBuildSpec spec)
+  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
     this.pushDownFilter = spec.getFilter();
-    return super.asCursorMaker(spec);
+    return super.makeCursorHolder(spec);
   }
 }
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 b8bbdee3b67a..1c884f9900ca 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
@@ -70,7 +70,7 @@
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.IndexSpec;
@@ -818,7 +818,7 @@ private CursorBuildSpec makeVectorCursorBuildSpec(@Nullable Filter filter)
   private VectorCursor makeVectorCursor(final Filter filter)
   {
     final CursorBuildSpec buildSpec = makeVectorCursorBuildSpec(filter);
-    return adapter.asCursorMaker(buildSpec).makeVectorCursor();
+    return adapter.makeCursorHolder(buildSpec).asVectorCursor();
   }
 
   /**
@@ -826,8 +826,8 @@ private VectorCursor makeVectorCursor(final Filter filter)
    */
   private List selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn)
   {
-    try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(makeFilter(filter)))) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(makeFilter(filter)))) {
+      final Cursor cursor = cursorHolder.asCursor();
       final DimensionSelector selector = cursor
           .getColumnSelectorFactory()
           .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn));
@@ -846,8 +846,8 @@ private List selectColumnValuesMatchingFilter(final DimFilter filter, fi
 
   private long selectCountUsingFilteredAggregator(final DimFilter filter)
   {
-    try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(null))) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(null))) {
+      final Cursor cursor = cursorHolder.asCursor();
       Aggregator agg = new FilteredAggregatorFactory(
           new CountAggregatorFactory("count"),
           maybeOptimize(filter)
@@ -870,8 +870,8 @@ private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFil
     );
 
 
-    try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(null))) {
-      final VectorCursor cursor = maker.makeVectorCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(null))) {
+      final VectorCursor cursor = cursorHolder.asVectorCursor();
       final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory(
           new CountAggregatorFactory("count"),
           maybeOptimize(dimFilter)
@@ -937,8 +937,8 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
       }
     };
 
-    try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(postFilteringFilter))) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(postFilteringFilter))) {
+      final Cursor cursor = cursorHolder.asCursor();
       final DimensionSelector selector = cursor
           .getColumnSelectorFactory()
           .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn));
@@ -997,8 +997,8 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
       }
     };
 
-    try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(postFilteringFilter))) {
-      final VectorCursor cursor = maker.makeVectorCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(postFilteringFilter))) {
+      final VectorCursor cursor = cursorHolder.asVectorCursor();
       final SingleValueDimensionVectorSelector selector = cursor
           .getColumnSelectorFactory()
           .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn));
@@ -1022,8 +1022,8 @@ private List selectColumnValuesMatchingFilterUsingVectorCursor(
       final String selectColumn
   )
   {
-    try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter)))) {
-      final VectorCursor cursor = maker.makeVectorCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) {
+      final VectorCursor cursor = cursorHolder.asVectorCursor();
       final SingleValueDimensionVectorSelector selector = cursor
           .getColumnSelectorFactory()
           .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn));
@@ -1049,8 +1049,8 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur
   )
   {
     final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE);
-    try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter)))) {
-      final VectorCursor cursor = maker.makeVectorCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) {
+      final VectorCursor cursor = cursorHolder.asVectorCursor();
 
       final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory());
       final List values = new ArrayList<>();
diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
index 4f6a7bca28a9..8f284d5284ee 100644
--- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
@@ -59,7 +59,7 @@
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.data.IndexedInts;
@@ -131,29 +131,29 @@ public void testSanity() throws Exception
     );
 
 
+    final GroupByQuery query = GroupByQuery.builder()
+                                           .setDataSource("test")
+                                           .setGranularity(Granularities.ALL)
+                                           .setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
+                                           .addDimension("billy")
+                                           .addDimension("sally")
+                                           .addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
+                                           .addOrderByColumn("billy")
+                                           .build();
+    final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
+    final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
     try (
         CloseableStupidPool pool = new CloseableStupidPool<>(
             "GroupByQueryEngine-bufferPool",
             () -> ByteBuffer.allocate(50000)
         );
-        ResourceHolder processingBuffer = pool.take()
+        ResourceHolder processingBuffer = pool.take();
+        final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)
     ) {
-      final GroupByQuery query = GroupByQuery.builder()
-                                             .setDataSource("test")
-                                             .setGranularity(Granularities.ALL)
-                                             .setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
-                                             .addDimension("billy")
-                                             .addDimension("sally")
-                                             .addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
-                                             .addOrderByColumn("billy")
-                                             .build();
-      final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
-      final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
-      final CursorMaker maker = adapter.asCursorMaker(buildSpec);
       final Sequence rows = GroupByQueryEngine.process(
           query,
           adapter,
-          maker,
+          cursorHolder,
           buildSpec,
           processingBuffer.get(),
           null,
@@ -195,41 +195,41 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception
         )
     );
 
+    final GroupByQuery query = GroupByQuery.builder()
+                                           .setDataSource("test")
+                                           .setGranularity(Granularities.ALL)
+                                           .setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
+                                           .addDimension("billy")
+                                           .addDimension("sally")
+                                           .addAggregator(
+                                               new LongSumAggregatorFactory("cnt", "cnt")
+                                           )
+                                           .addAggregator(
+                                               new JavaScriptAggregatorFactory(
+                                                   "fieldLength",
+                                                   Arrays.asList("sally", "billy"),
+                                                   "function(current, s, b) { return current + (s == null ? 0 : s.length) + (b == null ? 0 : b.length); }",
+                                                   "function() { return 0; }",
+                                                   "function(a,b) { return a + b; }",
+                                                   JavaScriptConfig.getEnabledInstance()
+                                               )
+                                           )
+                                           .addOrderByColumn("billy")
+                                           .build();
+    final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
+    final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
     try (
         CloseableStupidPool pool = new CloseableStupidPool<>(
             "GroupByQueryEngine-bufferPool",
             () -> ByteBuffer.allocate(50000)
         );
         ResourceHolder processingBuffer = pool.take();
+        final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)
     ) {
-      final GroupByQuery query = GroupByQuery.builder()
-                                             .setDataSource("test")
-                                             .setGranularity(Granularities.ALL)
-                                             .setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
-                                             .addDimension("billy")
-                                             .addDimension("sally")
-                                             .addAggregator(
-                                                 new LongSumAggregatorFactory("cnt", "cnt")
-                                             )
-                                             .addAggregator(
-                                                 new JavaScriptAggregatorFactory(
-                                                     "fieldLength",
-                                                     Arrays.asList("sally", "billy"),
-                                                     "function(current, s, b) { return current + (s == null ? 0 : s.length) + (b == null ? 0 : b.length); }",
-                                                     "function() { return 0; }",
-                                                     "function(a,b) { return a + b; }",
-                                                     JavaScriptConfig.getEnabledInstance()
-                                                 )
-                                             )
-                                             .addOrderByColumn("billy")
-                                             .build();
-      final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
-      final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
-      final CursorMaker maker = adapter.asCursorMaker(buildSpec);
       final Sequence rows = GroupByQueryEngine.process(
           query,
           adapter,
-          maker,
+          cursorHolder,
           buildSpec,
           processingBuffer.get(),
           null,
@@ -283,8 +283,8 @@ public void testResetSanity() throws IOException
                                                        .setInterval(interval)
                                                        .isDescending(descending)
                                                        .build();
-      try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-        Cursor cursor = maker.makeCursor();
+      try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+        Cursor cursor = cursorHolder.asCursor();
         DimensionSelector dimSelector;
 
         dimSelector = cursor
@@ -372,30 +372,31 @@ public void testFilterByNull() throws Exception
         )
     );
 
+
+    final GroupByQuery query = GroupByQuery.builder()
+                                           .setDataSource("test")
+                                           .setGranularity(Granularities.ALL)
+                                           .setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
+                                           .addDimension("billy")
+                                           .addDimension("sally")
+                                           .addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
+                                           .setDimFilter(DimFilters.dimEquals("sally", (String) null))
+                                           .build();
+    final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
+    final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
+
     try (
         CloseableStupidPool pool = new CloseableStupidPool<>(
             "GroupByQueryEngine-bufferPool",
             () -> ByteBuffer.allocate(50000)
         );
         ResourceHolder processingBuffer = pool.take();
+        final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)
     ) {
-
-      final GroupByQuery query = GroupByQuery.builder()
-                                             .setDataSource("test")
-                                             .setGranularity(Granularities.ALL)
-                                             .setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
-                                             .addDimension("billy")
-                                             .addDimension("sally")
-                                             .addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
-                                             .setDimFilter(DimFilters.dimEquals("sally", (String) null))
-                                             .build();
-      final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
-      final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
-      final CursorMaker maker = adapter.asCursorMaker(buildSpec);
       final Sequence rows = GroupByQueryEngine.process(
           query,
           adapter,
-          maker,
+          cursorHolder,
           buildSpec,
           processingBuffer.get(),
           null,
@@ -434,8 +435,8 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception
                                                      .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000))
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       DimensionSelector dimSelector = cursor
           .getColumnSelectorFactory()
           .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
@@ -492,8 +493,8 @@ public void testCursorDictionaryRaceConditionFix() throws Exception
                                                      .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000))
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
       DimensionSelector dimSelector = cursor
           .getColumnSelectorFactory()
           .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
@@ -532,8 +533,8 @@ public void testCursoringAndSnapshot() throws Exception
                                                      .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000))
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
-    try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) {
+      Cursor cursor = cursorHolder.asCursor();
 
       DimensionSelector dimSelector1A = cursor
           .getColumnSelectorFactory()
diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java
index 286d03b2ada9..c3a59659afc6 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java
@@ -223,7 +223,7 @@ protected JoinableClause regionToCountry(final JoinType joinType)
 
   /**
    * Creates a fact-to-country join segment without a {@link JoinFilterPreAnalysis}. This means it cannot
-   * have {@link org.apache.druid.segment.StorageAdapter#asCursorMaker(CursorBuildSpec)} called on it.
+   * have {@link org.apache.druid.segment.StorageAdapter#makeCursorHolder(CursorBuildSpec)} called on it.
    */
   protected HashJoinSegmentStorageAdapter makeFactToCountrySegment()
   {
diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
index ab5f3636f5e6..4bef35618d8c 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
@@ -311,7 +311,7 @@ public void test_makeCursor_factToCountryLeft()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -368,7 +368,7 @@ public void test_makeCursor_factToCountryLeftUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -423,7 +423,7 @@ public void test_makeCursor_factToCountryInner()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -473,7 +473,7 @@ public void test_makeCursor_factToCountryInnerUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -525,7 +525,7 @@ public void test_makeCursor_factToCountryInnerUsingCountryNumber()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -585,7 +585,7 @@ public void test_makeCursor_factToCountryInnerUsingCountryNumberUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -641,7 +641,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnFacts()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -672,7 +672,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnFactsUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -702,7 +702,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -735,7 +735,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup(
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -767,7 +767,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -800,7 +800,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -837,7 +837,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnJoinable()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -874,7 +874,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnJoinableUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -911,7 +911,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnJoinable()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -948,7 +948,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnJoinableUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -998,7 +998,7 @@ public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditi
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1069,7 +1069,7 @@ public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditi
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1123,7 +1123,7 @@ public void test_makeCursor_factToRegionToCountryLeft()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             FACT_TO_REGION_PREFIX + "regionName",
@@ -1180,7 +1180,7 @@ public void test_makeCursor_factToRegionToCountryInnerIncludeNull()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             FACT_TO_REGION_PREFIX + "regionName",
@@ -1246,7 +1246,7 @@ public void test_makeCursor_factToCountryAlwaysTrue()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1305,7 +1305,7 @@ public void test_makeCursor_factToCountryAlwaysFalse()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1346,7 +1346,7 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1405,7 +1405,7 @@ public void test_makeCursor_factToCountryAlwaysFalseUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1450,7 +1450,7 @@ public void test_makeCursor_factToCountryUsingVirtualColumn()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1501,7 +1501,7 @@ public void test_makeCursor_factToCountryUsingVirtualColumnUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1549,7 +1549,7 @@ public void test_makeCursor_factToCountryUsingExpression()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -1594,7 +1594,7 @@ public void test_makeCursor_factToCountryUsingExpressionUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -1641,7 +1641,7 @@ public void test_makeCursor_factToRegionTheWrongWay()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1692,7 +1692,7 @@ public void test_makeCursor_errorOnNonEquiJoin()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of()
     );
   }
@@ -1727,7 +1727,7 @@ public void test_makeCursor_errorOnNonEquiJoinUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of()
     );
   }
@@ -1762,7 +1762,7 @@ public void test_makeCursor_errorOnNonKeyBasedJoin()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of()
     );
   }
@@ -1797,7 +1797,7 @@ public void test_makeCursor_errorOnNonKeyBasedJoinUsingLookup()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of()
     );
   }
@@ -1818,7 +1818,7 @@ public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRows()
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1847,7 +1847,7 @@ public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRowsUsingLook
             factSegment.asStorageAdapter(),
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(
+        ).makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1875,7 +1875,7 @@ public void test_makeCursor_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE
         factSegment.asStorageAdapter(),
         joinableClauses,
         joinFilterPreAnalysis
-    ).asCursorMaker(
+    ).makeCursorHolder(
         CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
     );
   }
@@ -1901,7 +1901,7 @@ public void test_makeCursor_factToCountryLeftWithBaseFilter()
             baseFilter,
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -1938,7 +1938,7 @@ public void test_makeCursor_factToCountryInnerWithBaseFilter()
             baseFilter,
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -1974,7 +1974,7 @@ public void test_makeCursor_factToCountryRightWithBaseFilter()
             baseFilter,
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
@@ -2027,7 +2027,7 @@ public void test_makeCursor_factToCountryFullWithBaseFilter()
             baseFilter,
             joinableClauses,
             joinFilterPreAnalysis
-        ).asCursorMaker(CursorBuildSpec.FULL_SCAN),
+        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
         ImmutableList.of(
             "page",
             "countryIsoCode",
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 4d5966662ba7..30fa7110d30b 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
@@ -83,7 +83,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel()
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -155,7 +155,7 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -206,7 +206,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -268,7 +268,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns()
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -334,7 +334,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns(
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -399,7 +399,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder()
                            .setFilter(originalFilter)
                            .setVirtualColumns(virtualColumns)
@@ -472,7 +472,7 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder()
                            .setFilter(originalFilter)
                            .setVirtualColumns(virtualColumns)
@@ -575,7 +575,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -697,7 +697,7 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -786,7 +786,7 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan
         joinFilterPreAnalysis
     );
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -831,7 +831,7 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter()
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -928,7 +928,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1013,7 +1013,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1082,7 +1082,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1135,7 +1135,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1187,7 +1187,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns()
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1238,7 +1238,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1290,7 +1290,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1342,7 +1342,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan
     // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman"
     // is interpreted as 0 (a.k.a. Australia).
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1401,7 +1401,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan
     // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman"
     // is interpreted as 0 (a.k.a. Australia).
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1457,7 +1457,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1507,7 +1507,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1556,7 +1556,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1609,7 +1609,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1661,7 +1661,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls()
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1712,7 +1712,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup()
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1774,7 +1774,7 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1840,7 +1840,7 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1916,7 +1916,7 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -1972,7 +1972,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -2047,7 +2047,7 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -2245,7 +2245,7 @@ public boolean supportsRequiredColumnRewrite()
     }
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -2374,7 +2374,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
@@ -2449,7 +2449,7 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName
     );
 
     JoinTestHelper.verifyCursor(
-        adapter.asCursorMaker(
+        adapter.makeCursorHolder(
             CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
         ),
         ImmutableList.of(
diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java
index 6c99bd1a39e1..3518be0af022 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java
@@ -46,7 +46,7 @@
 import org.apache.druid.segment.ColumnProcessorFactory;
 import org.apache.druid.segment.ColumnProcessors;
 import org.apache.druid.segment.Cursor;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.IndexBuilder;
@@ -324,10 +324,10 @@ public static RowBasedIndexedTable> createRegionsIndexedTabl
     );
   }
 
-  public static List readCursor(final CursorMaker cursorMaker, final List columns)
+  public static List readCursor(final CursorHolder cursorHolder, final List columns)
   {
     try {
-      final Cursor cursor = cursorMaker.makeCursor();
+      final Cursor cursor = cursorHolder.asCursor();
       final List> readers = columns
           .stream()
           .map(
@@ -363,17 +363,17 @@ public static List readCursor(final CursorMaker cursorMaker, final Lis
       return rows;
     }
     finally {
-      cursorMaker.close();
+      cursorHolder.close();
     }
   }
 
   public static void verifyCursor(
-      final CursorMaker cursorMaker,
+      final CursorHolder cursorHolder,
       final List columns,
       final List expectedRows
   )
   {
-    final List rows = readCursor(cursorMaker, columns);
+    final List rows = readCursor(cursorHolder, columns);
 
     for (int i = 0; i < rows.size(); i++) {
       try {
diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
index 9e7ee660fa51..c7f12d4ec190 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
@@ -26,7 +26,7 @@
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.QueryableIndexSegment;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
@@ -70,16 +70,16 @@ public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch
       }
 
       @Override
-      public CursorMaker asCursorMaker(CursorBuildSpec spec)
+      public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       {
-        final CursorMaker delegate = super.asCursorMaker(spec);
-        return new CursorMaker()
+        final CursorHolder delegate = super.makeCursorHolder(spec);
+        return new CursorHolder()
         {
           @Nullable
           @Override
-          public Cursor makeCursor()
+          public Cursor asCursor()
           {
-            return new CursorNoAdvance(delegate.makeCursor(), countDownLatch);
+            return new CursorNoAdvance(delegate.asCursor(), countDownLatch);
           }
 
           @Override
@@ -242,8 +242,8 @@ public void makeCursorAndAdvance()
         joinFilterPreAnalysis
     );
 
-    try (final CursorMaker maker = hashJoinSegmentStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = hashJoinSegmentStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      Cursor cursor = cursorHolder.asCursor();
 
       ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher()
       {
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 e66460788360..113e0d4e84be 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
@@ -32,7 +32,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DoubleColumnSelector;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.LongColumnSelector;
@@ -350,8 +350,8 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir
                                                      .setVirtualColumns(virtualColumns)
                                                      .setGranularity(Granularities.DAY)
                                                      .build();
-    final CursorMaker maker = closer.register(storageAdapter.asCursorMaker(buildSpec));
-    final Cursor cursor = maker.makeCursor();
+    final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec));
+    final Cursor cursor = cursorHolder.asCursor();
     return cursor.getColumnSelectorFactory();
   }
 
@@ -373,7 +373,7 @@ private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumn
     Assert.assertEquals(1, segments.size());
     StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build();
-    VectorCursor cursor = closer.register(storageAdapter.asCursorMaker(buildSpec)).makeVectorCursor();
+    VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor();
     return cursor.getColumnSelectorFactory();
   }
 }
diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
index ef0ef087e62e..fb0a2e9fe227 100644
--- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
@@ -44,7 +44,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
@@ -142,8 +142,8 @@ public void test_single_value_string_bindings()
                                                        .setInterval(adapter.getInterval())
                                                        .setGranularity(Granularities.ALL)
                                                        .build();
-      try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-        Cursor cursor = maker.makeCursor();
+      try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+        Cursor cursor = cursorHolder.asCursor();
 
         ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
         ExpressionPlan plan = ExpressionPlanner.plan(
@@ -211,8 +211,8 @@ public void test_multi_value_string_bindings()
                                                        .setInterval(adapter.getInterval())
                                                        .setGranularity(Granularities.ALL)
                                                        .build();
-      try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-        Cursor cursor = maker.makeCursor();
+      try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+        Cursor cursor = cursorHolder.asCursor();
         ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
 
         // identifier, uses dimension selector supplier supplier, no null coercion
@@ -292,8 +292,8 @@ public void test_long_bindings()
                                                        .setInterval(adapter.getInterval())
                                                        .setGranularity(Granularities.ALL)
                                                        .build();
-      try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-        Cursor cursor = maker.makeCursor();
+      try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+        Cursor cursor = cursorHolder.asCursor();
         ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
         // an assortment of plans
         ExpressionPlan plan = ExpressionPlanner.plan(
@@ -341,8 +341,8 @@ public void test_double_bindings()
                                                        .setInterval(adapter.getInterval())
                                                        .setGranularity(Granularities.ALL)
                                                        .build();
-      try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-        Cursor cursor = maker.makeCursor();
+      try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+        Cursor cursor = cursorHolder.asCursor();
         ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
         // an assortment of plans
         ExpressionPlan plan = ExpressionPlanner.plan(
@@ -660,8 +660,8 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept
     );
 
     IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
-    try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) {
-      Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      Cursor cursor = cursorHolder.asCursor();
       DimensionSelector xExprSelector = ExpressionSelectors.makeDimensionSelector(
           cursor.getColumnSelectorFactory(),
           Parser.parse("concat(x, 'foo')", 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 b96bf9ebbfbd..8c1646e93a5a 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
@@ -121,7 +121,7 @@ public static void testCast(
                                                      .setVirtualColumns(virtualColumns)
                                                      .setInterval(index.getDataInterval())
                                                      .build();
-    VectorCursor cursor = closer.register(storageAdapter.asCursorMaker(buildSpec)).makeVectorCursor();
+    VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor();
 
     ColumnCapabilities capabilities = INDEX.getColumnCapabilities(column);
 
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 dedd6ad51a65..ab7ddf19b58f 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
@@ -36,7 +36,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.QueryableIndex;
@@ -240,8 +240,8 @@ public static void sanityTestVectorizedExpressionSelectors(
                                                      .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(virtualColumns)
                                                      .build();
-    try (final CursorMaker maker = storageAdapter.asCursorMaker(buildSpec)) {
-      final VectorCursor cursor = maker.makeVectorCursor();
+    try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) {
+      final VectorCursor cursor = cursorHolder.asVectorCursor();
       Assert.assertNotNull(cursor);
 
       ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v");
@@ -314,7 +314,7 @@ public static void sanityTestVectorizedExpressionSelectors(
       }
 
 
-      final Cursor nonVectorized = maker.makeCursor();
+      final Cursor nonVectorized = cursorHolder.asCursor();
       Assert.assertNotNull(nonVectorized);
       final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory()
                                                            .makeColumnValueSelector("v");
diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java
index 2cd239ebde0b..14eb3d44220d 100644
--- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java
+++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java
@@ -29,7 +29,7 @@
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionHandler;
 import org.apache.druid.segment.IndexIO;
 import org.apache.druid.segment.Metadata;
@@ -329,13 +329,13 @@ public Metadata getMetadata()
         }
 
         @Override
-        public CursorMaker asCursorMaker(CursorBuildSpec spec)
+        public CursorHolder makeCursorHolder(CursorBuildSpec spec)
         {
-          return new CursorMaker()
+          return new CursorHolder()
           {
             @Nullable
             @Override
-            public Cursor makeCursor()
+            public Cursor asCursor()
             {
               return null;
             }
diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java
index 116ce0169fb7..60ba59912df9 100644
--- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java
+++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java
@@ -72,7 +72,7 @@
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorMaker;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.IndexIO;
 import org.apache.druid.segment.QueryableIndex;
@@ -309,8 +309,8 @@ public static void runDump(
                                                      .setGranularity(Granularities.ALL)
                                                      .build();
 
-    try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
-      final Cursor cursor = maker.makeCursor();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
       if (cursor == null) {
         return;
       }

From 296fd80c58e31222f01353d0a13f04c3d9a73072 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Fri, 2 Aug 2024 18:48:08 -0700
Subject: [PATCH 30/74] javadocs

---
 .../main/java/org/apache/druid/query/CursorGranularizer.java | 5 ++++-
 .../apache/druid/query/vector/VectorCursorGranularizer.java  | 5 ++++-
 2 files changed, 8 insertions(+), 2 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java
index d54be0455a5e..b9d3418779a3 100644
--- a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java
+++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java
@@ -34,7 +34,10 @@
 import javax.annotation.Nullable;
 
 /**
- * Class that helps non-vectorized query engines handle "granularity" parameters. Given a set of intervals
+ * Class that helps non-vectorized query engines handle "granularity" parameters. Given a set of intervals, this class
+ * provides mechansims to advance a cursor to the start of an interval ({@link #advanceToBucket(Interval)}),
+ * advance a cursor within a bucket interval ({@link #advanceCursorWithinBucket()}), and check if the current cursor
+ * position is within the bucket {@link #currentOffsetWithinBucket()}.
  *
  * @see org.apache.druid.query.vector.VectorCursorGranularizer for vectorized query engines.
  */
diff --git a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java
index 38288dc7dc3f..d899e1f08884 100644
--- a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java
+++ b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java
@@ -31,7 +31,10 @@
 import javax.annotation.Nullable;
 
 /**
- * Class that helps vectorized query engines handle "granularity" parameters.
+ * Class that helps vectorized query engines handle "granularity" parameters. Given a set of intervals, this class
+ * provides mechansims to advance a cursor to the start of an interval ({@link #setCurrentOffsets(Interval)}),
+ * advance a cursor within a bucket interval ({@link #advanceCursorWithinBucket()}), and check the offsets of the
+ * current vector that are within the bucket ({@link #getStartOffset()}, {@link #getEndOffset()}).
  *
  * @see org.apache.druid.query.CursorGranularizer for non-vectorized query engines.
  */

From e567c165b31b54673c43b8a312af4ba0c3c53a1b Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Sat, 3 Aug 2024 11:12:50 -0700
Subject: [PATCH 31/74] remove Granularity from CursorBuildSpec, add
 granularity virtual column to CursorBuildSpec grouping columns

---
 .../ExpressionAggregationBenchmark.java       |  1 -
 .../benchmark/ExpressionFilterBenchmark.java  |  2 -
 .../ExpressionSelectorBenchmark.java          | 17 +----
 .../ExpressionVectorSelectorBenchmark.java    |  1 -
 .../benchmark/FilterPartitionBenchmark.java   |  2 -
 .../benchmark/JoinAndLookupBenchmark.java     | 10 ---
 .../IncrementalIndexReadBenchmark.java        |  2 -
 .../hadoop/DatasourceRecordReader.java        |  2 -
 .../indexing/input/DruidSegmentReader.java    |  2 -
 .../common/task/CompactionTaskRunTest.java    |  3 -
 .../indexing/common/task/IndexTaskTest.java   |  2 -
 .../frame/processor/FrameProcessors.java      |  2 -
 .../columnar/FrameCursorMakerFactory.java     | 75 +++++++------------
 .../segment/row/FrameCursorMakerFactory.java  | 62 +++++++--------
 .../common/granularity/Granularities.java     | 45 +++++++++++
 .../druid/math/expr/ExprMacroTable.java       | 10 +++
 .../java/org/apache/druid/query/Query.java    |  1 -
 .../expression/TimestampFloorExprMacro.java   | 37 +++++----
 .../druid/query/groupby/GroupByQuery.java     |  5 +-
 .../epinephelinae/GroupByQueryEngine.java     |  2 +-
 .../druid/query/metadata/SegmentAnalyzer.java |  2 -
 .../LazilyDecoratedRowsAndColumns.java        |  4 +-
 .../apache/druid/query/scan/ScanQuery.java    |  1 -
 .../query/timeseries/TimeseriesQuery.java     |  4 +-
 .../apache/druid/query/topn/TopNQuery.java    |  9 ++-
 .../druid/query/topn/TopNQueryEngine.java     |  2 +-
 .../apache/druid/segment/CursorBuildSpec.java | 48 +++++++-----
 .../segment/QueryableIndexCursorHolder.java   |  6 +-
 .../segment/QueryableIndexStorageAdapter.java | 23 +-----
 .../druid/segment/RowBasedStorageAdapter.java | 28 ++-----
 .../IncrementalIndexCursorHolder.java         | 27 +++----
 .../join/HashJoinSegmentStorageAdapter.java   |  1 -
 .../table/BroadcastSegmentIndexedTable.java   |  2 -
 .../segment/FrameStorageAdapterTest.java      |  3 -
 .../druid/frame/testutil/FrameTestUtil.java   |  2 -
 .../granularity/QueryGranularityTest.java     | 35 +++++++++
 .../druid/query/CursorGranularizerTest.java   |  1 -
 .../druid/query/groupby/GroupByQueryTest.java | 58 +++++++++++++-
 .../druid/query/lookup/LookupSegmentTest.java |  2 -
 .../semantic/RowsAndColumnsDecoratorTest.java |  4 +-
 .../druid/query/scan/ScanQueryTest.java       |  2 -
 .../query/timeseries/TimeseriesQueryTest.java | 59 ++++++++++++++-
 .../druid/query/topn/TopNQueryTest.java       | 41 +++++++++-
 .../segment/RowBasedStorageAdapterTest.java   | 35 ++-------
 .../segment/UnnestStorageAdapterTest.java     |  7 --
 .../druid/segment/filter/BaseFilterTest.java  |  3 -
 .../IncrementalIndexStorageAdapterTest.java   |  3 -
 .../HashJoinSegmentStorageAdapterTest.java    | 49 ++++++------
 .../segment/join/JoinFilterAnalyzerTest.java  | 67 ++++++++---------
 .../NestedFieldColumnSelectorsTest.java       |  1 -
 .../virtual/ExpressionSelectorsTest.java      |  4 -
 .../ExpressionVectorSelectorsTest.java        |  1 -
 .../org/apache/druid/cli/DumpSegment.java     |  2 -
 53 files changed, 438 insertions(+), 381 deletions(-)

diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
index f338a0605be3..2e6931637ece 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
@@ -165,7 +165,6 @@ private double compute(final Function a
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java
index db6ebb5e7a15..ac79a7bfd66a 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java
@@ -148,7 +148,6 @@ public void expressionFilter(Blackhole blackhole)
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(expressionFilter.toFilter())
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -168,7 +167,6 @@ public void nativeFilter(Blackhole blackhole)
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(nativeFilter.toFilter())
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
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 627cb4ca9457..89a99834899f 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
@@ -148,7 +148,6 @@ public void timeFloorUsingExpression(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -173,7 +172,6 @@ public void timeFloorUsingExtractionFn(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -196,7 +194,6 @@ public void timeFloorUsingCursor(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.HOUR)
                                                      .build();
     final StorageAdapter adapter = new QueryableIndexStorageAdapter(index);
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
@@ -204,7 +201,7 @@ public void timeFloorUsingCursor(Blackhole blackhole)
       final CursorGranularizer granularizer = CursorGranularizer.create(
           adapter,
           cursor,
-          buildSpec.getGranularity(),
+          Granularities.HOUR,
           buildSpec.getInterval(),
           buildSpec.isDescending()
       );
@@ -238,7 +235,6 @@ public void timeFormatUsingExpression(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -265,7 +261,6 @@ public void timeFormatUsingExtractionFn(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -287,7 +282,6 @@ public void strlenUsingExpressionAsLong(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -312,7 +306,6 @@ public void strlenUsingExpressionAsString(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -340,7 +333,6 @@ public void strlenUsingExtractionFn(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
 
     try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
@@ -358,7 +350,6 @@ public void arithmeticOnLong(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -383,7 +374,6 @@ public void stringConcatAndCompareOnLong(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -407,7 +397,6 @@ public void caseSearched1(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -431,7 +420,6 @@ public void caseSearched2(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -467,7 +455,6 @@ public void caseSearched100(Blackhole blackhole)
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -492,7 +479,6 @@ public void caseSearchedWithLookup(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              new ExpressionVirtualColumn(
@@ -523,7 +509,6 @@ public void caseSearchedWithLookup2(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(
                                                          VirtualColumns.create(
                                                              ImmutableList.of(
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 a6179a9b56b6..695ddcc6471d 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java
@@ -153,7 +153,6 @@ public void scan(Blackhole blackhole)
     );
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(virtualColumns)
                                                      .build();
     final CursorHolder cursorHolder = closer.register(
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java
index c4ee7ee842e9..65dde3cb65a3 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java
@@ -25,7 +25,6 @@
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.js.JavaScriptConfig;
 import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
@@ -478,7 +477,6 @@ private CursorHolder makeCursorHolder(StorageAdapter sa, Filter filter)
         CursorBuildSpec.builder()
                        .setFilter(filter)
                        .setInterval(schemaInfo.getDataInterval())
-                       .setGranularity(Granularities.ALL)
                        .build()
     );
   }
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 e4596cb36c4e..c66ef6baf0a7 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
@@ -24,7 +24,6 @@
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.FileUtils;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.expression.LookupExprMacro;
@@ -383,7 +382,6 @@ public void baseSegmentWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter("countryIsoCode", "CA", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setFilter(filter)
                                                      .build();
     try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) {
@@ -411,7 +409,6 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setFilter(filter)
                                                      .build();
     try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
@@ -440,7 +437,6 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setFilter(filter)
                                                      .build();
     try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
@@ -469,7 +465,6 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setFilter(filter)
                                                      .build();
     try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
@@ -498,7 +493,6 @@ public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setFilter(filter)
                                                      .build();
     try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
@@ -515,7 +509,6 @@ public void lookupVirtualColumnStringKey(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setVirtualColumns(lookupVirtualColumns)
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
                                                                          .makeCursorHolder(buildSpec)) {
@@ -531,7 +524,6 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_CODE_TO_NAME, "Canada", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setFilter(filter)
                                                      .setVirtualColumns(lookupVirtualColumns)
                                                      .build();
@@ -549,7 +541,6 @@ public void lookupVirtualColumnLongKey(Blackhole blackhole)
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setVirtualColumns(lookupVirtualColumns)
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -564,7 +555,6 @@ public void lookupVirtualColumnLongKeyWithFilter(Blackhole blackhole)
   {
     final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_NUMBER_TO_NAME, "Canada", null).toFilter();
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(lookupVirtualColumns)
                                                      .setFilter(filter)
                                                      .build();
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java
index f71271dd1ebc..6f0796f412fd 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java
@@ -20,7 +20,6 @@
 package org.apache.druid.benchmark.indexing;
 
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.js.JavaScriptConfig;
 import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
@@ -208,7 +207,6 @@ public void readWithFilters(Blackhole blackhole)
   private CursorHolder makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter)
   {
     CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder()
-                                                                    .setGranularity(Granularities.ALL)
                                                                     .setInterval(schemaInfo.getDataInterval());
     if (filter != null) {
       builder.setFilter(filter.toFilter());
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
index a006d76727a2..d1ceccc96a35 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
@@ -33,7 +33,6 @@
 import org.apache.druid.indexer.JobHelper;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.guava.Yielder;
@@ -218,7 +217,6 @@ public Sequence apply(WindowedStorageAdapter adapter)
                   final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                                    .setFilter(Filters.toFilter(dimFilter))
                                                                    .setInterval(adapter.getInterval())
-                                                                   .setGranularity(Granularities.ALL)
                                                                    .build();
                   final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec);
                   final Cursor cursor = cursorHolder.asCursor();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java
index 91493fc9520d..a85fa73349df 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java
@@ -34,7 +34,6 @@
 import org.apache.druid.data.input.impl.MapInputRowParser;
 import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.java.util.common.CloseableIterators;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.guava.Yielder;
@@ -128,7 +127,6 @@ protected CloseableIterator> intermediateRowIterator() throw
     final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder()
                                                            .setFilter(Filters.toFilter(dimFilter))
                                                            .setInterval(storageAdapter.getInterval())
-                                                           .setGranularity(Granularities.ALL)
                                                            .build();
 
     final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
index 8347048af4cc..279cd937a1e6 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
@@ -1703,7 +1703,6 @@ public void testRunWithSpatialDimensions() throws Exception
           segment.getInterval()
       );
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                       .setGranularity(Granularities.ALL)
                                                        .setInterval(segment.getInterval())
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) {
@@ -1835,7 +1834,6 @@ public void testRunWithAutoCastDimensions() throws Exception
           segment.getInterval()
       );
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                       .setGranularity(Granularities.ALL)
                                                        .setInterval(segment.getInterval())
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) {
@@ -2064,7 +2062,6 @@ private List getCSVFormatRowsFromSegments(List segments) th
           segment.getInterval()
       );
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                       .setGranularity(Granularities.ALL)
                                                        .setInterval(segment.getInterval())
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) {
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
index 740aec8f8b26..edb1d68c012b 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
@@ -541,7 +541,6 @@ public void testTransformSpec() throws Exception
         segment.getInterval()
     );
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setInterval(segment.getInterval())
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) {
@@ -775,7 +774,6 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception
       );
 
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                       .setGranularity(Granularities.ALL)
                                                        .setInterval(segment.getInterval())
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) {
diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java
index 5faa0cef5b8a..042668884e9d 100644
--- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java
+++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java
@@ -32,7 +32,6 @@
 import org.apache.druid.frame.segment.FrameStorageAdapter;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.VirtualColumns;
@@ -116,7 +115,6 @@ public static FrameCursor makeCursor(
   )
   {
     final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder()
-                                                           .setGranularity(Granularities.ALL)
                                                            .setVirtualColumns(virtualColumns)
                                                            .build();
     // Despite appearances of columnar FrameCursorMakerFactory with its closers, it is currently safe to never close
diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java
index cbc53dd7e805..c0e5194a0597 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java
@@ -26,8 +26,6 @@
 import org.apache.druid.frame.segment.FrameCursorUtils;
 import org.apache.druid.frame.segment.FrameFilteredOffset;
 import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.UOE;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.vector.VectorValueMatcher;
@@ -40,7 +38,6 @@
 import org.apache.druid.segment.SimpleAscendingOffset;
 import org.apache.druid.segment.SimpleDescendingOffset;
 import org.apache.druid.segment.SimpleSettableOffset;
-import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.vector.FilteredVectorOffset;
 import org.apache.druid.segment.vector.NoFilterVectorOffset;
@@ -49,7 +46,6 @@
 import org.apache.druid.segment.vector.VectorCursor;
 import org.apache.druid.segment.vector.VectorOffset;
 import org.apache.druid.utils.CloseableUtils;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.util.List;
@@ -96,22 +92,27 @@ public boolean canVectorize()
       public Cursor asCursor()
       {
         final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders);
+        final ColumnCache columnCache = new ColumnCache(index, closer);
+        final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval());
+        final SimpleSettableOffset baseOffset = spec.isDescending()
+                                                ? new SimpleDescendingOffset(frame.numRows())
+                                                : new SimpleAscendingOffset(frame.numRows());
+
+        final QueryableIndexColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory(
+                spec.getVirtualColumns(),
+                spec.isDescending(),
+                baseOffset,
+                columnCache
+        );
 
-        if (Granularities.ALL.equals(spec.getGranularity())) {
-          final Cursor cursor = makeGranularityAllCursor(
-              new ColumnCache(index, closer),
-              frame.numRows(),
-              spec.getFilter(),
-              spec.getInterval(),
-              spec.getVirtualColumns(),
-              spec.isDescending()
-          );
-
-          return cursor;
+        final SimpleSettableOffset offset;
+        if (filterToUse == null) {
+          offset = baseOffset;
         } else {
-          // Not currently needed for the intended use cases of frame-based cursors.
-          throw new UOE("Granularity [%s] not supported", spec.getGranularity());
+          offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse);
         }
+
+        return new FrameCursor(offset, columnSelectorFactory);
       }
 
       @Nullable
@@ -119,7 +120,7 @@ public Cursor asCursor()
       public VectorCursor asVectorCursor()
       {
         if (!canVectorize()) {
-          throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'.");
+          throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'asVectorCursor'.");
         }
 
         final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders);
@@ -131,8 +132,8 @@ public VectorCursor asVectorCursor()
         );
         final ColumnCache columnCache = new ColumnCache(index, closer);
 
-        // baseColumnSelectorFactory using baseOffset is the column selector for filtering.
-        final VectorColumnSelectorFactory baseColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory(
+        // baseSelectorFactory using baseOffset is the column selector for filtering.
+        final VectorColumnSelectorFactory baseSelectorFactory = new QueryableIndexVectorColumnSelectorFactory(
             index,
             baseOffset,
             columnCache,
@@ -140,22 +141,22 @@ public VectorCursor asVectorCursor()
         );
 
         if (filterToUse == null) {
-          return new FrameVectorCursor(baseOffset, baseColumnSelectorFactory);
+          return new FrameVectorCursor(baseOffset, baseSelectorFactory);
         } else {
-          final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseColumnSelectorFactory);
+          final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseSelectorFactory);
           final FilteredVectorOffset filteredOffset = FilteredVectorOffset.create(
               baseOffset,
               matcher
           );
 
-          final VectorColumnSelectorFactory filteredColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory(
+          final VectorColumnSelectorFactory filteredSelectorFactory = new QueryableIndexVectorColumnSelectorFactory(
               index,
               filteredOffset,
               columnCache,
               spec.getVirtualColumns()
           );
 
-          return new FrameVectorCursor(filteredOffset, filteredColumnSelectorFactory);
+          return new FrameVectorCursor(filteredOffset, filteredSelectorFactory);
         }
       }
 
@@ -166,30 +167,4 @@ public void close()
       }
     };
   }
-
-  private static Cursor makeGranularityAllCursor(
-      final ColumnCache columnSelector,
-      final int numRows,
-      @Nullable final Filter filter,
-      final Interval interval,
-      final VirtualColumns virtualColumns,
-      final boolean descending
-  )
-  {
-    final Filter filterToUse = FrameCursorUtils.buildFilter(filter, interval);
-    final SimpleSettableOffset baseOffset =
-        descending ? new SimpleDescendingOffset(numRows) : new SimpleAscendingOffset(numRows);
-    final SimpleSettableOffset offset;
-
-    final QueryableIndexColumnSelectorFactory columnSelectorFactory =
-        new QueryableIndexColumnSelectorFactory(virtualColumns, descending, baseOffset, columnSelector);
-
-    if (filterToUse == null) {
-      offset = baseOffset;
-    } else {
-      offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse);
-    }
-
-    return new FrameCursor(offset, columnSelectorFactory);
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java
index 65426ddaec0d..f9948c00749e 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java
@@ -26,10 +26,9 @@
 import org.apache.druid.frame.segment.FrameCursor;
 import org.apache.druid.frame.segment.FrameCursorUtils;
 import org.apache.druid.frame.segment.FrameFilteredOffset;
-import org.apache.druid.java.util.common.UOE;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.CursorMakerFactory;
@@ -37,6 +36,7 @@
 import org.apache.druid.segment.SimpleDescendingOffset;
 import org.apache.druid.segment.SimpleSettableOffset;
 
+import javax.annotation.Nullable;
 import java.util.List;
 
 /**
@@ -66,40 +66,42 @@ public FrameCursorMakerFactory(
   @Override
   public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
-    if (!Granularities.ALL.equals(spec.getGranularity())) {
-      // Not currently needed for the intended use cases of frame-based cursors.
-      throw new UOE("Granularity [%s] not supported", spec.getGranularity());
-    }
-    return () -> {
-      final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval());
+    return new CursorHolder()
+    {
+      @Nullable
+      @Override
+      public Cursor asCursor()
+      {
+        final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval());
 
-      final SimpleSettableOffset baseOffset = spec.isDescending()
-                                              ? new SimpleDescendingOffset(frame.numRows())
-                                              : new SimpleAscendingOffset(frame.numRows());
+        final SimpleSettableOffset baseOffset = spec.isDescending()
+                                                ? new SimpleDescendingOffset(frame.numRows())
+                                                : new SimpleAscendingOffset(frame.numRows());
 
-      final SimpleSettableOffset offset;
 
-      final ColumnSelectorFactory columnSelectorFactory =
-          spec.getVirtualColumns().wrap(
-              new FrameColumnSelectorFactory(
-                  frame,
-                  frameReader.signature(),
-                  fieldReaders,
-                  new CursorFrameRowPointer(frame, baseOffset)
-              )
-          );
+        final ColumnSelectorFactory columnSelectorFactory =
+            spec.getVirtualColumns().wrap(
+                new FrameColumnSelectorFactory(
+                    frame,
+                    frameReader.signature(),
+                    fieldReaders,
+                    new CursorFrameRowPointer(frame, baseOffset)
+                )
+            );
 
-      if (filterToUse == null) {
-        offset = baseOffset;
-      } else {
-        offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse);
-      }
+        final SimpleSettableOffset offset;
+        if (filterToUse == null) {
+          offset = baseOffset;
+        } else {
+          offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse);
+        }
 
-      final FrameCursor cursor = new FrameCursor(offset, columnSelectorFactory);
+        final FrameCursor cursor = new FrameCursor(offset, columnSelectorFactory);
 
-      // Note: if anything closeable is ever added to this Sequence, make sure to update FrameProcessors.makeCursor.
-      // Currently, it assumes that closing the Sequence does nothing.
-      return cursor;
+        // Note: if anything closeable is ever added to this Sequence, make sure to update FrameProcessors.makeCursor.
+        // Currently, it assumes that closing the Sequence does nothing.
+        return cursor;
+      }
     };
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
index 303f835d889e..084c87b33830 100644
--- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
+++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
@@ -19,6 +19,15 @@
 
 package org.apache.druid.java.util.common.granularity;
 
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.joda.time.chrono.ISOChronology;
+
+import javax.annotation.Nullable;
+
 /**
  * This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
  * See: #2979, #3979
@@ -42,8 +51,44 @@ public class Granularities
   public static final Granularity ALL = GranularityType.ALL.getDefaultGranularity();
   public static final Granularity NONE = GranularityType.NONE.getDefaultGranularity();
 
+  public static final String GRANULARITY_VIRTUAL_COLUMN_NAME = "__virtualGranularity";
+
   public static Granularity nullToAll(Granularity granularity)
   {
     return granularity == null ? Granularities.ALL : granularity;
   }
+
+  /**
+   * Translates a {@link Granularity} to a {@link ExpressionVirtualColumn} on {@link ColumnHolder#TIME_COLUMN_NAME} of
+   * the equivalent grouping column. If granularity is {@link #ALL}, this method returns null since we are not grouping
+   * on time. If granularity is a {@link PeriodGranularity} with UTC timezone and no origin, this method returns a
+   * virtual column with {@link TimestampFloorExprMacro.TimestampFloorExpr} of the specified period. If granularity is
+   * {@link #NONE}, or any other kind of granularity (duration, period with non-utc timezone or origin) this method
+   * returns a virtual column with {@link org.apache.druid.math.expr.IdentifierExpr} specifying
+   * {@link ColumnHolder#TIME_COLUMN_NAME} directly.
+   */
+  @Nullable
+  public static ExpressionVirtualColumn toVirtualColumn(Granularity granularity)
+  {
+    if (ALL.equals(granularity)) {
+      return null;
+    }
+    final String expression;
+    if (NONE.equals(granularity) || granularity instanceof DurationGranularity) {
+      expression = ColumnHolder.TIME_COLUMN_NAME;
+    } else {
+      PeriodGranularity period = (PeriodGranularity) granularity;
+      if (!ISOChronology.getInstanceUTC().getZone().equals(period.getTimeZone()) || period.getOrigin() != null) {
+        expression = ColumnHolder.TIME_COLUMN_NAME;
+      } else {
+        expression = TimestampFloorExprMacro.forQueryGranularity(period.getPeriod());
+      }
+    }
+    return new ExpressionVirtualColumn(
+        GRANULARITY_VIRTUAL_COLUMN_NAME,
+        expression,
+        ColumnType.LONG,
+        ExprMacroTable.granularity()
+    );
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java b/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java
index a9f5b941c78f..34bf16538d81 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java
@@ -27,6 +27,7 @@
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.vector.ExprVectorProcessor;
 import org.apache.druid.math.expr.vector.FallbackVectorProcessor;
+import org.apache.druid.query.expression.TimestampFloorExprMacro;
 
 import javax.annotation.Nullable;
 import java.util.Collections;
@@ -54,6 +55,10 @@ public class ExprMacroTable
   );
   private static final ExprMacroTable NIL = new ExprMacroTable(Collections.emptyList());
 
+  private static final ExprMacroTable TIME_FLOOR_MACRO_TABLE = new ExprMacroTable(
+      Collections.singletonList(new TimestampFloorExprMacro())
+  );
+
   private final Map macroMap;
 
   public ExprMacroTable(final List macros)
@@ -68,6 +73,11 @@ public static ExprMacroTable nil()
     return NIL;
   }
 
+  public static ExprMacroTable granularity()
+  {
+    return TIME_FLOOR_MACRO_TABLE;
+  }
+
   public List getMacros()
   {
     return ImmutableList.copyOf(macroMap.values());
diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java
index 1a75d45de2ed..c2a5ffd9e612 100644
--- a/processing/src/main/java/org/apache/druid/query/Query.java
+++ b/processing/src/main/java/org/apache/druid/query/Query.java
@@ -287,7 +287,6 @@ default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics
     );
     return CursorBuildSpec.builder()
                           .setInterval(interval)
-                          .setGranularity(getGranularity())
                           .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())))
                           .setVirtualColumns(getVirtualColumns())
                           .setQueryContext(context())
diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
index 02eed7327f17..5cd2c88fc803 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
@@ -28,6 +28,8 @@
 import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor;
 import org.apache.druid.math.expr.vector.ExprVectorProcessor;
 import org.apache.druid.math.expr.vector.LongOutLongInFunctionVectorValueProcessor;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.joda.time.Period;
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
@@ -36,8 +38,28 @@
 
 public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
 {
+  public static String forQueryGranularity(Period period)
+  {
+    return FN_NAME + "(" + ColumnHolder.TIME_COLUMN_NAME + ",'" + period + "')";
+  }
+
   private static final String FN_NAME = "timestamp_floor";
 
+  private static PeriodGranularity computeGranularity(
+      final Expr expr,
+      final List args,
+      final Expr.ObjectBinding bindings
+  )
+  {
+    return ExprUtils.toPeriodGranularity(
+        expr,
+        args.get(1),
+        args.size() > 2 ? args.get(2) : null,
+        args.size() > 3 ? args.get(3) : null,
+        bindings
+    );
+  }
+
   @Override
   public String name()
   {
@@ -56,21 +78,6 @@ public Expr apply(final List args)
     }
   }
 
-  private static PeriodGranularity computeGranularity(
-      final Expr expr,
-      final List args,
-      final Expr.ObjectBinding bindings
-  )
-  {
-    return ExprUtils.toPeriodGranularity(
-        expr,
-        args.get(1),
-        args.size() > 2 ? args.get(2) : null,
-        args.size() > 3 ? args.get(3) : null,
-        bindings
-    );
-  }
-
   public static class TimestampFloorExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
   {
     private final PeriodGranularity granularity;
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 a758b8560e55..6d4b2f1643cc 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
@@ -28,7 +28,6 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
 import com.google.common.primitives.Longs;
@@ -821,10 +820,8 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics)
     );
     return CursorBuildSpec.builder()
                           .setInterval(interval)
-                          .setGranularity(getGranularity())
                           .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())))
-                          .setGroupingColumns(groupingColumns)
-                          .setVirtualColumns(getVirtualColumns())
+                          .setGroupingAndVirtualColumns(getGranularity(), groupingColumns, virtualColumns)
                           .setAggregators(getAggregatorSpecs())
                           .setQueryContext(context())
                           .isDescending(isDescending())
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java
index 3cc503ae8f6f..b3c1773c76c1 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java
@@ -112,7 +112,7 @@ public static Sequence process(
     final CursorGranularizer granularizer = CursorGranularizer.create(
         storageAdapter,
         cursor,
-        buildSpec.getGranularity(),
+        query.getGranularity(),
         buildSpec.getInterval(),
         buildSpec.isDescending()
     );
diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
index be8d5627c9fe..2e5283344392 100644
--- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
+++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
@@ -23,7 +23,6 @@
 import com.google.common.base.Preconditions;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
@@ -277,7 +276,6 @@ private ColumnAnalysis analyzeStringColumn(
       final DateTime end = storageAdapter.getMaxTime();
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                        .setInterval(new Interval(start, end))
-                                                       .setGranularity(Granularities.ALL)
                                                        .build();
       try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) {
         final Cursor cursor = cursorHolder.asCursor();
diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
index f6e265664c0e..895d81e67377 100644
--- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
+++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
@@ -30,7 +30,6 @@
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.UOE;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.query.operator.ColumnWithDirection;
@@ -220,8 +219,7 @@ private Pair materializeStorageAdapter(StorageAdapter as)
       }
     }
     final CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder()
-                                                                          .setFilter(filter)
-                                                                          .setGranularity(Granularities.ALL);
+                                                                          .setFilter(filter);
     if (interval != null) {
       builder.setInterval(interval);
     }
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java
index 2caf1771a3d6..be616cc28027 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java
@@ -508,7 +508,6 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics)
     );
     return CursorBuildSpec.builder()
                           .setInterval(interval)
-                          .setGranularity(getGranularity())
                           .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())))
                           .setVirtualColumns(getVirtualColumns())
                           .setQueryContext(context())
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java
index ddb74d6c9d71..b6535fcc5252 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java
@@ -25,7 +25,6 @@
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import org.apache.commons.lang.StringUtils;
 import org.apache.druid.error.DruidException;
 import org.apache.druid.java.util.common.granularity.Granularity;
@@ -215,9 +214,8 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics)
     );
     return CursorBuildSpec.builder()
                           .setInterval(interval)
-                          .setGranularity(getGranularity())
                           .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())))
-                          .setVirtualColumns(getVirtualColumns())
+                          .setGroupingAndVirtualColumns(getGranularity(), null, virtualColumns)
                           .setAggregators(getAggregatorSpecs())
                           .setQueryContext(context())
                           .isDescending(isDescending())
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java
index f58a0b0b8d75..0990b6dbb48d 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java
@@ -24,7 +24,6 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import org.apache.druid.error.DruidException;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.query.BaseQuery;
@@ -197,10 +196,12 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics)
     );
     return CursorBuildSpec.builder()
                           .setInterval(interval)
-                          .setGranularity(getGranularity())
                           .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter())))
-                          .setGroupingColumns(Collections.singletonList(dimensionSpec.getDimension()))
-                          .setVirtualColumns(getVirtualColumns())
+                          .setGroupingAndVirtualColumns(
+                              getGranularity(),
+                              Collections.singletonList(dimensionSpec.getDimension()),
+                              virtualColumns
+                          )
                           .setAggregators(getAggregatorSpecs())
                           .setQueryContext(context())
                           .isDescending(isDescending())
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
index ba695eeae668..79ca63d0daec 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
@@ -81,7 +81,7 @@ public Sequence> query(
     final CursorGranularizer granularizer = CursorGranularizer.create(
         adapter,
         cursor,
-        buildSpec.getGranularity(),
+        query.getGranularity(),
         buildSpec.getInterval(),
         buildSpec.isDescending()
     );
diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
index f26569fde392..a0fd6cd27874 100644
--- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
+++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
@@ -19,6 +19,8 @@
 
 package org.apache.druid.segment;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.Granularity;
@@ -29,11 +31,13 @@
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 public class CursorBuildSpec
 {
-  public static final CursorBuildSpec FULL_SCAN = CursorBuildSpec.builder().setGranularity(Granularities.ALL).build();
+  public static final CursorBuildSpec FULL_SCAN = CursorBuildSpec.builder().build();
 
   public static CursorBuildSpecBuilder builder()
   {
@@ -48,7 +52,6 @@ public static CursorBuildSpecBuilder builder(CursorBuildSpec spec)
   @Nullable
   private final Filter filter;
   private final Interval interval;
-  private final Granularity granularity;
   @Nullable
   private final List groupingColumns;
   private final VirtualColumns virtualColumns;
@@ -64,7 +67,6 @@ public static CursorBuildSpecBuilder builder(CursorBuildSpec spec)
   public CursorBuildSpec(
       @Nullable Filter filter,
       Interval interval,
-      Granularity granularity,
       @Nullable List groupingColumns,
       VirtualColumns virtualColumns,
       @Nullable List aggregators,
@@ -75,7 +77,6 @@ public CursorBuildSpec(
   {
     this.filter = filter;
     this.interval = interval;
-    this.granularity = granularity;
     this.groupingColumns = groupingColumns;
     this.virtualColumns = virtualColumns;
     this.aggregators = aggregators;
@@ -95,11 +96,6 @@ public Interval getInterval()
     return interval;
   }
 
-  public Granularity getGranularity()
-  {
-    return granularity;
-  }
-
   @Nullable
   public List getGroupingColumns()
   {
@@ -138,7 +134,6 @@ public static class CursorBuildSpecBuilder
     @Nullable
     private Filter filter;
     private Interval interval = Intervals.ETERNITY;
-    private Granularity granularity = Granularities.NONE;
 
     @Nullable
     private List groupingColumns;
@@ -160,7 +155,6 @@ private CursorBuildSpecBuilder(CursorBuildSpec buildSpec)
     {
       this.filter = buildSpec.filter;
       this.interval = buildSpec.interval;
-      this.granularity = buildSpec.granularity;
       this.groupingColumns = buildSpec.groupingColumns;
       this.virtualColumns = buildSpec.virtualColumns;
       this.aggregators = buildSpec.aggregators;
@@ -181,15 +175,30 @@ public CursorBuildSpecBuilder setInterval(Interval interval)
       return this;
     }
 
-    public CursorBuildSpecBuilder setGranularity(Granularity granularity)
-    {
-      this.granularity = granularity;
-      return this;
-    }
-
-    public CursorBuildSpecBuilder setGroupingColumns(@Nullable List columns)
+    public CursorBuildSpecBuilder setGroupingAndVirtualColumns(
+        Granularity granularity,
+        @Nullable List groupingColumns,
+        VirtualColumns virtualColumns
+    )
     {
-      this.groupingColumns = columns;
+      final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(granularity);
+      if (granularityVirtual == null) {
+        this.virtualColumns = virtualColumns;
+        this.groupingColumns = groupingColumns;
+      } else {
+        this.virtualColumns = VirtualColumns.fromIterable(
+            Iterables.concat(
+                Collections.singletonList(granularityVirtual),
+                () -> Arrays.stream(virtualColumns.getVirtualColumns()).iterator()
+            )
+        );
+        ImmutableList.Builder bob = ImmutableList.builder()
+                                                         .add(granularityVirtual.getOutputName());
+        if (groupingColumns != null) {
+          bob.addAll(groupingColumns);
+        }
+        this.groupingColumns = bob.build();
+      }
       return this;
     }
 
@@ -228,7 +237,6 @@ public CursorBuildSpec build()
       return new CursorBuildSpec(
           filter,
           interval,
-          granularity,
           groupingColumns,
           virtualColumns,
           aggregators,
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
index 8553caa0c3f5..e35bb943c92a 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
@@ -24,7 +24,6 @@
 import com.google.common.base.Suppliers;
 import org.apache.druid.collections.bitmap.BitmapFactory;
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.BaseQuery;
@@ -66,8 +65,6 @@ public class QueryableIndexCursorHolder implements CursorHolder
   private static final Logger log = new Logger(QueryableIndexCursorHolder.class);
   private final QueryableIndex index;
   private final Interval interval;
-  @SuppressWarnings("unused")
-  private final Granularity gran;
   private final VirtualColumns virtualColumns;
 
   @Nullable
@@ -88,7 +85,6 @@ public QueryableIndexCursorHolder(
   {
     this.index = index;
     this.interval = cursorBuildSpec.getInterval();
-    this.gran = cursorBuildSpec.getGranularity();
     this.virtualColumns = cursorBuildSpec.getVirtualColumns();
     this.aggregatorFactories = cursorBuildSpec.getAggregators();
     this.filter = cursorBuildSpec.getFilter();
@@ -689,7 +685,7 @@ private FilterBundle makeFilterBundle(
       final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory();
       final BitmapResultFactory bitmapResultFactory;
       if (metrics != null) {
-        bitmapResultFactory =metrics.makeBitmapResultFactory(bitmapFactory);
+        bitmapResultFactory = metrics.makeBitmapResultFactory(bitmapFactory);
         metrics.reportSegmentRows(numRows);
       } else {
         bitmapResultFactory = new DefaultBitmapResultFactory(bitmapFactory);
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
index de16b37d461d..4dbe532fb013 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
@@ -20,7 +20,6 @@
 package org.apache.druid.segment;
 
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.segment.column.BaseColumn;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ColumnHolder;
@@ -173,15 +172,9 @@ public DateTime getMaxIngestedEventTime()
   @Override
   public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
-    final Interval actualInterval = computeCursorInterval(spec.getGranularity(), spec.getInterval());
-
-    if (actualInterval == null) {
-      return CursorHolder.EMPTY;
-    }
-
     return new QueryableIndexCursorHolder(
         index,
-        CursorBuildSpec.builder(spec).setInterval(actualInterval).build()
+        CursorBuildSpec.builder(spec).setInterval(spec.getInterval()).build()
     );
   }
 
@@ -200,18 +193,4 @@ private void populateMinMaxTime()
       this.maxTime = DateTimes.utc(column.getLongSingleValueRow(column.length() - 1));
     }
   }
-
-  @Nullable
-  private Interval computeCursorInterval(final Granularity gran, final Interval interval)
-  {
-    final DateTime minTime = getMinTime();
-    final DateTime maxTime = getMaxTime();
-    final Interval dataInterval = new Interval(minTime, gran.bucketEnd(maxTime));
-
-    if (!interval.overlaps(dataInterval)) {
-      return null;
-    }
-
-    return interval.overlap(dataInterval);
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
index 4e880c4e4ce6..2b710d01db0c 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
@@ -21,7 +21,6 @@
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.Granularity;
@@ -176,32 +175,17 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       @Override
       public Cursor asCursor()
       {
-        final Granularity gran = spec.getGranularity();
-        final Interval actualInterval = spec.getInterval()
-                                            .overlap(new Interval(getMinTime(), gran.bucketEnd(getMaxTime())));
-
-        if (actualInterval == null) {
-          return null;
-        }
-
-        if (!isQueryGranularityAllowed(actualInterval, gran)) {
-          throw new IAE(
-              "Cannot support interval [%s] with granularity [%s]",
-              Intervals.ETERNITY.equals(actualInterval) ? "ETERNITY" : actualInterval,
-              gran
-          );
-        }
-
-        final RowWalker rowWalker = new RowWalker<>(
-            spec.isDescending() ? reverse(rowSequence) : rowSequence,
-            rowAdapter
+        final RowWalker rowWalker = closer.register(
+            new RowWalker<>(
+                spec.isDescending() ? reverse(rowSequence) : rowSequence,
+                rowAdapter
+            )
         );
-        closer.register(rowWalker);
         return new RowBasedCursor<>(
             rowWalker,
             rowAdapter,
             spec.getFilter(),
-            actualInterval,
+            spec.getInterval(),
             spec.getVirtualColumns(),
             spec.isDescending(),
             rowSignature
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
index efcfed15f268..92be2553ddb4 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
@@ -39,17 +39,17 @@ public class IncrementalIndexCursorHolder implements CursorHolder
 {
   private final IncrementalIndexStorageAdapter storageAdapter;
   private final IncrementalIndex index;
-  private final CursorBuildSpec builder;
+  private final CursorBuildSpec spec;
 
   public IncrementalIndexCursorHolder(
       IncrementalIndexStorageAdapter storageAdapter,
       IncrementalIndex index,
-      CursorBuildSpec builder
+      CursorBuildSpec spec
   )
   {
     this.storageAdapter = storageAdapter;
     this.index = index;
-    this.builder = builder;
+    this.spec = spec;
   }
 
   @Override
@@ -59,27 +59,18 @@ public Cursor asCursor()
       return null;
     }
 
-    if (builder.getQueryMetrics() != null) {
-      builder.getQueryMetrics().vectorized(false);
+    if (spec.getQueryMetrics() != null) {
+      spec.getQueryMetrics().vectorized(false);
     }
 
-    final Interval dataInterval = new Interval(
-        index.getMinTime(),
-        builder.getGranularity().bucketEnd(index.getMaxTime())
-    );
-
-    if (!builder.getInterval().overlaps(dataInterval)) {
-      return null;
-    }
-    final Interval actualInterval = builder.getInterval().overlap(dataInterval);
 
     return new IncrementalIndexCursor(
         storageAdapter,
         index,
-        builder.getVirtualColumns(),
-        builder.isDescending(),
-        builder.getFilter(),
-        actualInterval
+        spec.getVirtualColumns(),
+        spec.isDescending(),
+        spec.getFilter(),
+        spec.getInterval()
     );
   }
 
diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
index 3eae756f2bc2..684bac294c15 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
@@ -221,7 +221,6 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
     final CursorBuildSpec.CursorBuildSpecBuilder cursorBuildSpecBuilder =
         CursorBuildSpec.builder()
                        .setInterval(spec.getInterval())
-                       .setGranularity(spec.getGranularity())
                        .isDescending(spec.isDescending())
                        .setQueryMetrics(spec.getQueryMetrics());
 
diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
index 59ffbacca99b..8807dc5b4a7a 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
@@ -21,7 +21,6 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.cache.CacheKeyBuilder;
@@ -119,7 +118,6 @@ public BroadcastSegmentIndexedTable(
                                                          queryableIndex.getDataInterval()
                                                                        .withChronology(ISOChronology.getInstanceUTC())
                                                      )
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
index d8a3ddbc0e48..5d6af7eb67b6 100644
--- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
@@ -26,7 +26,6 @@
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.QueryContext;
@@ -270,8 +269,6 @@ public CursorTests(
       this.buildSpec = CursorBuildSpec.builder()
                                       .setFilter(this.filter)
                                       .setInterval(this.interval)
-                                      // Frames only support Granularities.ALL: no point testing the others.
-                                      .setGranularity(Granularities.ALL)
                                       .setVirtualColumns(this.virtualColumns)
                                       .isDescending(this.descending)
                                       .setQueryContext(queryContext)
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 b876af20c4d8..d3a2ce42832a 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
@@ -33,7 +33,6 @@
 import org.apache.druid.frame.segment.FrameStorageAdapter;
 import org.apache.druid.frame.util.SettableLongVirtualColumn;
 import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
@@ -294,7 +293,6 @@ public static CursorHolder makeCursorForAdapter(
     }
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(virtualColumns)
                                                      .build();
 
diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
index dbb952fbf1cf..16fa189e1893 100644
--- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
+++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
@@ -33,6 +33,8 @@
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.granularity.GranularityType;
 import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Days;
@@ -1021,6 +1023,39 @@ public void testTruncateDhaka()
     );
   }
 
+  @Test
+  public void testToVirtualColumn()
+  {
+    final Granularity hour = new PeriodGranularity(new Period("PT1H"), null, null);
+    final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
+    final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
+    final Granularity hourWithOrigin = new PeriodGranularity(new Period("PT1H"), origin, tz);
+    final PeriodGranularity hourWithTz = new PeriodGranularity(new Period("PT1H"), null, tz);
+    final Granularity duration = new DurationGranularity(
+        new Period("PT12H5M").toStandardDuration().getMillis(),
+        origin
+    );
+
+    ExpressionVirtualColumn column = Granularities.toVirtualColumn(hour);
+    Assert.assertEquals("timestamp_floor(__time,'PT1H')", column.getExpression());
+    column = Granularities.toVirtualColumn(hourWithOrigin);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
+    column = Granularities.toVirtualColumn(hourWithTz);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
+    column = Granularities.toVirtualColumn(duration);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
+    column = Granularities.toVirtualColumn(Granularities.NONE);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
+    column = Granularities.toVirtualColumn(Granularities.ALL);
+    Assert.assertNull(column);
+    column = Granularities.toVirtualColumn(Granularities.HOUR);
+    Assert.assertEquals("timestamp_floor(__time,'PT1H')", column.getExpression());
+    column = Granularities.toVirtualColumn(Granularities.MINUTE);
+    Assert.assertEquals("timestamp_floor(__time,'PT1M')", column.getExpression());
+    column = Granularities.toVirtualColumn(Granularities.FIFTEEN_MINUTE);
+    Assert.assertEquals("timestamp_floor(__time,'PT15M')", column.getExpression());
+  }
+
   private void assertBucketStart(final Granularity granularity, final DateTime in, final DateTime expectedInProperTz)
   {
     Assert.assertEquals(
diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
index 723ea11a49ff..67939e1044f8 100644
--- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
@@ -203,7 +203,6 @@ public void testGranularizeFullScan()
   public void testGranularizeFullScanDescending()
   {
     final CursorBuildSpec descending = CursorBuildSpec.builder()
-                                                      .setGranularity(Granularities.ALL)
                                                       .isDescending(true)
                                                       .build();
     try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) {
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java
index 57a546550058..2de5c0b8d79b 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java
@@ -165,7 +165,7 @@ public void testSegmentLookUpForNestedQueries()
   }
 
   @Test
-  public void testAsCursorBuildSpec()
+  public void testAsCursorBuildSpecAllGranularity()
   {
     final VirtualColumns virtualColumns = VirtualColumns.create(
         new ExpressionVirtualColumn("v0", "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil())
@@ -186,7 +186,7 @@ public void testAsCursorBuildSpec()
         )
         .setVirtualColumns(virtualColumns)
         .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, longSum)
-        .setGranularity(QueryRunnerTestHelper.DAY_GRAN)
+        .setGranularity(Granularities.ALL)
         .setPostAggregatorSpecs(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
         .setLimitSpec(
             new DefaultLimitSpec(
@@ -202,13 +202,65 @@ public void testAsCursorBuildSpec()
 
     final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
     Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
-    Assert.assertEquals(Granularities.DAY, buildSpec.getGranularity());
     Assert.assertEquals(ImmutableList.of("quality", "market", "v0"), buildSpec.getGroupingColumns());
     Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
     Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
     Assert.assertEquals(query.isDescending(), buildSpec.isDescending());
   }
 
+  @Test
+  public void testAsCursorBuildSpecDayGranularity()
+  {
+    final VirtualColumns virtualColumns = VirtualColumns.create(
+        new ExpressionVirtualColumn("v0", "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil())
+    );
+    final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index");
+    Query query = GroupByQuery
+        .builder()
+        .setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
+        .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
+        .setDimensions(
+            new DefaultDimensionSpec(QueryRunnerTestHelper.QUALITY_DIMENSION, "alias"),
+            new DefaultDimensionSpec(
+                QueryRunnerTestHelper.MARKET_DIMENSION,
+                "market",
+                ColumnType.STRING_ARRAY
+            ),
+            new DefaultDimensionSpec("v0", "v0", ColumnType.STRING)
+        )
+        .setVirtualColumns(virtualColumns)
+        .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, longSum)
+        .setGranularity(Granularities.DAY)
+        .setPostAggregatorSpecs(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
+        .setLimitSpec(
+            new DefaultLimitSpec(
+                ImmutableList.of(new OrderByColumnSpec(
+                    "alias",
+                    OrderByColumnSpec.Direction.ASCENDING,
+                    StringComparators.LEXICOGRAPHIC
+                )),
+                100
+            )
+        )
+        .build();
+
+    final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
+    Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
+    Assert.assertEquals(
+        ImmutableList.of(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME, "quality", "market", "v0"),
+        buildSpec.getGroupingColumns()
+    );
+    Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
+    Assert.assertEquals(
+        VirtualColumns.create(
+            Granularities.toVirtualColumn(Granularities.DAY),
+            virtualColumns.getVirtualColumns()[0]
+        ),
+        buildSpec.getVirtualColumns()
+    );
+    Assert.assertEquals(query.isDescending(), buildSpec.isDescending());
+  }
+
   @Test
   public void testEquals()
   {
diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
index 0ab09ed95a41..5826301bcd7a 100644
--- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
+++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
@@ -24,7 +24,6 @@
 import com.google.common.collect.Lists;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.Pair;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.query.extraction.MapLookupExtractor;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
@@ -194,7 +193,6 @@ public void test_asStorageAdapter_makeCursor()
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("1970/PT1H"))
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asStorageAdapter().makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
index 1d707c68850d..023a6294ab9d 100644
--- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
@@ -24,7 +24,6 @@
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.UOE;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.InDimFilter;
 import org.apache.druid.query.groupby.ResultRow;
@@ -251,8 +250,7 @@ private void validateDecorated(
           siggy
       );
       final CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder()
-                                                                            .setFilter(filter)
-                                                                            .setGranularity(Granularities.ALL);
+                                                                            .setFilter(filter);
       if (interval != null) {
         builder.setInterval(interval);
       }
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java
index 694e29b4416e..b8e2815e97d6 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java
@@ -25,7 +25,6 @@
 import com.google.common.collect.ImmutableSet;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -444,7 +443,6 @@ public void testAsCursorBuildSpec()
 
     final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
     Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
-    Assert.assertEquals(Granularities.ALL, buildSpec.getGranularity());
     Assert.assertNull(buildSpec.getGroupingColumns());
     Assert.assertNull(buildSpec.getAggregators());
     Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
index 444863135d84..821b2955ee2e 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
@@ -41,6 +41,7 @@
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 
 @RunWith(Parameterized.class)
 public class TimeseriesQueryTest extends InitializedNullHandlingTest
@@ -108,7 +109,7 @@ public void testGetRequiredColumns()
   }
 
   @Test
-  public void testAsCursorBuildSpec()
+  public void testAsCursorBuildSpecAllGranularity()
   {
     final VirtualColumns virtualColumns = VirtualColumns.create(
         new ExpressionVirtualColumn(
@@ -122,7 +123,7 @@ public void testAsCursorBuildSpec()
     final TimeseriesQuery query =
         Druids.newTimeseriesQueryBuilder()
               .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
-              .granularity(QueryRunnerTestHelper.DAY_GRAN)
+              .granularity(Granularities.ALL)
               .virtualColumns(virtualColumns)
               .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
               .aggregators(
@@ -137,7 +138,6 @@ public void testAsCursorBuildSpec()
 
     final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
     Assert.assertEquals(QueryRunnerTestHelper.FULL_ON_INTERVAL, buildSpec.getInterval());
-    Assert.assertEquals(Granularities.DAY, buildSpec.getGranularity());
     Assert.assertNull(buildSpec.getGroupingColumns());
     Assert.assertEquals(
         ImmutableList.of(
@@ -151,4 +151,57 @@ public void testAsCursorBuildSpec()
     Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
     Assert.assertEquals(query.isDescending(), buildSpec.isDescending());
   }
+
+  @Test
+  public void testAsCursorBuildSpecDayGranularity()
+  {
+    final VirtualColumns virtualColumns = VirtualColumns.create(
+        new ExpressionVirtualColumn(
+            "index",
+            "\"fieldFromVirtualColumn\"",
+            ColumnType.LONG,
+            ExprMacroTable.nil()
+        )
+    );
+    final LongSumAggregatorFactory beep = new LongSumAggregatorFactory("beep", "aField");
+    final TimeseriesQuery query =
+        Druids.newTimeseriesQueryBuilder()
+              .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
+              .granularity(Granularities.DAY)
+              .virtualColumns(virtualColumns)
+              .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
+              .aggregators(
+                  QueryRunnerTestHelper.ROWS_COUNT,
+                  QueryRunnerTestHelper.INDEX_DOUBLE_SUM,
+                  QueryRunnerTestHelper.INDEX_LONG_MAX,
+                  beep
+              )
+              .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT)
+              .descending(descending)
+              .build();
+
+    final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
+    Assert.assertEquals(QueryRunnerTestHelper.FULL_ON_INTERVAL, buildSpec.getInterval());
+    Assert.assertEquals(
+        Collections.singletonList(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME),
+        buildSpec.getGroupingColumns()
+    );
+    Assert.assertEquals(
+        ImmutableList.of(
+            QueryRunnerTestHelper.ROWS_COUNT,
+            QueryRunnerTestHelper.INDEX_DOUBLE_SUM,
+            QueryRunnerTestHelper.INDEX_LONG_MAX,
+            beep
+        ),
+        buildSpec.getAggregators()
+    );
+    Assert.assertEquals(
+        VirtualColumns.create(
+            Granularities.toVirtualColumn(Granularities.DAY),
+            virtualColumns.getVirtualColumns()[0]
+        ),
+        buildSpec.getVirtualColumns()
+    );
+    Assert.assertEquals(query.isDescending(), buildSpec.isDescending());
+  }
 }
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
index 028f958ca5b7..2b15b932d479 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
@@ -272,7 +272,7 @@ public void testGetRequiredColumns()
   }
 
   @Test
-  public void testAsCursorBuildSpec()
+  public void testAsCursorBuildSpecAllGranularity()
   {
     final VirtualColumns virtualColumns = VirtualColumns.create(
         new ExpressionVirtualColumn("v", "\"other\"", ColumnType.STRING, ExprMacroTable.nil())
@@ -284,7 +284,7 @@ public void testAsCursorBuildSpec()
         .virtualColumns(virtualColumns)
         .dimension(DefaultDimensionSpec.of("v"))
         .aggregators(QueryRunnerTestHelper.ROWS_COUNT, longSum)
-        .granularity(QueryRunnerTestHelper.DAY_GRAN)
+        .granularity(Granularities.ALL)
         .postAggregators(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
         .metric(new NumericTopNMetricSpec("idx"))
         .threshold(100)
@@ -292,10 +292,45 @@ public void testAsCursorBuildSpec()
 
     final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
     Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
-    Assert.assertEquals(Granularities.DAY, buildSpec.getGranularity());
     Assert.assertEquals(ImmutableList.of("v"), buildSpec.getGroupingColumns());
     Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
     Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
     Assert.assertEquals(query.isDescending(), buildSpec.isDescending());
   }
+
+  @Test
+  public void testAsCursorBuildSpecDayGranularity()
+  {
+    final VirtualColumns virtualColumns = VirtualColumns.create(
+        new ExpressionVirtualColumn("v", "\"other\"", ColumnType.STRING, ExprMacroTable.nil())
+    );
+    final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index");
+    final TopNQuery query = new TopNQueryBuilder()
+        .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
+        .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD)
+        .virtualColumns(virtualColumns)
+        .dimension(DefaultDimensionSpec.of("v"))
+        .aggregators(QueryRunnerTestHelper.ROWS_COUNT, longSum)
+        .granularity(Granularities.DAY)
+        .postAggregators(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
+        .metric(new NumericTopNMetricSpec("idx"))
+        .threshold(100)
+        .build();
+
+    final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null);
+    Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
+    Assert.assertEquals(
+        ImmutableList.of(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME, "v"),
+        buildSpec.getGroupingColumns()
+    );
+    Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
+    Assert.assertEquals(
+        VirtualColumns.create(
+            Granularities.toVirtualColumn(Granularities.DAY),
+            virtualColumns.getVirtualColumns()[0]
+        ),
+        buildSpec.getVirtualColumns()
+    );
+    Assert.assertEquals(query.isDescending(), buildSpec.isDescending());
+  }
 }
diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
index 1376e48789bc..0b0e5b0c7730 100644
--- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
@@ -25,10 +25,10 @@
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.common.guava.GuavaUtils;
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -456,7 +456,6 @@ public void test_makeCursor_filterOnLong()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -480,7 +479,6 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNull()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -516,7 +514,6 @@ public void test_makeCursor_filterOnVirtualColumn()
                                                              )
                                                          )
                                                      )
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
 
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
@@ -537,7 +534,7 @@ public void test_makeCursor_descending()
   {
     final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2);
 
-    final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build();
+    final CursorBuildSpec buildSpec = CursorBuildSpec.builder().isDescending(true).build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
       Assert.assertEquals(
@@ -560,7 +557,6 @@ public void test_makeCursor_intervalDoesNotMatch()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("2000/P1D"))
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -580,7 +576,6 @@ public void test_makeCursor_intervalPartiallyMatches()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("1970-01-01T01/PT1H"))
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -602,7 +597,6 @@ public void test_makeCursor_hourGranularity()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("1970/1971"))
-                                                     .setGranularity(Granularities.HOUR)
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -614,7 +608,7 @@ public void test_makeCursor_hourGranularity()
               ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"),
               ImmutableList.of(DateTimes.of("1970-01-01T03"), "3")
           ),
-          walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN)
+          walkCursorGranularized(adapter, cursor, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN)
       );
     }
 
@@ -628,7 +622,6 @@ public void test_makeCursor_hourGranularityWithInterval()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("1970-01-01T01/PT2H"))
-                                                     .setGranularity(Granularities.HOUR)
                                                      .build();
 
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
@@ -640,7 +633,7 @@ public void test_makeCursor_hourGranularityWithInterval()
               ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"),
               ImmutableList.of(DateTimes.of("1970-01-01T02"), "2")
           ),
-          walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN)
+          walkCursorGranularized(adapter, cursor, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN)
       );
     }
 
@@ -654,7 +647,6 @@ public void test_makeCursor_hourGranularityWithIntervalDescending()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("1970-01-01T01/PT2H"))
-                                                     .setGranularity(Granularities.HOUR)
                                                      .isDescending(true)
                                                      .build();
 
@@ -666,7 +658,7 @@ public void test_makeCursor_hourGranularityWithIntervalDescending()
               ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"),
               ImmutableList.of(DateTimes.of("1970-01-01T01"), "1")
           ),
-          walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN)
+          walkCursorGranularized(adapter, cursor, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN)
       );
     }
 
@@ -785,7 +777,6 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
 
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
@@ -799,19 +790,6 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull()
     Assert.assertEquals(1, numCloses.get());
   }
 
-  @Test
-  public void test_makeCursor_eternityIntervalWithMonthGranularity()
-  {
-    final RowBasedStorageAdapter adapter = createIntAdapter(0, 1);
-    Assert.assertThrows(IAE.class, () -> {
-      adapter.makeCursorHolder(
-          CursorBuildSpec.builder()
-                         .setGranularity(Granularities.MONTH)
-                         .build()
-      ).asCursor();
-    });
-  }
-
   private static List> walkCursor(
       final Cursor cursor,
       final List>> processors
@@ -842,13 +820,14 @@ private static List> walkCursorGranularized(
       final StorageAdapter adapter,
       final Cursor cursor,
       final CursorBuildSpec buildSpec,
+      final Granularity granularity,
       final List>> processors
   )
   {
     CursorGranularizer granularizer = CursorGranularizer.create(
         adapter,
         cursor,
-        buildSpec.getGranularity(),
+        granularity,
         buildSpec.getInterval(),
         buildSpec.isDescending()
     );
diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
index 1e9246682990..229e337be523 100644
--- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
@@ -230,7 +230,6 @@ public void test_unnest_adapters_basic()
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(UNNEST_STORAGE_ADAPTER.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
@@ -260,7 +259,6 @@ public void test_two_levels_of_unnest_adapters()
   {
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
@@ -318,7 +316,6 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(baseFilter)
                                                      .setInterval(unnestStorageAdapter.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
@@ -365,7 +362,6 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(baseFilter)
                                                      .setInterval(unnestStorageAdapter.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
@@ -644,7 +640,6 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(unnestStorageAdapter.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
 
     try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
@@ -687,7 +682,6 @@ public void test_pushdown_filters_unnested_dimension_outside()
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(queryFilter)
                                                      .setInterval(unnestStorageAdapter.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
 
     try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) {
@@ -735,7 +729,6 @@ public void testUnnestValueMatcherValueDoesntExist()
     );
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(withNullsStorageAdapter.getInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
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 1c884f9900ca..0b03c52bd2b1 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
@@ -46,7 +46,6 @@
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprType;
 import org.apache.druid.math.expr.ExpressionType;
@@ -796,7 +795,6 @@ private CursorBuildSpec makeCursorBuildSpec(@Nullable Filter filter)
     return CursorBuildSpec.builder()
                           .setFilter(filter)
                           .setVirtualColumns(VIRTUAL_COLUMNS)
-                          .setGranularity(Granularities.ALL)
                           .build();
 
   }
@@ -806,7 +804,6 @@ private CursorBuildSpec makeVectorCursorBuildSpec(@Nullable Filter filter)
     return CursorBuildSpec.builder()
                           .setFilter(filter)
                           .setVirtualColumns(virtualColumns)
-                          .setGranularity(Granularities.ALL)
                           .setQueryContext(
                               QueryContext.of(
                                   ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3)
diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
index 8f284d5284ee..d1bb227afecc 100644
--- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
@@ -433,7 +433,6 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000))
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
@@ -491,7 +490,6 @@ public void testCursorDictionaryRaceConditionFix() throws Exception
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setFilter(new DictionaryRaceTestFilter(index, timestamp))
                                                      .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000))
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
@@ -531,7 +529,6 @@ public void testCursoringAndSnapshot() throws Exception
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000))
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
     try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) {
       Cursor cursor = cursorHolder.asCursor();
diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
index 4bef35618d8c..4ddbe36c68db 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
@@ -26,7 +26,6 @@
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.filter.ExpressionDimFilter;
 import org.apache.druid.query.filter.Filter;
@@ -526,7 +525,7 @@ public void test_makeCursor_factToCountryInnerUsingCountryNumber()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -586,7 +585,7 @@ public void test_makeCursor_factToCountryInnerUsingCountryNumberUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -642,7 +641,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnFacts()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -673,7 +672,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnFactsUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -703,7 +702,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -736,7 +735,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup(
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -768,7 +767,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -801,7 +800,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -838,7 +837,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnJoinable()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -875,7 +874,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnJoinableUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -912,7 +911,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnJoinable()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -949,7 +948,7 @@ public void test_makeCursor_factToCountryLeftWithFilterOnJoinableUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -999,7 +998,7 @@ public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditi
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1070,7 +1069,7 @@ public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditi
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1247,7 +1246,7 @@ public void test_makeCursor_factToCountryAlwaysTrue()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1306,7 +1305,7 @@ public void test_makeCursor_factToCountryAlwaysFalse()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1347,7 +1346,7 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1406,7 +1405,7 @@ public void test_makeCursor_factToCountryAlwaysFalseUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1451,7 +1450,7 @@ public void test_makeCursor_factToCountryUsingVirtualColumn()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build()
         ),
         ImmutableList.of(
             "page",
@@ -1502,7 +1501,7 @@ public void test_makeCursor_factToCountryUsingVirtualColumnUsingLookup()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build()
         ),
         ImmutableList.of(
             "page",
@@ -1642,7 +1641,7 @@ public void test_makeCursor_factToRegionTheWrongWay()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1819,7 +1818,7 @@ public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRows()
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1848,7 +1847,7 @@ public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRowsUsingLook
             joinableClauses,
             joinFilterPreAnalysis
         ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1876,7 +1875,7 @@ public void test_makeCursor_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE
         joinableClauses,
         joinFilterPreAnalysis
     ).makeCursorHolder(
-        CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+        CursorBuildSpec.builder().setFilter(filter).build()
     );
   }
 
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 30fa7110d30b..656cbdb2ba24 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
@@ -25,7 +25,6 @@
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.expression.TestExprMacroTable;
@@ -84,7 +83,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -156,7 +155,7 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -207,7 +206,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -269,7 +268,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -335,7 +334,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns(
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -403,7 +402,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC
             CursorBuildSpec.builder()
                            .setFilter(originalFilter)
                            .setVirtualColumns(virtualColumns)
-                           .setGranularity(Granularities.ALL)
                            .build()
         ),
         ImmutableList.of(
@@ -476,7 +474,6 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu
             CursorBuildSpec.builder()
                            .setFilter(originalFilter)
                            .setVirtualColumns(virtualColumns)
-                           .setGranularity(Granularities.ALL)
                            .build()
         ),
         ImmutableList.of(
@@ -576,7 +573,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -698,7 +695,7 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -787,7 +784,7 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan
     );
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -832,7 +829,7 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -929,7 +926,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1014,7 +1011,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1083,7 +1080,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1136,7 +1133,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1188,7 +1185,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1239,7 +1236,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1291,7 +1288,7 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1343,7 +1340,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan
     // is interpreted as 0 (a.k.a. Australia).
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1402,7 +1399,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan
     // is interpreted as 0 (a.k.a. Australia).
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1458,7 +1455,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1508,7 +1505,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1557,7 +1554,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1610,7 +1607,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(filter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1662,7 +1659,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1713,7 +1710,7 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1775,7 +1772,7 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1841,7 +1838,7 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1917,7 +1914,7 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -1973,7 +1970,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -2048,7 +2045,7 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -2246,7 +2243,7 @@ public boolean supportsRequiredColumnRewrite()
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -2375,7 +2372,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
@@ -2450,7 +2447,7 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName
 
     JoinTestHelper.verifyCursor(
         adapter.makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build()
+            CursorBuildSpec.builder().setFilter(originalFilter).build()
         ),
         ImmutableList.of(
             "page",
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 113e0d4e84be..322f9608a7b0 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
@@ -348,7 +348,6 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(storageAdapter.getInterval())
                                                      .setVirtualColumns(virtualColumns)
-                                                     .setGranularity(Granularities.DAY)
                                                      .build();
     final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec));
     final Cursor cursor = cursorHolder.asCursor();
diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
index fb0a2e9fe227..2f67dc454110 100644
--- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java
@@ -140,7 +140,6 @@ public void test_single_value_string_bindings()
     for (StorageAdapter adapter : ADAPTERS) {
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                        .setInterval(adapter.getInterval())
-                                                       .setGranularity(Granularities.ALL)
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
         Cursor cursor = cursorHolder.asCursor();
@@ -209,7 +208,6 @@ public void test_multi_value_string_bindings()
     for (StorageAdapter adapter : ADAPTERS) {
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                        .setInterval(adapter.getInterval())
-                                                       .setGranularity(Granularities.ALL)
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
         Cursor cursor = cursorHolder.asCursor();
@@ -290,7 +288,6 @@ public void test_long_bindings()
     for (StorageAdapter adapter : ADAPTERS) {
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                        .setInterval(adapter.getInterval())
-                                                       .setGranularity(Granularities.ALL)
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
         Cursor cursor = cursorHolder.asCursor();
@@ -339,7 +336,6 @@ public void test_double_bindings()
     for (StorageAdapter adapter : ADAPTERS) {
       final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                        .setInterval(adapter.getInterval())
-                                                       .setGranularity(Granularities.ALL)
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
         Cursor cursor = cursorHolder.asCursor();
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 ab7ddf19b58f..1214987c9c98 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
@@ -237,7 +237,6 @@ public static void sanityTestVectorizedExpressionSelectors(
     final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index);
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(index.getDataInterval())
-                                                     .setGranularity(Granularities.ALL)
                                                      .setVirtualColumns(virtualColumns)
                                                      .build();
     try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) {
diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java
index 60ba59912df9..3c509f65e0c2 100644
--- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java
+++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java
@@ -48,7 +48,6 @@
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.jackson.JacksonUtils;
@@ -306,7 +305,6 @@ public static void runDump(
                                                          index.getDataInterval()
                                                               .withChronology(ISOChronology.getInstanceUTC())
                                                      )
-                                                     .setGranularity(Granularities.ALL)
                                                      .build();
 
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {

From ae40775f8b5b6431fa2e3e2774566359d6f1e186 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Sat, 3 Aug 2024 11:16:58 -0700
Subject: [PATCH 32/74] fix build

---
 .../src/test/java/org/apache/druid/cli/DumpSegmentTest.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java
index 3ebb140815b0..0ca08523fb2a 100644
--- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java
+++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java
@@ -120,15 +120,15 @@ public void testDumpRows() throws Exception
   {
     Injector injector = Mockito.mock(Injector.class);
     ObjectMapper mapper = TestHelper.makeJsonMapper();
-    mapper.registerModules(NestedDataModule.getJacksonModulesList());
+    mapper.registerModules(BuiltInTypesModule.getJacksonModulesList());
     mapper.setInjectableValues(
         new InjectableValues.Std()
             .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
             .addValue(ObjectMapper.class.getName(), mapper)
-            .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null))
+            .addValue(DefaultColumnFormatConfig.class, new DefaultColumnFormatConfig(null, null))
     );
     Mockito.when(injector.getInstance(Key.get(ObjectMapper.class, Json.class))).thenReturn(mapper);
-    Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null));
+    Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null));
 
     List segments = createSegments(tempFolder, closer);
     QueryableIndex queryableIndex = segments.get(0).asQueryableIndex();

From d2828fb44de370cf519a4b311c5e707cc3527511 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Sat, 3 Aug 2024 13:19:41 -0700
Subject: [PATCH 33/74] fix static checks

---
 .../java/org/apache/druid/segment/Cursor.java |  4 ++--
 .../apache/druid/segment/CursorHolder.java    | 23 -------------------
 2 files changed, 2 insertions(+), 25 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java
index 170502808f22..3c41dbb6aa82 100644
--- a/processing/src/main/java/org/apache/druid/segment/Cursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java
@@ -43,8 +43,8 @@
  *   }
  * 
* - * {@link QueryableIndexCursorHolder.QueryableIndexCursor} is an implementation for historical segments, and {@link - * IncrementalIndexCursorHolder.IncrementalIndexCursor} is an implementation for + * {@link QueryableIndexCursorHolder.QueryableIndexCursor} is an implementation for historical segments, and + * {@link IncrementalIndexCursorHolder.IncrementalIndexCursor} is an implementation for * {@link org.apache.druid.segment.incremental.IncrementalIndex}. * * Cursor is conceptually similar to {@link TimeAndDimsPointer}, but the latter is used for historical segment creation diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java index aa6ea6b6ea2a..85776c7214a3 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java @@ -58,27 +58,4 @@ default void close() { // nothing to close } - - CursorHolder EMPTY = new CursorHolder() - { - @Override - public boolean canVectorize() - { - return true; - } - - @Nullable - @Override - public Cursor asCursor() - { - return null; - } - - @Nullable - @Override - public VectorCursor asVectorCursor() - { - return null; - } - }; } From db91f8a533acb06d599d1c42edbe0123aaa9cb41 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 11:47:46 -0700 Subject: [PATCH 34/74] switch descending to preferredOrdering --- .../ExpressionSelectorBenchmark.java | 2 +- .../druid/benchmark/query/ScanBenchmark.java | 3 +- .../movingaverage/MovingAverageQuery.java | 4 +- .../msq/indexing/MSQCompactionRunner.java | 7 +- .../scan/ScanQueryFrameProcessor.java | 3 +- .../druid/msq/querykit/scan/ScanQueryKit.java | 6 +- .../apache/druid/msq/exec/MSQArraysTest.java | 9 +- .../druid/msq/exec/MSQComplexGroupByTest.java | 5 +- .../druid/msq/exec/MSQLoadedSegmentTests.java | 4 +- .../apache/druid/msq/exec/MSQSelectTest.java | 4 +- .../msq/indexing/MSQCompactionRunnerTest.java | 6 +- .../columnar/FrameCursorMakerFactory.java | 7 +- .../segment/row/FrameCursorMakerFactory.java | 2 +- .../org/apache/druid/query/BaseQuery.java | 3 +- .../java/org/apache/druid/query/Druids.java | 10 +- .../java/org/apache/druid/query/Query.java | 2 +- .../DataSourceMetadataQuery.java | 2 +- .../druid/query/groupby/GroupByQuery.java | 1 - .../epinephelinae/GroupByQueryEngine.java | 2 +- .../metadata/SegmentMetadataQuery.java | 2 +- .../query/operator/WindowOperatorQuery.java | 7 +- .../org/apache/druid/query/scan/Order.java | 44 +++++++++ .../org/apache/druid/query/scan/OrderBy.java | 96 +++++++++++++++++++ .../apache/druid/query/scan/ScanQuery.java | 82 +--------------- .../druid/query/scan/ScanQueryEngine.java | 4 +- .../query/scan/ScanQueryLimitRowIterator.java | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 4 +- .../ScanResultValueTimestampComparator.java | 2 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../query/timeboundary/TimeBoundaryQuery.java | 2 +- .../TimeBoundaryQueryRunnerFactory.java | 14 ++- .../query/timeseries/TimeseriesQuery.java | 10 +- .../apache/druid/query/topn/TopNQuery.java | 1 - .../druid/query/topn/TopNQueryEngine.java | 4 +- .../apache/druid/segment/CursorBuildSpec.java | 35 +++++-- .../segment/QueryableIndexCursorHolder.java | 2 +- .../druid/segment/RowBasedStorageAdapter.java | 5 +- .../apache/druid/segment/StorageAdapter.java | 10 +- .../IncrementalIndexCursorHolder.java | 2 +- .../join/HashJoinSegmentStorageAdapter.java | 13 +-- .../apache/druid/utils/CollectionUtils.java | 2 +- .../segment/FrameStorageAdapterTest.java | 9 +- .../druid/query/CursorGranularizerTest.java | 9 +- .../apache/druid/query/QueryContextsTest.java | 10 -- ...TimelineMissingSegmentQueryRunnerTest.java | 1 - .../org/apache/druid/query/TestQuery.java | 5 +- .../druid/query/groupby/GroupByQueryTest.java | 2 - .../scan/ScanQueryLimitRowIteratorTest.java | 6 +- .../scan/ScanQueryResultOrderingTest.java | 6 +- .../scan/ScanQueryRunnerFactoryTest.java | 20 ++-- .../druid/query/scan/ScanQueryRunnerTest.java | 8 +- .../druid/query/scan/ScanQuerySpecTest.java | 16 ++-- .../druid/query/scan/ScanQueryTest.java | 39 ++++---- ...canResultValueTimestampComparatorTest.java | 8 +- .../query/scan/UnnestScanQueryRunnerTest.java | 2 +- .../query/timeseries/TimeseriesQueryTest.java | 2 - .../druid/query/topn/TopNQueryTest.java | 2 - .../segment/RowBasedStorageAdapterTest.java | 18 +++- .../IncrementalIndexStorageAdapterTest.java | 10 +- .../CachingClusteredClientPerfTest.java | 4 +- .../appenderator/StreamAppenderatorTest.java | 3 +- .../coordination/ServerManagerTest.java | 1 - .../server/log/LoggingRequestLoggerTest.java | 15 +-- .../druid/sql/calcite/rel/DruidQuery.java | 10 +- .../CalciteCatalogIngestionDmlTest.java | 11 ++- .../sql/calcite/CalciteInsertDmlTest.java | 34 +++---- .../sql/calcite/CalciteJoinQueryTest.java | 3 +- .../druid/sql/calcite/CalciteQueryTest.java | 5 +- .../sql/calcite/CalciteReplaceDmlTest.java | 16 ++-- .../sql/calcite/CalciteSelectQueryTest.java | 25 ++--- .../sql/calcite/CalciteSubqueryTest.java | 3 +- 71 files changed, 432 insertions(+), 298 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/scan/Order.java create mode 100644 processing/src/main/java/org/apache/druid/query/scan/OrderBy.java 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 89a99834899f..7047896ae7ce 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -203,7 +203,7 @@ public void timeFloorUsingCursor(Blackhole blackhole) cursor, Granularities.HOUR, buildSpec.getInterval(), - buildSpec.isDescending() + false ); final Sequence results = Sequences.simple(granularizer.getBucketIterable()) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index a33dbc0dcfc3..884370d28c7f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -49,6 +49,7 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryConfig; import org.apache.druid.query.scan.ScanQueryEngine; @@ -116,7 +117,7 @@ public class ScanBenchmark private int limit; @Param({"NONE", "DESCENDING", "ASCENDING"}) - private static ScanQuery.Order ordering; + private static Order ordering; private static final Logger log = new Logger(ScanBenchmark.class); private static final int RNG_SEED = 9999; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 280bc8ccceb2..cbd9272dd8d8 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -103,7 +103,7 @@ public MovingAverageQuery( @JsonProperty("context") Map context ) { - super(dataSource, querySegmentSpec, false, context); + super(dataSource, querySegmentSpec, context); //TBD: Implement null awareness to respect the contract of this flag. Preconditions.checkArgument( @@ -207,7 +207,7 @@ private MovingAverageQuery( Map context ) { - super(dataSource, querySegmentSpec, false, context); + super(dataSource, querySegmentSpec, context); this.dimFilter = dimFilter; this.granularity = granularity; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 7b4d3235dc0c..8eacf0035ded 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -60,7 +60,8 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -423,9 +424,9 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i orderByColumnSpecs .stream() .map(orderByColumnSpec -> - new ScanQuery.OrderBy( + new OrderBy( orderByColumnSpec.getDimension(), - ScanQuery.Order.fromString(orderByColumnSpec.getDirection().toString()) + Order.fromString(orderByColumnSpec.getDirection().toString()) )) .collect(Collectors.toList()) ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index fa2a56ecb1f2..13fc7ed71098 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -59,6 +59,7 @@ import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.query.Druids; import org.apache.druid.query.IterableRowsCursorHelper; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.segment.ColumnSelectorFactory; @@ -173,7 +174,7 @@ public static Sequence mappingFunction(Sequence input */ private static ScanQuery prepareScanQueryForDataServer(@NotNull ScanQuery scanQuery) { - if (ScanQuery.Order.NONE.equals(scanQuery.getTimeOrder()) && !scanQuery.getOrderBys().isEmpty()) { + if (Order.NONE.equals(scanQuery.getTimeOrder()) && !scanQuery.getOrderBys().isEmpty()) { return Druids.ScanQueryBuilder.copy(scanQuery) .orderBy(ImmutableList.of()) .limit(0) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index 48a17a9e84e2..006983d3cb04 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -39,6 +39,8 @@ import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Query; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -120,11 +122,11 @@ public QueryDefinition makeQueryDefinition( final List clusterByColumns = new ArrayList<>(); // Add regular orderBys. - for (final ScanQuery.OrderBy orderBy : queryToRun.getOrderBys()) { + for (final OrderBy orderBy : queryToRun.getOrderBys()) { clusterByColumns.add( new KeyColumn( orderBy.getColumnName(), - orderBy.getOrder() == ScanQuery.Order.DESCENDING ? KeyOrder.DESCENDING : KeyOrder.ASCENDING + orderBy.getOrder() == Order.DESCENDING ? KeyOrder.DESCENDING : KeyOrder.ASCENDING ) ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index 20a871b953cb..17e01b4fb4df 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -38,7 +38,8 @@ import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.Query; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -862,7 +863,7 @@ public void testScanWithOrderByOnStringArray(String contextName, Map cont .intervals(querySegmentSpec(Filtration.eternity())) .columns("obj") .context(defaultScanQueryContext(context, rowSignature)) - .orderBy(Collections.singletonList(new ScanQuery.OrderBy("obj", ScanQuery.Order.ASCENDING))) + .orderBy(Collections.singletonList(new OrderBy("obj", Order.ASCENDING))) .build() ) .columnMappings(new ColumnMappings(ImmutableList.of( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index ab06f851af06..c3d617545eba 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -37,6 +37,8 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.column.ColumnType; @@ -189,7 +191,7 @@ public void testSelectWithLoadedSegmentsOnFooWithOrderBy() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .columns("cnt", "dim1") - .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) + .orderBy(ImmutableList.of(new OrderBy("dim1", Order.ASCENDING))) .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) .build() ) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index c6d67f14428f..b1681e5c717e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -67,6 +67,8 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -463,7 +465,7 @@ public void testSelectAndOrderByOnFooWhereMatchesNoData(String contextName, Map< .columns("cnt", "dim1") .filters(equality("dim2", "nonexistent", ColumnType.STRING)) .context(defaultScanQueryContext(context, resultSignature)) - .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) + .orderBy(ImmutableList.of(new OrderBy("dim1", Order.ASCENDING))) .build() ) .columnMappings(ColumnMappings.identity(resultSignature)) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 6c5d19572652..e064c832b992 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -55,6 +55,8 @@ import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; @@ -271,9 +273,9 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce ); Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); - Assert.assertEquals(PARTITION_DIMENSIONS.stream().map(col -> new ScanQuery.OrderBy( + Assert.assertEquals(PARTITION_DIMENSIONS.stream().map(col -> new OrderBy( col, - ScanQuery.Order.ASCENDING + Order.ASCENDING )).collect(Collectors.toList()), ((ScanQuery) actualMSQSpec.getQuery()).getOrderBys()); } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java index c0e5194a0597..b7dfd2cf597b 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java @@ -85,7 +85,7 @@ public boolean canVectorize() { return (spec.getFilter() == null || spec.getFilter().canVectorizeMatcher(signature)) && spec.getVirtualColumns().canVectorize(signature) - && !spec.isDescending(); + && CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()); } @Override @@ -94,13 +94,14 @@ public Cursor asCursor() final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); final ColumnCache columnCache = new ColumnCache(index, closer); final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); - final SimpleSettableOffset baseOffset = spec.isDescending() + + final SimpleSettableOffset baseOffset = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()) ? new SimpleDescendingOffset(frame.numRows()) : new SimpleAscendingOffset(frame.numRows()); final QueryableIndexColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( spec.getVirtualColumns(), - spec.isDescending(), + spec.isPreferDescendingTimeOrder(), baseOffset, columnCache ); diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java index f9948c00749e..1f07954bac96 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java @@ -74,7 +74,7 @@ public Cursor asCursor() { final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); - final SimpleSettableOffset baseOffset = spec.isDescending() + final SimpleSettableOffset baseOffset = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()) ? new SimpleDescendingOffset(frame.numRows()) : new SimpleAscendingOffset(frame.numRows()); diff --git a/processing/src/main/java/org/apache/druid/query/BaseQuery.java b/processing/src/main/java/org/apache/druid/query/BaseQuery.java index ed97c549b623..f25a5ef317ea 100644 --- a/processing/src/main/java/org/apache/druid/query/BaseQuery.java +++ b/processing/src/main/java/org/apache/druid/query/BaseQuery.java @@ -66,11 +66,10 @@ public static void checkInterrupted() public BaseQuery( DataSource dataSource, QuerySegmentSpec querySegmentSpec, - boolean descending, Map context ) { - this(dataSource, querySegmentSpec, descending, context, Granularities.ALL); + this(dataSource, querySegmentSpec, false, context, Granularities.ALL); } public BaseQuery( 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 824444410c08..ed5029cd32d0 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -39,6 +39,8 @@ import org.apache.druid.query.metadata.metadata.AggregatorMergeStrategy; import org.apache.druid.query.metadata.metadata.ColumnIncluderator; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.query.search.FragmentSearchQuerySpec; @@ -824,8 +826,8 @@ public static class ScanQueryBuilder private long limit; private DimFilter dimFilter; private List columns = new ArrayList<>(); - private ScanQuery.Order order; - private List orderBy; + private Order order; + private List orderBy; private List columnTypes = null; public ScanQuery build() @@ -956,13 +958,13 @@ public ScanQueryBuilder columns(String... c) return this; } - public ScanQueryBuilder order(ScanQuery.Order order) + public ScanQueryBuilder order(Order order) { this.order = order; return this; } - public ScanQueryBuilder orderBy(List orderBys) + public ScanQueryBuilder orderBy(List orderBys) { this.orderBy = orderBys; return this; diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index c2a5ffd9e612..6be01e0d108b 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -285,12 +285,12 @@ default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics i ) ); + return CursorBuildSpec.builder() .setInterval(interval) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setVirtualColumns(getVirtualColumns()) .setQueryContext(context()) - .isDescending(isDescending()) .setQueryMetrics(queryMetrics) .build(); } diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQuery.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQuery.java index 37e6a48ea480..5c43c6445bc9 100644 --- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQuery.java +++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQuery.java @@ -51,7 +51,7 @@ public DataSourceMetadataQuery( @JsonProperty("context") Map context ) { - super(dataSource, querySegmentSpec == null ? DEFAULT_SEGMENT_SPEC : querySegmentSpec, false, context); + super(dataSource, querySegmentSpec == null ? DEFAULT_SEGMENT_SPEC : querySegmentSpec, context); } @Override 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 6d4b2f1643cc..3ae82a9f2a12 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 @@ -824,7 +824,6 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setGroupingAndVirtualColumns(getGranularity(), groupingColumns, virtualColumns) .setAggregators(getAggregatorSpecs()) .setQueryContext(context()) - .isDescending(isDescending()) .setQueryMetrics(queryMetrics) .build(); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index b3c1773c76c1..9f2507724621 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -114,7 +114,7 @@ public static Sequence process( cursor, query.getGranularity(), buildSpec.getInterval(), - buildSpec.isDescending() + false ); if (granularizer == null) { return Sequences.empty(); diff --git a/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java index 85c84fe38583..f2d434bab8a5 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -97,7 +97,7 @@ public SegmentMetadataQuery( @JsonProperty("aggregatorMergeStrategy") AggregatorMergeStrategy aggregatorMergeStrategy ) { - super(dataSource, querySegmentSpec == null ? DEFAULT_SEGMENT_SPEC : querySegmentSpec, false, context); + super(dataSource, querySegmentSpec == null ? DEFAULT_SEGMENT_SPEC : querySegmentSpec, context); if (querySegmentSpec == null) { this.usingDefaultInterval = true; diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java index 50289b9851be..ac9a2f9e4d54 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java @@ -29,6 +29,8 @@ import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.column.RowSignature; @@ -97,7 +99,6 @@ public WindowOperatorQuery( super( validateMaybeRewriteDataSource(dataSource, leafOperators != null), intervals, - false, context ); this.rowSignature = rowSignature; @@ -114,11 +115,11 @@ public WindowOperatorQuery( ScanQuery scan = (ScanQuery) subQuery; ArrayList ordering = new ArrayList<>(); - for (ScanQuery.OrderBy orderBy : scan.getOrderBys()) { + for (OrderBy orderBy : scan.getOrderBys()) { ordering.add( new ColumnWithDirection( orderBy.getColumnName(), - ScanQuery.Order.DESCENDING == orderBy.getOrder() + Order.DESCENDING == orderBy.getOrder() ? ColumnWithDirection.Direction.DESC : ColumnWithDirection.Direction.ASC ) diff --git a/processing/src/main/java/org/apache/druid/query/scan/Order.java b/processing/src/main/java/org/apache/druid/query/scan/Order.java new file mode 100644 index 000000000000..00821f5fed20 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scan/Order.java @@ -0,0 +1,44 @@ +/* + * 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.query.scan; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.java.util.common.StringUtils; + +public enum Order +{ + ASCENDING, + DESCENDING, + NONE; + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static Order fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/scan/OrderBy.java b/processing/src/main/java/org/apache/druid/query/scan/OrderBy.java new file mode 100644 index 000000000000..f1ecf95277cf --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scan/OrderBy.java @@ -0,0 +1,96 @@ +/* + * 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.query.scan; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.Objects; + +public class OrderBy +{ + public static OrderBy ascending(String columnName) + { + return new OrderBy(columnName, Order.ASCENDING); + } + + public static OrderBy descending(String columnName) + { + return new OrderBy(columnName, Order.DESCENDING); + } + + private final String columnName; + private final Order order; + + @JsonCreator + public OrderBy( + @JsonProperty("columnName") final String columnName, + @JsonProperty("order") final Order order + ) + { + this.columnName = Preconditions.checkNotNull(columnName, "columnName"); + this.order = Preconditions.checkNotNull(order, "order"); + + if (order == Order.NONE) { + throw new IAE("Order required for column [%s]", columnName); + } + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @JsonProperty + public Order getOrder() + { + return order; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OrderBy that = (OrderBy) o; + return Objects.equals(columnName, that.columnName) && order == that.order; + } + + @Override + public int hashCode() + { + return Objects.hash(columnName, order); + } + + @Override + public String toString() + { + return StringUtils.format("%s %s", columnName, order == Order.ASCENDING ? "ASC" : "DESC"); + } + +} diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index be616cc28027..fec02576ed47 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; @@ -102,83 +101,6 @@ public static ResultFormat fromString(String name) } } - public static class OrderBy - { - private final String columnName; - private final Order order; - - @JsonCreator - public OrderBy( - @JsonProperty("columnName") final String columnName, - @JsonProperty("order") final Order order - ) - { - this.columnName = Preconditions.checkNotNull(columnName, "columnName"); - this.order = Preconditions.checkNotNull(order, "order"); - - if (order == Order.NONE) { - throw new IAE("Order required for column [%s]", columnName); - } - } - - @JsonProperty - public String getColumnName() - { - return columnName; - } - - @JsonProperty - public Order getOrder() - { - return order; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OrderBy that = (OrderBy) o; - return Objects.equals(columnName, that.columnName) && order == that.order; - } - - @Override - public int hashCode() - { - return Objects.hash(columnName, order); - } - - @Override - public String toString() - { - return StringUtils.format("%s %s", columnName, order == Order.ASCENDING ? "ASC" : "DESC"); - } - } - - public enum Order - { - ASCENDING, - DESCENDING, - NONE; - - @JsonValue - @Override - public String toString() - { - return StringUtils.toLowerCase(this.name()); - } - - @JsonCreator - public static Order fromString(String name) - { - return valueOf(StringUtils.toUpperCase(name)); - } - } - /** * This context flag corresponds to whether the query is running on the "outermost" process (i.e. the process * the query is sent to). @@ -216,7 +138,7 @@ public ScanQuery( @JsonProperty("columnTypes") List columnTypes ) { - super(dataSource, querySegmentSpec, false, context); + super(dataSource, querySegmentSpec, context); this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns); this.resultFormat = (resultFormat == null) ? ResultFormat.RESULT_FORMAT_LIST : resultFormat; this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize; @@ -510,8 +432,8 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setInterval(interval) .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) .setVirtualColumns(getVirtualColumns()) + .setPreferredOrdering(getOrderBys()) .setQueryContext(context()) - .isDescending(isDescending() || Order.DESCENDING.equals(timeOrder)) .setQueryMetrics(queryMetrics) .build(); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index c648e99d05ff..87b316dbf0cf 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -66,7 +66,7 @@ public Sequence process( ) { final Long numScannedRows = responseContext.getRowScanCount(); - if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(ScanQuery.Order.NONE)) { + if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(Order.NONE)) { return Sequences.empty(); } final boolean hasTimeout = query.context().hasTimeout(); @@ -231,7 +231,7 @@ public void cleanup(Iterator iterFromMake) */ private long calculateRemainingScanRowsLimit(ScanQuery query, ResponseContext responseContext) { - if (query.getTimeOrder().equals(ScanQuery.Order.NONE)) { + if (query.getTimeOrder().equals(Order.NONE)) { return query.getScanRowsLimit() - (Long) responseContext.getRowScanCount(); } return query.getScanRowsLimit(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 785b71ed42d1..69681891c998 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -99,7 +99,7 @@ public ScanResultValue next() // We want to perform multi-event ScanResultValue limiting if we are not time-ordering or are at the // inner-level if we are time-ordering - if (query.getTimeOrder() == ScanQuery.Order.NONE || + if (query.getTimeOrder() == Order.NONE || !query.context().getBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { ScanResultValue batch = yielder.get(); List events = (List) batch.getEvents(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 0013c4f84fcd..8e85476e7bbc 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -96,7 +96,7 @@ public QueryRunner mergeRunners( final long timeoutAt = System.currentTimeMillis() + queryPlus.getQuery().context().getTimeout(); responseContext.putTimeoutTime(timeoutAt); - if (query.getTimeOrder().equals(ScanQuery.Order.NONE)) { + if (query.getTimeOrder().equals(Order.NONE)) { // Use normal strategy Sequence returnedRows = Sequences.concat( Sequences.map( @@ -113,7 +113,7 @@ public QueryRunner mergeRunners( List intervalsOrdered = getIntervalsFromSpecificQuerySpec(query.getQuerySegmentSpec()); List> queryRunnersOrdered = Lists.newArrayList(queryRunners); - if (ScanQuery.Order.DESCENDING.equals(query.getTimeOrder())) { + if (Order.DESCENDING.equals(query.getTimeOrder())) { intervalsOrdered = Lists.reverse(intervalsOrdered); queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 9724c2408bae..c74d52464f1e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -45,7 +45,7 @@ public int compare(ScanResultValue o1, ScanResultValue o2) int comparison = Longs.compare( o1.getFirstEventTimestamp(scanQuery.getResultFormat()), o2.getFirstEventTimestamp(scanQuery.getResultFormat())); - if (scanQuery.getTimeOrder().equals(ScanQuery.Order.ASCENDING)) { + if (scanQuery.getTimeOrder().equals(Order.ASCENDING)) { return comparison; } return comparison * -1; diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index c15e1d0d99c4..3519de9b2f82 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -97,7 +97,7 @@ public BinaryOperator> createMergeFn( @Override public Comparator> createResultComparator(Query> query) { - return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); + return ResultGranularTimestampComparator.create(query.getGranularity(), false); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java index aac161c27286..17db30455e0f 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java @@ -66,7 +66,7 @@ public TimeBoundaryQuery( @JsonProperty("context") Map context ) { - super(dataSource, querySegmentSpec == null ? DEFAULT_SEGMENT_SPEC : querySegmentSpec, false, context); + super(dataSource, querySegmentSpec == null ? DEFAULT_SEGMENT_SPEC : querySegmentSpec, context); this.dimFilter = dimFilter; this.bound = bound == null ? "" : bound; diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 5ac83e2aebd2..1734b5b310be 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -37,6 +37,7 @@ import org.apache.druid.query.Result; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; @@ -49,6 +50,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collections; import java.util.Iterator; /** @@ -115,10 +117,14 @@ public Result apply(Cursor cursor) @Nullable private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legacyQuery, boolean descending) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder(legacyQuery.asCursorBuildSpec(null)) - .isDescending(descending) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final CursorBuildSpec.CursorBuildSpecBuilder bob = CursorBuildSpec.builder(legacyQuery.asCursorBuildSpec(null)); + if (descending) { + bob.setPreferredOrdering(Collections.singletonList(OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME))); + } else { + bob.setPreferredOrdering(Collections.singletonList(OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME))); + } + + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(bob.build())) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return null; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index b6535fcc5252..6459003d6d45 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -40,9 +40,11 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec.LimitJsonIncludeFilter; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; @@ -218,7 +220,13 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) .setGroupingAndVirtualColumns(getGranularity(), null, virtualColumns) .setAggregators(getAggregatorSpecs()) .setQueryContext(context()) - .isDescending(isDescending()) + .setPreferredOrdering( + Collections.singletonList( + isDescending() ? + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) : + OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME) + ) + ) .setQueryMetrics(queryMetrics) .build(); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 0990b6dbb48d..1e186bd5329d 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -204,7 +204,6 @@ public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) ) .setAggregators(getAggregatorSpecs()) .setQueryContext(context()) - .isDescending(isDescending()) .setQueryMetrics(queryMetrics) .build(); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index 79ca63d0daec..7f9254a1ac52 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -83,7 +83,7 @@ public Sequence> query( cursor, query.getGranularity(), buildSpec.getInterval(), - buildSpec.isDescending() + false ); if (granularizer == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); @@ -96,7 +96,7 @@ public Sequence> query( Sequences.simple(granularizer.getBucketIterable()) .map(bucketInterval -> { granularizer.advanceToBucket(bucketInterval); - cursor.mark(buildSpec.isDescending() ? bucketInterval.getEnd() : bucketInterval.getStart()); + cursor.mark(bucketInterval.getStart()); return mapFn.apply(cursor, granularizer, queryMetrics); }), Predicates.notNull() diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index a0fd6cd27874..50a1a39a6282 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -28,6 +28,9 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; +import org.apache.druid.segment.column.ColumnHolder; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -57,10 +60,11 @@ public static CursorBuildSpecBuilder builder(CursorBuildSpec spec) private final VirtualColumns virtualColumns; @Nullable private final List aggregators; + @Nullable + private final List orderByColumns; private final QueryContext queryContext; - private final boolean descending; @Nullable private final QueryMetrics queryMetrics; @@ -70,8 +74,8 @@ public CursorBuildSpec( @Nullable List groupingColumns, VirtualColumns virtualColumns, @Nullable List aggregators, + @Nullable List preferredOrdering, QueryContext queryContext, - boolean descending, @Nullable QueryMetrics queryMetrics ) { @@ -80,7 +84,7 @@ public CursorBuildSpec( this.groupingColumns = groupingColumns; this.virtualColumns = virtualColumns; this.aggregators = aggregators; - this.descending = descending; + this.orderByColumns = preferredOrdering; this.queryContext = queryContext; this.queryMetrics = queryMetrics; } @@ -113,9 +117,10 @@ public List getAggregators() return aggregators; } - public boolean isDescending() + @Nullable + public List getPreferredOrdering() { - return descending; + return orderByColumns; } public QueryContext getQueryContext() @@ -129,6 +134,15 @@ public QueryMetrics getQueryMetrics() return queryMetrics; } + public static boolean preferDescendingTimeOrder(@Nullable List preferredOrdering) + { + if (preferredOrdering != null && preferredOrdering.size() == 1) { + final OrderBy orderBy = Iterables.getOnlyElement(preferredOrdering); + return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()) && Order.DESCENDING == orderBy.getOrder(); + } + return false; + } + public static class CursorBuildSpecBuilder { @Nullable @@ -140,7 +154,8 @@ public static class CursorBuildSpecBuilder private VirtualColumns virtualColumns = VirtualColumns.EMPTY; @Nullable private List aggregators; - private boolean descending = false; + @Nullable + private List preferredOrdering; private QueryContext queryContext = QueryContext.empty(); @Nullable @@ -158,7 +173,7 @@ private CursorBuildSpecBuilder(CursorBuildSpec buildSpec) this.groupingColumns = buildSpec.groupingColumns; this.virtualColumns = buildSpec.virtualColumns; this.aggregators = buildSpec.aggregators; - this.descending = buildSpec.descending; + this.preferredOrdering = buildSpec.orderByColumns; this.queryContext = buildSpec.queryContext; this.queryMetrics = buildSpec.queryMetrics; } @@ -214,9 +229,9 @@ public CursorBuildSpecBuilder setAggregators(@Nullable List a return this; } - public CursorBuildSpecBuilder isDescending(boolean descending) + public CursorBuildSpecBuilder setPreferredOrdering(@Nullable List orderBy) { - this.descending = descending; + this.preferredOrdering = orderBy; return this; } @@ -240,8 +255,8 @@ public CursorBuildSpec build() groupingColumns, virtualColumns, aggregators, + preferredOrdering, queryContext, - descending, queryMetrics ); } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index e35bb943c92a..626255bce89f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -88,7 +88,7 @@ public QueryableIndexCursorHolder( this.virtualColumns = cursorBuildSpec.getVirtualColumns(); this.aggregatorFactories = cursorBuildSpec.getAggregators(); this.filter = cursorBuildSpec.getFilter(); - this.descending = cursorBuildSpec.isDescending(); + this.descending = CursorBuildSpec.preferDescendingTimeOrder(cursorBuildSpec.getPreferredOrdering()); this.queryContext = cursorBuildSpec.getQueryContext(); this.vectorSize = cursorBuildSpec.getQueryContext().getVectorSize(); this.metrics = cursorBuildSpec.getQueryMetrics(); diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index 2b710d01db0c..946ae9970b8f 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -175,9 +175,10 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec) @Override public Cursor asCursor() { + final boolean descendingTimeOrder = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()); final RowWalker rowWalker = closer.register( new RowWalker<>( - spec.isDescending() ? reverse(rowSequence) : rowSequence, + descendingTimeOrder ? reverse(rowSequence) : rowSequence, rowAdapter ) ); @@ -187,7 +188,7 @@ public Cursor asCursor() spec.getFilter(), spec.getInterval(), spec.getVirtualColumns(), - spec.isDescending(), + descendingTimeOrder, rowSignature ); } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index fdb5cdc2fd8e..1cfc56b352d1 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -52,7 +52,11 @@ default CursorHolder makeCursorHolder(CursorBuildSpec spec) @Override public boolean canVectorize() { - return StorageAdapter.this.canVectorize(spec.getFilter(), spec.getVirtualColumns(), spec.isDescending()); + return StorageAdapter.this.canVectorize( + spec.getFilter(), + spec.getVirtualColumns(), + CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()) + ); } @Override @@ -64,7 +68,7 @@ public Cursor asCursor() spec.getInterval(), spec.getVirtualColumns(), Granularities.ALL, - spec.isDescending(), + CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()), spec.getQueryMetrics() ).toList() ); @@ -77,7 +81,7 @@ public VectorCursor asVectorCursor() spec.getFilter(), spec.getInterval(), spec.getVirtualColumns(), - spec.isDescending(), + CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()), spec.getQueryContext().getVectorSize(), spec.getQueryMetrics() ); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 92be2553ddb4..382ead7200d6 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -68,7 +68,7 @@ public Cursor asCursor() storageAdapter, index, spec.getVirtualColumns(), - spec.isDescending(), + CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()), spec.getFilter(), spec.getInterval() ); diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 684bac294c15..d53478b9df31 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -218,19 +218,15 @@ public boolean hasBuiltInFilters() @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - final CursorBuildSpec.CursorBuildSpecBuilder cursorBuildSpecBuilder = - CursorBuildSpec.builder() - .setInterval(spec.getInterval()) - .isDescending(spec.isDescending()) - .setQueryMetrics(spec.getQueryMetrics()); + // make a copy of CursorBuildSpec with filters removed + final CursorBuildSpec.CursorBuildSpecBuilder cursorBuildSpecBuilder = CursorBuildSpec.builder(spec) + .setFilter(null); final Filter combinedFilter = baseFilterAnd(spec.getFilter()); - if (clauses.isEmpty()) { // if there are no clauses, we can just use the base cursor directly if we apply the combined filter final CursorBuildSpec newSpec = cursorBuildSpecBuilder.setFilter(combinedFilter) - .setVirtualColumns(spec.getVirtualColumns()) .build(); return baseAdapter.makeCursorHolder(newSpec); } @@ -294,9 +290,10 @@ public Cursor asCursor() } Cursor retVal = baseCursor; + final boolean isDescendingTimeOrdering = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()); for (JoinableClause clause : clauses) { - retVal = HashJoinEngine.makeJoinCursor(retVal, clause, spec.isDescending(), joinablesCloser); + retVal = HashJoinEngine.makeJoinCursor(retVal, clause, isDescendingTimeOrdering, joinablesCloser); } return PostJoinCursor.wrap( diff --git a/processing/src/main/java/org/apache/druid/utils/CollectionUtils.java b/processing/src/main/java/org/apache/druid/utils/CollectionUtils.java index 15c11ebea556..bd7600f6c03c 100644 --- a/processing/src/main/java/org/apache/druid/utils/CollectionUtils.java +++ b/processing/src/main/java/org/apache/druid/utils/CollectionUtils.java @@ -177,7 +177,7 @@ public static Set intersect(Set left, Set right) } /** - * Intersection of two sets: {@code C = A ∪ B}. + * Union of two sets: {@code C = A ∪ B}. */ public static Set union(Set left, Set right) { diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index 5d6af7eb67b6..acce8020acef 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -35,6 +35,7 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; @@ -270,7 +271,13 @@ public CursorTests( .setFilter(this.filter) .setInterval(this.interval) .setVirtualColumns(this.virtualColumns) - .isDescending(this.descending) + .setPreferredOrdering( + descending ? + Collections.singletonList( + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) + ) : + null + ) .setQueryContext(queryContext) .build(); } diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java index 67939e1044f8..0f6dbe60c792 100644 --- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; @@ -34,6 +35,7 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -46,6 +48,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; public class CursorGranularizerTest extends InitializedNullHandlingTest @@ -203,7 +206,11 @@ public void testGranularizeFullScan() public void testGranularizeFullScanDescending() { final CursorBuildSpec descending = CursorBuildSpec.builder() - .isDescending(true) + .setPreferredOrdering( + Collections.singletonList( + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) + ) + ) .build(); try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) { final Cursor cursor = cursorHolder.asCursor(); diff --git a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java index 554ca7b8f4a5..64538f33d8dd 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java @@ -43,7 +43,6 @@ public void testDefaultQueryTimeout() final Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, new HashMap<>() ); Assert.assertEquals(300_000, query.context().getDefaultTimeout()); @@ -55,7 +54,6 @@ public void testEmptyQueryTimeout() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, new HashMap<>() ); Assert.assertEquals(300_000, query.context().getTimeout()); @@ -70,7 +68,6 @@ public void testQueryTimeout() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1000) ); Assert.assertEquals(1000, query.context().getTimeout()); @@ -87,7 +84,6 @@ public void testQueryMaxTimeout() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1000) ); @@ -102,7 +98,6 @@ public void testMaxScatterGatherBytes() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, 1000) ); @@ -115,7 +110,6 @@ public void testDisableSegmentPruning() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of(QueryContexts.SECONDARY_PARTITION_PRUNING_KEY, false) ); Assert.assertFalse(query.context().isSecondaryPartitionPruningEnabled()); @@ -127,7 +121,6 @@ public void testDefaultSegmentPruning() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of() ); Assert.assertTrue(query.context().isSecondaryPartitionPruningEnabled()); @@ -213,7 +206,6 @@ public void testGetTimeout_withNonNumericValue() new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, queryContext ).context().getTimeout(); } @@ -297,7 +289,6 @@ public void testGetEnum() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of("e1", "FORCE", "e2", "INVALID_ENUM" ) @@ -320,7 +311,6 @@ public void testExecutionModeEnum() Query query = new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, "SYNC", QueryContexts.CTX_EXECUTION_MODE + "_1", "ASYNC") ); diff --git a/processing/src/test/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunnerTest.java index 1228c42b17d9..34cb3dbcdea5 100644 --- a/processing/src/test/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/ReportTimelineMissingSegmentQueryRunnerTest.java @@ -73,7 +73,6 @@ private TestQuery() super( new TableDataSource("datasource"), new MultipleSpecificSegmentSpec(Collections.emptyList()), - false, new HashMap<>() ); } diff --git a/processing/src/test/java/org/apache/druid/query/TestQuery.java b/processing/src/test/java/org/apache/druid/query/TestQuery.java index 200151c200ca..186b02444b52 100644 --- a/processing/src/test/java/org/apache/druid/query/TestQuery.java +++ b/processing/src/test/java/org/apache/druid/query/TestQuery.java @@ -31,9 +31,9 @@ public class TestQuery extends BaseQuery @Nullable private Set requiredColumns; - public TestQuery(DataSource dataSource, QuerySegmentSpec querySegmentSpec, boolean descending, Map context) + public TestQuery(DataSource dataSource, QuerySegmentSpec querySegmentSpec, Map context) { - super(dataSource, querySegmentSpec, descending, context); + super(dataSource, querySegmentSpec, context); } @Override @@ -72,7 +72,6 @@ public Query withOverriddenContext(Map contextOverride) return new TestQuery( getDataSource(), getQuerySegmentSpec(), - isDescending(), BaseQuery.computeOverriddenContext(getContext(), contextOverride) ); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java index 2de5c0b8d79b..ccaad1fa984d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java @@ -205,7 +205,6 @@ public void testAsCursorBuildSpecAllGranularity() Assert.assertEquals(ImmutableList.of("quality", "market", "v0"), buildSpec.getGroupingColumns()); Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } @Test @@ -258,7 +257,6 @@ public void testAsCursorBuildSpecDayGranularity() ), buildSpec.getVirtualColumns() ); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java index 60b098b43bfb..a3390bef347b 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java @@ -104,7 +104,7 @@ public void testNonOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .order(ScanQuery.Order.NONE) + .order(Order.NONE) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -145,7 +145,7 @@ public void testBrokerOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -184,7 +184,7 @@ public void testHistoricalOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index 6bbdabf6619b..e6c915b66c57 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -207,7 +207,7 @@ public void testOrderNone() .dataSource("ds") .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2000/P1D")))) .columns(ColumnHolder.TIME_COLUMN_NAME, ID_COLUMN) - .order(ScanQuery.Order.NONE) + .order(Order.NONE) .build(), ImmutableList.of( 101, @@ -241,7 +241,7 @@ public void testOrderTimeAscending() .dataSource("ds") .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2000/P1D")))) .columns(ColumnHolder.TIME_COLUMN_NAME, ID_COLUMN) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .build(), ImmutableList.of( 101, @@ -275,7 +275,7 @@ public void testOrderTimeDescending() .dataSource("ds") .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2000/P1D")))) .columns(ColumnHolder.TIME_COLUMN_NAME, ID_COLUMN) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .build(), ImmutableList.of( 8, diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 55c23a8d8c99..4c841a35e27b 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -91,7 +91,7 @@ public ScanQueryRunnerFactoryParameterizedTest( final int batchSize, final long limit, final ScanQuery.ResultFormat resultFormat, - final ScanQuery.Order order + final Order order ) { this.numElements = numElements; @@ -116,9 +116,9 @@ public static Iterable constructorFeeder() ScanQuery.ResultFormat.RESULT_FORMAT_LIST, ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST ); - List order = ImmutableList.of( - ScanQuery.Order.ASCENDING, - ScanQuery.Order.DESCENDING + List order = ImmutableList.of( + Order.ASCENDING, + Order.DESCENDING ); return QueryRunnerTestHelper.cartesian( @@ -147,7 +147,7 @@ public void testSortAndLimitScanResultValues() throws IOException } else if (o1 < o2) { retVal = -1; } - if (query.getTimeOrder().equals(ScanQuery.Order.DESCENDING)) { + if (query.getTimeOrder().equals(Order.DESCENDING)) { return retVal * -1; } return retVal; @@ -200,7 +200,7 @@ public void testNWayMerge() scanResultValues3.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); } - if (query.getTimeOrder() == ScanQuery.Order.DESCENDING) { + if (query.getTimeOrder() == Order.DESCENDING) { Collections.reverse(scanResultValues1); Collections.reverse(scanResultValues2); Collections.reverse(scanResultValues3); @@ -221,7 +221,7 @@ public void testNWayMerge() List>> groupedRunners = new ArrayList<>(2); - if (query.getTimeOrder() == ScanQuery.Order.DESCENDING) { + if (query.getTimeOrder() == Order.DESCENDING) { groupedRunners.add(Arrays.asList(runnerSegment2Partition1, runnerSegment2Partition2)); groupedRunners.add(Arrays.asList(runnerSegment1Partition1, runnerSegment1Partition2)); } else { @@ -236,7 +236,7 @@ public void testNWayMerge() } else if (o1 < o2) { retVal = -1; } - if (query.getTimeOrder().equals(ScanQuery.Order.DESCENDING)) { + if (query.getTimeOrder().equals(Order.DESCENDING)) { return retVal * -1; } return retVal; @@ -268,7 +268,7 @@ private void validateSortedOutput(List output, List expec // check ordering is correct for (int i = 1; i < output.size(); i++) { - if (query.getTimeOrder().equals(ScanQuery.Order.DESCENDING)) { + if (query.getTimeOrder().equals(Order.DESCENDING)) { Assert.assertTrue(output.get(i).getFirstEventTimestamp(resultFormat) < output.get(i - 1).getFirstEventTimestamp(resultFormat)); } else { @@ -370,7 +370,7 @@ public void testMergeRunnersGuardrailsExceeded() .collect(Collectors.toList()) ) ) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .build() ), ResponseContext.createEmpty() diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index f2c3a9ea18f5..dec9eb70879d 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -535,7 +535,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() QueryRunnerTestHelper.INDEX_METRIC ) .limit(limit) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .build(); @@ -607,7 +607,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() QueryRunnerTestHelper.INDEX_METRIC ) .limit(limit) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .build(); Iterable results = runner.run(QueryPlus.wrap(query)).toList(); @@ -703,7 +703,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList QueryRunnerTestHelper.INDEX_METRIC ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .limit(limit) .build(); @@ -776,7 +776,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis QueryRunnerTestHelper.INDEX_METRIC ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .limit(limit) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index e5069743a9ed..82886c705a3a 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -57,7 +57,7 @@ public void testSerialization() throws Exception 0, 0, 3, - ScanQuery.Order.NONE, + Order.NONE, null, null, Arrays.asList("market", "quality", "index"), @@ -85,7 +85,7 @@ public void testSerializationWithTimeOrder() throws Exception 0, 0, 3, - ScanQuery.Order.ASCENDING, + Order.ASCENDING, null, null, Arrays.asList("market", "quality", "index", "__time"), @@ -96,9 +96,9 @@ public void testSerializationWithTimeOrder() throws Exception String serializedJson = JSON_MAPPER.writeValueAsString(expectedQuery); Assert.assertEquals(originalJson, serializedJson); Assert.assertEquals(expectedQuery, JSON_MAPPER.readValue(originalJson, ScanQuery.class)); - Assert.assertEquals(ScanQuery.Order.ASCENDING, expectedQuery.getTimeOrder()); + Assert.assertEquals(Order.ASCENDING, expectedQuery.getTimeOrder()); Assert.assertEquals( - Collections.singletonList(new ScanQuery.OrderBy("__time", ScanQuery.Order.ASCENDING)), + Collections.singletonList(new OrderBy("__time", Order.ASCENDING)), expectedQuery.getOrderBys() ); } @@ -118,7 +118,7 @@ public void testSerializationWithOrderBy() throws Exception 0, 3, null, - Collections.singletonList(new ScanQuery.OrderBy("quality", ScanQuery.Order.ASCENDING)), + Collections.singletonList(new OrderBy("quality", Order.ASCENDING)), null, Arrays.asList("market", "quality", "index", "__time"), null, @@ -128,9 +128,9 @@ public void testSerializationWithOrderBy() throws Exception String serializedJson = JSON_MAPPER.writeValueAsString(expectedQuery); Assert.assertEquals(originalJson, serializedJson); Assert.assertEquals(expectedQuery, JSON_MAPPER.readValue(originalJson, ScanQuery.class)); - Assert.assertEquals(ScanQuery.Order.NONE, expectedQuery.getTimeOrder()); + Assert.assertEquals(Order.NONE, expectedQuery.getTimeOrder()); Assert.assertEquals( - Collections.singletonList(new ScanQuery.OrderBy("quality", ScanQuery.Order.ASCENDING)), + Collections.singletonList(new OrderBy("quality", Order.ASCENDING)), expectedQuery.getOrderBys() ); } @@ -146,7 +146,7 @@ public void testSerializationLegacyString() throws Exception 0, 1, 3, - ScanQuery.Order.NONE, + Order.NONE, null, null, Arrays.asList("market", "quality", "index"), diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index b8e2815e97d6..54d736ec26db 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -123,7 +123,7 @@ public void testSerdeAndLegacyBackwardsCompat() throws JsonProcessingException public void testAscendingScanQueryWithInvalidColumns() { Druids.newScanQueryBuilder() - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .columns(ImmutableList.of("not time", "also not time")) .dataSource("source") .intervals(intervalSpec) @@ -134,7 +134,7 @@ public void testAscendingScanQueryWithInvalidColumns() public void testDescendingScanQueryWithInvalidColumns() { Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .columns(ImmutableList.of("not time", "also not time")) .dataSource("source") .intervals(intervalSpec) @@ -149,12 +149,12 @@ public void testConflictingOrderByAndTimeOrder() IllegalArgumentException.class, () -> Druids.newScanQueryBuilder() - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .orderBy( // Not ok, even though it starts with __time ASC, because it also has non-time component. ImmutableList.of( - new ScanQuery.OrderBy("__time", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("quality", ScanQuery.Order.DESCENDING) + new OrderBy("__time", Order.ASCENDING), + new OrderBy("quality", Order.DESCENDING) ) ) .columns(ImmutableList.of("__time", "quality")) @@ -169,8 +169,8 @@ public void testCompatibleOrderByAndTimeOrder() { Assert.assertNotNull( Druids.newScanQueryBuilder() - .order(ScanQuery.Order.ASCENDING) - .orderBy(ImmutableList.of(new ScanQuery.OrderBy("__time", ScanQuery.Order.ASCENDING))) + .order(Order.ASCENDING) + .orderBy(ImmutableList.of(new OrderBy("__time", Order.ASCENDING))) .columns(ImmutableList.of("__time", "quality")) .dataSource("source") .intervals(intervalSpec) @@ -182,9 +182,9 @@ public void testCompatibleOrderByAndTimeOrder() @Test public void testValidScanQueryInitialization() { - List nonOrderedOrders = Arrays.asList(null, ScanQuery.Order.NONE); + List nonOrderedOrders = Arrays.asList(null, Order.NONE); - for (ScanQuery.Order order : nonOrderedOrders) { + for (Order order : nonOrderedOrders) { Druids.newScanQueryBuilder() .order(order) .columns(ImmutableList.of("not time")) @@ -214,9 +214,9 @@ public void testValidScanQueryInitialization() .build(); } - Set orderedOrders = ImmutableSet.of(ScanQuery.Order.ASCENDING, ScanQuery.Order.DESCENDING); + Set orderedOrders = ImmutableSet.of(Order.ASCENDING, Order.DESCENDING); - for (ScanQuery.Order order : orderedOrders) { + for (Order order : orderedOrders) { Druids.newScanQueryBuilder() .order(order) .columns((List) null) @@ -252,7 +252,7 @@ public void testMergeSequenceForResults() { // Should be able to handle merging s1, s2, s3 ScanQuery noOrderScan = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.NONE) + .order(Order.NONE) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -260,7 +260,7 @@ public void testMergeSequenceForResults() // Should only handle s1 and s2 ScanQuery descendingOrderScan = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -268,7 +268,7 @@ public void testMergeSequenceForResults() // Should only handle s1 and s2 ScanQuery ascendingOrderScan = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -317,7 +317,7 @@ public void testMergeSequenceForResults() public void testTimeOrderingWithoutTimeColumn() { ScanQuery descendingOrderScan = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -340,7 +340,7 @@ public void testGetResultOrderingWithTimeBasedOrderBy() final ScanQuery scanQuery = Druids.newScanQueryBuilder() .columns("__time") - .orderBy(Collections.singletonList(new ScanQuery.OrderBy("__time", ScanQuery.Order.DESCENDING))) + .orderBy(Collections.singletonList(new OrderBy("__time", Order.DESCENDING))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -356,7 +356,7 @@ public void testGetResultOrderingWithNonTimeOrderBy() final ScanQuery scanQuery = Druids.newScanQueryBuilder() .columns("quality") - .orderBy(Collections.singletonList(new ScanQuery.OrderBy("quality", ScanQuery.Order.ASCENDING))) + .orderBy(Collections.singletonList(new OrderBy("quality", Order.ASCENDING))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -370,7 +370,7 @@ public void testGetRequiredColumnsWithNoColumns() { final ScanQuery query = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -384,7 +384,7 @@ public void testGetRequiredColumnsWithEmptyColumns() { final ScanQuery query = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -446,6 +446,5 @@ public void testAsCursorBuildSpec() Assert.assertNull(buildSpec.getGroupingColumns()); Assert.assertNull(buildSpec.getAggregators()); Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 47cc42086402..f2d8ebb28d02 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -52,7 +52,7 @@ public static void setup() public void testComparisonDescendingList() { ScanQuery query = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -89,7 +89,7 @@ public void testComparisonDescendingList() public void testComparisonAscendingList() { ScanQuery query = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -126,7 +126,7 @@ public void testComparisonAscendingList() public void testComparisonDescendingCompactedList() { ScanQuery query = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -161,7 +161,7 @@ public void testComparisonDescendingCompactedList() public void testAscendingCompactedList() { ScanQuery query = Druids.newScanQueryBuilder() - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) diff --git a/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java index 23d7f7a96af8..a70eae07ec46 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java @@ -308,7 +308,7 @@ public void testUnnestRunnerWithOrdering() .columns(QueryRunnerTestHelper.TIME_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) .limit(3) .filters(new SelectorDimFilter(QueryRunnerTestHelper.MARKET_DIMENSION, "spot", null)) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java index 821b2955ee2e..04f0ae9e4825 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java @@ -149,7 +149,6 @@ public void testAsCursorBuildSpecAllGranularity() buildSpec.getAggregators() ); Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } @Test @@ -202,6 +201,5 @@ public void testAsCursorBuildSpecDayGranularity() ), buildSpec.getVirtualColumns() ); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } } diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java index 2b15b932d479..303ea85efce9 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java @@ -295,7 +295,6 @@ public void testAsCursorBuildSpecAllGranularity() Assert.assertEquals(ImmutableList.of("v"), buildSpec.getGroupingColumns()); Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns()); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } @Test @@ -331,6 +330,5 @@ public void testAsCursorBuildSpecDayGranularity() ), buildSpec.getVirtualColumns() ); - Assert.assertEquals(query.isDescending(), buildSpec.isDescending()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 0b0e5b0c7730..36650c65bcec 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -35,7 +35,9 @@ import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.OrderBy; 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; import org.apache.druid.segment.column.ValueType; @@ -534,7 +536,13 @@ public void test_makeCursor_descending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder().isDescending(true).build(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setPreferredOrdering( + Collections.singletonList( + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) + ) + ) + .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertEquals( @@ -647,7 +655,11 @@ public void test_makeCursor_hourGranularityWithIntervalDescending() final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT2H")) - .isDescending(true) + .setPreferredOrdering( + Collections.singletonList( + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) + ) + ) .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { @@ -829,7 +841,7 @@ private static List> walkCursorGranularized( cursor, granularity, buildSpec.getInterval(), - buildSpec.isDescending() + CursorBuildSpec.preferDescendingTimeOrder(buildSpec.getPreferredOrdering()) ); final List> suppliers = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index bf7dd286657c..d74916b25dbf 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -52,6 +52,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; @@ -62,6 +63,7 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; @@ -281,7 +283,13 @@ public void testResetSanity() throws IOException final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorFilter("sally", "bo")) .setInterval(interval) - .isDescending(descending) + .setPreferredOrdering( + descending ? + Collections.singletonList( + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) + ) : + null + ) .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java index 6270f3685406..b31799cf8b4c 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java @@ -155,7 +155,6 @@ private Query makeFakeQuery(Interval interval) return new TestQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(Collections.singletonList(interval)), - false, ImmutableMap.of(BaseQuery.QUERY_ID, "testQuery") ); } @@ -202,11 +201,10 @@ private static class TestQuery extends BaseQuery public TestQuery( DataSource dataSource, QuerySegmentSpec querySegmentSpec, - boolean descending, Map context ) { - super(dataSource, querySegmentSpec, descending, context); + super(dataSource, querySegmentSpec, context); } @Override diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index 538784a88539..d1a25b54e954 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -40,6 +40,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; @@ -1421,7 +1422,7 @@ public void testQueryBySegments() throws Exception ) ) ) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .batchSize(10) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java index 9f20dc3be310..ec4a008c1ed7 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java @@ -599,7 +599,6 @@ public void testGetQueryRunnerForSegmentsForUnknownQueryThrowingException() new BaseQuery( new TableDataSource("test"), new MultipleSpecificSegmentSpec(descriptors), - false, new HashMap<>() ) { diff --git a/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java b/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java index f3dd1eb6809b..0ee15ab1f3f2 100644 --- a/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java +++ b/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java @@ -87,7 +87,8 @@ public QueryRunner lookup(Query query, QuerySegmentWalker walker) { return null; } - }, false, queryContext + }, + queryContext ); final Query nestedQuery = new FakeQuery( @@ -105,7 +106,8 @@ public QueryRunner lookup(Query query, QuerySegmentWalker walker) { return null; } - }, false, queryContext + }, + queryContext ); final Query nestedNestedQuery = new FakeQuery( @@ -123,7 +125,8 @@ public QueryRunner lookup(Query query, QuerySegmentWalker walker) { return null; } - }, false, queryContext + }, + queryContext ); final Query unionQuery = new FakeQuery( @@ -141,7 +144,8 @@ public QueryRunner lookup(Query query, QuerySegmentWalker walker) { return null; } - }, false, queryContext + }, + queryContext ); @@ -325,11 +329,10 @@ class FakeQuery extends BaseQuery public FakeQuery( DataSource dataSource, QuerySegmentSpec querySegmentSpec, - boolean descending, Map context ) { - super(dataSource, querySegmentSpec, descending, context); + super(dataSource, querySegmentSpec, context); } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index bec5335f7c95..b1226c3d8454 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -74,6 +74,8 @@ import org.apache.druid.query.operator.ScanOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.ordering.StringComparator; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.query.timeboundary.TimeBoundaryQuery; @@ -1614,7 +1616,7 @@ private ScanQuery toScanQuery(final boolean considerSorting) final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; - final List orderByColumns; + final List orderByColumns; long scanOffset = 0L; long scanLimit = 0L; @@ -1634,11 +1636,11 @@ private ScanQuery toScanQuery(final boolean considerSorting) orderByColumns = sorting.getOrderBys().stream().map( orderBy -> - new ScanQuery.OrderBy( + new OrderBy( orderBy.getDimension(), orderBy.getDirection() == OrderByColumnSpec.Direction.DESCENDING - ? ScanQuery.Order.DESCENDING - : ScanQuery.Order.ASCENDING + ? Order.DESCENDING + : Order.ASCENDING ) ).collect(Collectors.toList()); } else { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java index 514695c4afe3..0ec629c8626f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java @@ -42,7 +42,8 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.CalciteCatalogIngestionDmlTest.CatalogIngestionDmlComponentSupplier; @@ -581,8 +582,8 @@ public void testInsertTableWithClusteringWithClusteringFromCatalog() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("b", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("d", ScanQuery.Order.ASCENDING) + new OrderBy("b", Order.ASCENDING), + new OrderBy("d", Order.ASCENDING) ) ) // Scan query lists columns in alphabetical order independent of the @@ -643,7 +644,7 @@ public void testInsertTableWithClusteringWithClusteringFromQuery() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("b", ScanQuery.Order.ASCENDING) + new OrderBy("b", Order.ASCENDING) ) ) // Scan query lists columns in alphabetical order independent of the @@ -708,7 +709,7 @@ public void testInsertTableWithClusteringWithClusteringOnNewColumnFromQuery() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("e", ScanQuery.Order.ASCENDING) + new OrderBy("e", Order.ASCENDING) ) ) // Scan query lists columns in alphabetical order independent of the diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index ba7b19311ce9..e8cffd3cac9c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -38,6 +38,8 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -727,9 +729,9 @@ public void testExplainPlanInsertWithClusteredBy() throws JsonProcessingExceptio ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING), + new OrderBy("v1", Order.ASCENDING) ) ) .context( @@ -930,9 +932,9 @@ public void testInsertWithClusteredBy() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING), + new OrderBy("v1", Order.ASCENDING) ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -971,9 +973,9 @@ public void testInsertPeriodFormGranularityWithClusteredBy() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING), + new OrderBy("v1", Order.ASCENDING) ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -1021,8 +1023,8 @@ public void testInsertWithPartitionedByAndClusteredBy() .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT)) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING) ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -1210,9 +1212,9 @@ public void testExplainPlanForInsertWithClusteredBy() throws JsonProcessingExcep ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING), + new OrderBy("v1", Order.ASCENDING) ) ) .context( @@ -1325,8 +1327,8 @@ public void testInsertFromExternalProjectSort() .columns("v0", "z") .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("z", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("z", Order.ASCENDING) ) ) .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 37a1a82f0df4..ddc2b69bcb27 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -71,6 +71,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.timeboundary.TimeBoundaryQuery; import org.apache.druid.query.topn.DimensionTopNMetricSpec; @@ -5872,7 +5873,7 @@ public void testJoinWithAliasAndOrderByNoGroupBy() ) ) .columns("__time") - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .context(context) .build() ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 84b437c20e08..3d19408f16df 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -106,6 +106,7 @@ import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.operator.window.WindowFrame; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQuery.ResultFormat; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -15282,7 +15283,7 @@ public void testOrderByAlongWithInternalScanQuery() "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"))) .limit(1) .columns(ImmutableList.of("__time", "m1")) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .build() ), ImmutableList.of( @@ -15324,7 +15325,7 @@ public void testOrderByAlongWithInternalScanQueryNoDistinct() "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"))) .limit(1) .columns(ImmutableList.of("__time", "m1")) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .build() ), ImmutableList.of( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 721eec21900b..5161fa321148 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -34,6 +34,8 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.scan.Order; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -585,8 +587,8 @@ public void testReplaceWithClusteredBy() .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT)) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING) + new OrderBy("v0", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING) ) ) .context( @@ -759,7 +761,7 @@ public void testExplainReplaceTimeChunksWithPartitioningAndClustering() throws I .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING) + new OrderBy("dim1", Order.ASCENDING) ) ) .columnTypes(LONG, LONG, STRING, STRING, STRING, FLOAT, DOUBLE, ofComplex("hyperUnique")) @@ -842,10 +844,10 @@ public void testExplainReplaceWithLimitAndClusteredByOrdinals() throws IOExcepti .limit(10) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("__time", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim3", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim2", ScanQuery.Order.ASCENDING) + new OrderBy("__time", Order.ASCENDING), + new OrderBy("dim1", Order.ASCENDING), + new OrderBy("dim3", Order.ASCENDING), + new OrderBy("dim2", Order.ASCENDING) ) ) .columnTypes(LONG, LONG, STRING, STRING, STRING, FLOAT, DOUBLE, ColumnType.ofComplex("hyperUnique")) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index 6a0f1742155c..9e003eac3a46 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -40,6 +40,7 @@ import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.topn.DimensionTopNMetricSpec; @@ -1443,7 +1444,7 @@ public void testSelectStarWithLimitTimeDescending() .intervals(querySegmentSpec(Filtration.eternity())) .columns(ImmutableList.of("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1")) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1469,7 +1470,7 @@ public void testSelectStarWithoutLimitTimeAscending() .intervals(querySegmentSpec(Filtration.eternity())) .columns(ImmutableList.of("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1")) .limit(Long.MAX_VALUE) - .order(ScanQuery.Order.ASCENDING) + .order(Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1519,7 +1520,7 @@ public void testSelectSingleColumnWithLimitDescending() .intervals(querySegmentSpec(Filtration.eternity())) .columns(ImmutableList.of("__time", "dim1")) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1545,7 +1546,7 @@ public void testSelectStarFromSelectSingleColumnWithLimitDescending() .intervals(querySegmentSpec(Filtration.eternity())) .columns(ImmutableList.of("dim1")) .limit(2) - .order(ScanQuery.Order.NONE) + .order(Order.NONE) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1569,7 +1570,7 @@ public void testSelectLimitWrapping() .intervals(querySegmentSpec(Filtration.eternity())) .columns(ImmutableList.of("__time", "dim1")) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(OUTER_LIMIT_CONTEXT) .build() @@ -1594,7 +1595,7 @@ public void testSelectLimitWrappingOnTopOfOffset() .columns(ImmutableList.of("__time", "dim1")) .offset(1) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(OUTER_LIMIT_CONTEXT) .build() @@ -1619,7 +1620,7 @@ public void testSelectLimitWrappingOnTopOfOffsetAndLowLimit() .columns(ImmutableList.of("__time", "dim1")) .offset(1) .limit(1) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(OUTER_LIMIT_CONTEXT) .build() @@ -1643,7 +1644,7 @@ public void testSelectLimitWrappingOnTopOfOffsetAndHighLimit() .columns(ImmutableList.of("__time", "dim1")) .offset(1) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(OUTER_LIMIT_CONTEXT) .build() @@ -1815,7 +1816,7 @@ public void testSelectProjectionFromSelectSingleColumnWithInnerLimitDescending() .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1843,7 +1844,7 @@ public void testSelectProjectionFromSelectSingleColumnDescending() .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ColumnType.STRING)) .columns(ImmutableList.of("v0")) - .order(ScanQuery.Order.NONE) + .order(Order.NONE) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1871,7 +1872,7 @@ public void testSelectProjectionFromSelectSingleColumnWithInnerAndOuterLimitDesc .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .limit(2) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -1899,7 +1900,7 @@ public void testOrderThenLimitThenFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns(ImmutableList.of("__time", "dim1")) .limit(4) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index c1236162fa02..e81919283fd4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -62,6 +62,7 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.topn.DimensionTopNMetricSpec; import org.apache.druid.query.topn.TopNQueryBuilder; @@ -1305,7 +1306,7 @@ public void testSingleValueStringAgg(String testName, Map queryC .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .offset(6L) .limit(1L) - .order(ScanQuery.Order.DESCENDING) + .order(Order.DESCENDING) .columns("__time", "channel") .context(QUERY_CONTEXT_DEFAULT) .build() From 2782511b8fdc1d4b4719b8f0217b07abd8f8cdef Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 11:51:04 -0700 Subject: [PATCH 35/74] javadoc --- .../src/main/java/org/apache/druid/segment/Cursor.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 3c41dbb6aa82..4b340746d208 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -88,8 +88,10 @@ public interface Cursor /** * Mark a position on the cursor which can recalled with {@link #resetToMark()}. This method is used by * {@link org.apache.druid.query.topn.TopNQueryEngine} when computing results in query granularity buckets. The - * {@link DateTime} parameter supplied to this method corresponds to the start of the bucket interval. Cursor - * implementations may use this if useful, or ignore it and mark the current offset if not. + * {@link DateTime} parameter supplied to this method corresponds to the start of the bucket interval. The mark is + * not guaranteed to be a row with the specified timestamp, rather it is to assist to seeking to the actual offset + * at the time of marking for cursor implementations which do not have random access offsets or might be mutable. + * Cursor implementations may use this parameter if useful, or ignore it and mark the current offset if not. */ void mark(DateTime mark); From e678ab4651bdcfdcea0e832cf449e8801c9674f6 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 12:15:36 -0700 Subject: [PATCH 36/74] fixes --- .../segment/columnar/FrameCursorMakerFactory.java | 6 +++--- .../main/java/org/apache/druid/segment/Cursor.java | 12 ++++++------ .../org/apache/druid/segment/RowBasedCursor.java | 3 +-- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java index b7dfd2cf597b..0067d2540b36 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java @@ -94,14 +94,14 @@ public Cursor asCursor() final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); final ColumnCache columnCache = new ColumnCache(index, closer); final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); - - final SimpleSettableOffset baseOffset = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()) + final boolean descendingTimeOrder = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()); + final SimpleSettableOffset baseOffset = descendingTimeOrder ? new SimpleDescendingOffset(frame.numRows()) : new SimpleAscendingOffset(frame.numRows()); final QueryableIndexColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( spec.getVirtualColumns(), - spec.isPreferDescendingTimeOrder(), + descendingTimeOrder, baseOffset, columnCache ); diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 4b340746d208..43a3df99acb3 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -86,12 +86,12 @@ public interface Cursor boolean isDoneOrInterrupted(); /** - * Mark a position on the cursor which can recalled with {@link #resetToMark()}. This method is used by - * {@link org.apache.druid.query.topn.TopNQueryEngine} when computing results in query granularity buckets. The - * {@link DateTime} parameter supplied to this method corresponds to the start of the bucket interval. The mark is - * not guaranteed to be a row with the specified timestamp, rather it is to assist to seeking to the actual offset - * at the time of marking for cursor implementations which do not have random access offsets or might be mutable. - * Cursor implementations may use this parameter if useful, or ignore it and mark the current offset if not. + * Mark a position on the cursor which can recalled with {@link #resetToMark()}. The {@link DateTime} parameter + * supplied to this method corresponds to the start of the bucket interval and acts as a lower/upper bound on the + * first possible timestamp, it is to assist to seeking to the actual offset at the time of marking for cursor + * implementations which do not have random access offsets or might be mutable. Cursor implementations may use this + * parameter if useful, or ignore it and mark the current offset if not. This method is used by + * {@link org.apache.druid.query.topn.TopNQueryEngine} when computing results in query granularity buckets. */ void mark(DateTime mark); diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index b9df2754b557..dd24b15135bb 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -126,10 +126,9 @@ public void mark(DateTime mark) @Override public void resetToMark() { - rowId = markId; rowWalker.reset(); + rowId = markId; rowWalker.skipToDateTime(markDate, descending); - advanceToMatchingRow(); } @Override From 45e1cc40578fe56b068e9937d2e797ba454a6234 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 14:15:32 -0700 Subject: [PATCH 37/74] fix --- .../druid/frame/segment/columnar/FrameCursorMakerFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java index 0067d2540b36..2a3bf4b7c3aa 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java @@ -85,7 +85,7 @@ public boolean canVectorize() { return (spec.getFilter() == null || spec.getFilter().canVectorizeMatcher(signature)) && spec.getVirtualColumns().canVectorize(signature) - && CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()); + && !CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()); } @Override From 4c2d9762c44280a7150ff5ef5cd9780961415aef Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 14:45:05 -0700 Subject: [PATCH 38/74] adjust --- .../main/java/org/apache/druid/segment/RowBasedCursor.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index dd24b15135bb..5ccabd6355be 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -127,8 +127,12 @@ public void mark(DateTime mark) public void resetToMark() { rowWalker.reset(); - rowId = markId; + rowId = 0; rowWalker.skipToDateTime(markDate, descending); + while (!isDone() && !valueMatcher.matches(false) && rowId < markId) { + rowWalker.advance(); + rowId++; + } } @Override From c1f6ca5d5aab3ae5e99496cd9fcaf576156c24f3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 14:51:44 -0700 Subject: [PATCH 39/74] adjust --- .../apache/druid/segment/RowBasedCursor.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index 5ccabd6355be..d2ad38b5c093 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -41,12 +41,12 @@ public class RowBasedCursor implements Cursor private final ToLongFunction timestampFunction; private final Interval interval; private final boolean descending; + private DateTime startTime; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher valueMatcher; private long rowId = 0; private long markId = 0; - private DateTime markDate; public RowBasedCursor( final RowWalker rowWalker, @@ -62,6 +62,7 @@ public RowBasedCursor( this.timestampFunction = rowAdapter.timestampFunction(); this.interval = interval; this.descending = descending; + this.startTime = descending ? interval.getEnd().minus(1) : interval.getStart(); this.columnSelectorFactory = virtualColumns.wrap( new RowBasedColumnSelectorFactory<>( rowWalker::currentRow, @@ -78,8 +79,7 @@ public RowBasedCursor( } else { this.valueMatcher = filter.makeMatcher(this.columnSelectorFactory); } - - rowWalker.skipToDateTime(descending ? interval.getEnd().minus(1) : interval.getStart(), descending); + rowWalker.skipToDateTime(startTime, descending); advanceToMatchingRow(); } @@ -120,29 +120,24 @@ public boolean isDoneOrInterrupted() public void mark(DateTime mark) { markId = rowId; - markDate = mark; } @Override public void resetToMark() { - rowWalker.reset(); rowId = 0; - rowWalker.skipToDateTime(markDate, descending); - while (!isDone() && !valueMatcher.matches(false) && rowId < markId) { - rowWalker.advance(); - rowId++; - } + rowWalker.reset(); + rowWalker.skipToDateTime(startTime, descending); + advanceToMatchingMarkRow(); } @Override public void reset() - { + {g rowId = 0; markId = 0; - markDate = descending ? interval.getEnd().minus(1) : interval.getStart(); rowWalker.reset(); - rowWalker.skipToDateTime(markDate, descending); + rowWalker.skipToDateTime(startTime, descending); advanceToMatchingRow(); } @@ -153,4 +148,12 @@ private void advanceToMatchingRow() rowId++; } } + + private void advanceToMatchingMarkRow() + { + while (!isDone() && !valueMatcher.matches(false) && rowId < markId) { + rowWalker.advance(); + rowId++; + } + } } From 6532b13c996baaa712ccac5cd7b099e5912878c3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 15:38:59 -0700 Subject: [PATCH 40/74] oops --- .../src/main/java/org/apache/druid/segment/RowBasedCursor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index d2ad38b5c093..ebf6fe5943a8 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -133,7 +133,7 @@ public void resetToMark() @Override public void reset() - {g + { rowId = 0; markId = 0; rowWalker.reset(); From 4b20621ee6f469805aca0083dc34c6eb5aed009a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 16:26:18 -0700 Subject: [PATCH 41/74] fix --- .../IncrementalIndexCursorHolder.java | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 382ead7200d6..3c2482660939 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -89,7 +89,7 @@ static class IncrementalIndexCursor implements Cursor private int numAdvanced; private boolean done; private DateTime markDate; - private int markAdvanced = 0; + private int markRowId = 0; IncrementalIndexCursor( IncrementalIndexStorageAdapter storageAdapter, @@ -201,26 +201,44 @@ public boolean isDoneOrInterrupted() public void mark(DateTime mark) { markDate = mark; - markAdvanced = numAdvanced; + markRowId = currEntry.get() != null ? currEntry.get().getRowIndex() : -1; } @Override public void resetToMark() { - numAdvanced = markAdvanced; + numAdvanced = markRowId; baseIter = facts.timeRangeIterable( isDescending, isDescending ? interval.getStartMillis() : markDate.getMillis(), isDescending ? markDate.getMillis() : interval.getEndMillis() ).iterator(); - seekNextOffset(); + BaseQuery.checkInterrupted(); + + boolean foundMatched = false; + while (baseIter.hasNext()) { + IncrementalIndexRow entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { + numAdvanced++; + continue; + } + currEntry.set(entry); + if (filterMatcher.matches(false) && (markRowId < 0 || entry.getRowIndex() == markRowId)) { + foundMatched = true; + break; + } + + numAdvanced++; + } + + done = !foundMatched && (emptyRange || !baseIter.hasNext()); } @Override public void reset() { - markAdvanced = 0; + markRowId = 0; markDate = isDescending ? interval.getEnd() : interval.getStart(); baseIter = cursorIterable.iterator(); @@ -230,11 +248,6 @@ public void reset() Iterators.advance(baseIter, numAdvanced); } - seekNextOffset(); - } - - private void seekNextOffset() - { BaseQuery.checkInterrupted(); boolean foundMatched = false; From c42be761e3dd97fcf44e44ba62edadcf3bf9b32a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 5 Aug 2024 17:16:29 -0700 Subject: [PATCH 42/74] simplify --- .../druid/frame/segment/FrameCursor.java | 3 +- .../druid/query/topn/TopNQueryEngine.java | 2 +- .../java/org/apache/druid/segment/Cursor.java | 12 +-- .../segment/QueryableIndexCursorHolder.java | 3 +- .../apache/druid/segment/RowBasedCursor.java | 2 +- .../UnnestColumnValueSelectorCursor.java | 5 +- .../druid/segment/UnnestDimensionCursor.java | 5 +- .../IncrementalIndexCursorHolder.java | 36 ++++--- .../IncrementalIndexRowHolder.java | 1 + .../druid/segment/join/HashJoinEngine.java | 5 +- .../druid/segment/join/PostJoinCursor.java | 5 +- .../druid/frame/TestArrayStorageAdapter.java | 5 +- .../testutil/RowNumberUpdatingCursor.java | 5 +- .../org/apache/druid/segment/ListCursor.java | 3 +- .../IncrementalIndexStorageAdapterTest.java | 97 +++++++++++++++++++ .../segment/join/PostJoinCursorTest.java | 5 +- .../selector/TestColumnValueSelector.java | 2 +- 17 files changed, 141 insertions(+), 55 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index c67c2b0e671c..2155aad7247e 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -24,7 +24,6 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; -import org.joda.time.DateTime; /** * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorMakerFactory} @@ -80,7 +79,7 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { markOffset = offset.getOffset(); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index 7f9254a1ac52..e6d7830e6559 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -96,7 +96,7 @@ public Sequence> query( Sequences.simple(granularizer.getBucketIterable()) .map(bucketInterval -> { granularizer.advanceToBucket(bucketInterval); - cursor.mark(bucketInterval.getStart()); + cursor.mark(); return mapFn.apply(cursor, granularizer, queryMetrics); }), Predicates.notNull() diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 43a3df99acb3..d53b87567536 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -21,7 +21,6 @@ import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.segment.incremental.IncrementalIndexCursorHolder; -import org.joda.time.DateTime; /** * Cursor is an interface for iteration over a range of data points, used during query execution. Cursors are available @@ -86,17 +85,12 @@ public interface Cursor boolean isDoneOrInterrupted(); /** - * Mark a position on the cursor which can recalled with {@link #resetToMark()}. The {@link DateTime} parameter - * supplied to this method corresponds to the start of the bucket interval and acts as a lower/upper bound on the - * first possible timestamp, it is to assist to seeking to the actual offset at the time of marking for cursor - * implementations which do not have random access offsets or might be mutable. Cursor implementations may use this - * parameter if useful, or ignore it and mark the current offset if not. This method is used by - * {@link org.apache.druid.query.topn.TopNQueryEngine} when computing results in query granularity buckets. + * Mark a position on the cursor at the current row, which can recalled with {@link #resetToMark()}. */ - void mark(DateTime mark); + void mark(); /** - * Reset to position set by {@link #mark(DateTime)} + * Reset to position set by {@link #mark()} */ void resetToMark(); diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 626255bce89f..487a50319286 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -51,7 +51,6 @@ import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.vector.VectorOffset; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -455,7 +454,7 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { cursorOffset.mark(); } diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index ebf6fe5943a8..a82d74da892f 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -117,7 +117,7 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { markId = rowId; } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index 13a3a701a2bf..76e30b4a9270 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -24,7 +24,6 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Arrays; @@ -243,10 +242,10 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { markIndex = index; - baseCursor.mark(mark); + baseCursor.mark(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index e6773d9095f0..7f4a8aa3806e 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -292,10 +291,10 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { markIndex = index; - baseCursor.mark(mark); + baseCursor.mark(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 3c2482660939..8b90b367ec4c 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.filter.ValueMatchers; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -88,8 +87,8 @@ static class IncrementalIndexCursor implements Cursor private boolean emptyRange; private int numAdvanced; private boolean done; - private DateTime markDate; - private int markRowId = 0; + private int markRowId = -1; + private long markMillis = -1; IncrementalIndexCursor( IncrementalIndexStorageAdapter storageAdapter, @@ -114,7 +113,6 @@ static class IncrementalIndexCursor implements Cursor facts = index.getFacts(); interval = actualInterval; isDescending = descending; - markDate = isDescending ? interval.getEnd() : interval.getStart(); cursorIterable = facts.timeRangeIterable( descending, actualInterval.getStartMillis(), @@ -198,20 +196,29 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { - markDate = mark; - markRowId = currEntry.get() != null ? currEntry.get().getRowIndex() : -1; + if (!done && currEntry.get() != null) { + IncrementalIndexRow row = currEntry.get(); + markRowId = row.getRowIndex(); + markMillis = row.getTimestamp(); + } else { + markRowId = -1; + markMillis = isDescending ? Long.MAX_VALUE : Long.MIN_VALUE; + } } @Override public void resetToMark() { - numAdvanced = markRowId; + if (markRowId < 0) { + reset(); + return; + } baseIter = facts.timeRangeIterable( isDescending, - isDescending ? interval.getStartMillis() : markDate.getMillis(), - isDescending ? markDate.getMillis() : interval.getEndMillis() + isDescending ? interval.getStartMillis() : markMillis, + isDescending ? markMillis : interval.getEndMillis() ).iterator(); BaseQuery.checkInterrupted(); @@ -220,16 +227,13 @@ public void resetToMark() while (baseIter.hasNext()) { IncrementalIndexRow entry = baseIter.next(); if (beyondMaxRowIndex(entry.getRowIndex())) { - numAdvanced++; continue; } currEntry.set(entry); - if (filterMatcher.matches(false) && (markRowId < 0 || entry.getRowIndex() == markRowId)) { + if (entry.getRowIndex() == markRowId && filterMatcher.matches(false)) { foundMatched = true; break; } - - numAdvanced++; } done = !foundMatched && (emptyRange || !baseIter.hasNext()); @@ -238,8 +242,8 @@ public void resetToMark() @Override public void reset() { - markRowId = 0; - markDate = isDescending ? interval.getEnd() : interval.getStart(); + markRowId = -1; + markMillis = isDescending ? Long.MAX_VALUE : Long.MIN_VALUE; baseIter = cursorIterable.iterator(); if (numAdvanced == -1) { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowHolder.java index c46ef559b00f..f8841efae151 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowHolder.java @@ -38,6 +38,7 @@ public class IncrementalIndexRowHolder implements LongColumnSelector @Nullable private IncrementalIndexRow currEntry = null; + @Nullable public IncrementalIndexRow get() { return currEntry; diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index 2666552ee974..a29dde8a9c01 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -28,7 +28,6 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowIdSupplier; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -246,9 +245,9 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { - leftCursor.mark(mark); + leftCursor.mark(); joinMarkId = joinColumnSelectorFactory.getRowId(); } diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 4195dc4975bd..264f25c67932 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -26,7 +26,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -141,9 +140,9 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { - baseCursor.mark(mark); + baseCursor.mark(); } @Override diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index ca2f33cca591..6a4e1e616833 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -38,7 +38,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -210,9 +209,9 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { - cursor.mark(mark); + cursor.mark(); } @Override diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 27a0688032a2..00001525456d 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -22,7 +22,6 @@ import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.joda.time.DateTime; /** * Used by {@link FrameTestUtil#readRowsFromAdapter} and {@link FrameTestUtil#readRowsFromCursor}. @@ -72,9 +71,9 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { - baseCursor.mark(mark); + baseCursor.mark(); markRowNumber = rowNumberVirtualColumn.getValue(); } diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index d781f5893158..30b04fbb724d 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -26,7 +26,6 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.List; @@ -222,7 +221,7 @@ public void resetToMark() } @Override - public void mark(DateTime mark) + public void mark() { markIndex = index; } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index d74916b25dbf..cd8f5d5004be 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.incremental; import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -66,6 +67,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.filter.OrFilter; import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; @@ -319,6 +321,101 @@ public void testResetSanity() throws IOException } } + @Test + public void testMarkResetMarkSanity() throws IOException + { + + IncrementalIndex index = indexCreator.createIndex(); + DateTime t = DateTimes.nowUtc(); + Interval interval = new Interval(t.minusMinutes(1), t.plusMinutes(1)); + + index.add( + new MapBasedInputRow( + t.minus(1).getMillis(), + Collections.singletonList("billy"), + ImmutableMap.of("billy", "hi") + ) + ); + index.add( + new MapBasedInputRow( + t.minus(1).getMillis(), + Collections.singletonList("sally"), + ImmutableMap.of("sally", "bo") + ) + ); + + index.add( + new MapBasedInputRow( + t.getMillis(), + Collections.singletonList("billy"), + ImmutableMap.of("billy", "hi2") + ) + ); + index.add( + new MapBasedInputRow( + t.getMillis(), + Collections.singletonList("sally"), + ImmutableMap.of("sally", "bo2") + ) + ); + + IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + + for (boolean descending : Arrays.asList(false, true)) { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter( + new OrFilter( + ImmutableList.of( + new SelectorFilter("sally", "bo"), + new SelectorFilter("sally", "bo2") + ) + ) + ) + .setInterval(interval) + .setPreferredOrdering( + descending ? + Collections.singletonList( + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) + ) : + null + ) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + DimensionSelector dimSelector; + dimSelector = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals(descending ? "bo2" : "bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + cursor.advance(); + cursor.mark(); + Assert.assertEquals(descending ? "bo" : "bo2", dimSelector.lookupName(dimSelector.getRow().get(0))); + index.add( + new MapBasedInputRow( + t.minus(1).getMillis(), + Collections.singletonList("sally"), + ImmutableMap.of("sally", "ah") + ) + ); + + // Cursor mark and resetToMark should not be affected by out of order values + cursor.resetToMark(); + + dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals(descending ? "bo" : "bo2", dimSelector.lookupName(dimSelector.getRow().get(0))); + + // Cursor reset should not be affected by mark/reset mark either + cursor.reset(); + + dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals(descending ? "bo2" : "bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + } + } + } + @Test public void testSingleValueTopN() throws IOException { diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index c7f12d4ec190..80e7c9c87322 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -34,7 +34,6 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; -import org.joda.time.DateTime; import org.junit.Test; import javax.annotation.Nullable; @@ -135,9 +134,9 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { - cursor.mark(mark); + cursor.mark(); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index 84be585155f9..01f6f8cba39b 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -115,7 +115,7 @@ public boolean isDoneOrInterrupted() } @Override - public void mark(DateTime mark) + public void mark() { markCounter = rowCounter; } From a995140dcf650c7f6aa171f37a85877c9aa82779 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 00:06:31 -0700 Subject: [PATCH 43/74] fixes and test --- .../apache/druid/segment/RowBasedCursor.java | 2 +- .../segment/RowBasedStorageAdapterTest.java | 63 ++++++++++++++++--- 2 files changed, 57 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index a82d74da892f..281d22bc33fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -151,7 +151,7 @@ private void advanceToMatchingRow() private void advanceToMatchingMarkRow() { - while (!isDone() && !valueMatcher.matches(false) && rowId < markId) { + while (!isDone() && rowId < markId) { rowWalker.advance(); rowId++; } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 36650c65bcec..b6ee4a352ba9 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -451,6 +451,28 @@ public void test_getMetadata() assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata()); } + @Test + public void test_makeCursor() + { + final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("0"), + ImmutableList.of("1"), + ImmutableList.of("2") + ), + walkCursor(cursor, READ_STRING) + ); + } + + Assert.assertEquals(3, numCloses.get()); + } + @Test public void test_makeCursor_filterOnLong() { @@ -471,7 +493,7 @@ public void test_makeCursor_filterOnLong() } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } @Test @@ -494,7 +516,7 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNull() ); } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } @Test @@ -528,7 +550,7 @@ public void test_makeCursor_filterOnVirtualColumn() ); } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } @Test @@ -574,7 +596,7 @@ public void test_makeCursor_intervalDoesNotMatch() ); } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } @Test @@ -595,7 +617,7 @@ public void test_makeCursor_intervalPartiallyMatches() ); } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } @Test @@ -779,7 +801,7 @@ public void test_makeCursor_allProcessors() ); } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } @Test @@ -799,7 +821,7 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() ); } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(3, numCloses.get()); } private static List> walkCursor( @@ -815,6 +837,16 @@ private static List> walkCursor( final List> retVal = new ArrayList<>(); while (!cursor.isDone()) { + cursor.advanceUninterruptibly(); + } + + cursor.reset(); + + int ctr = 0; + while (!cursor.isDone()) { + if (ctr == 1) { + cursor.mark(); + } final List row = new ArrayList<>(); for (Supplier supplier : suppliers) { @@ -822,9 +854,26 @@ private static List> walkCursor( } retVal.add(row); + ctr++; cursor.advanceUninterruptibly(); } + cursor.resetToMark(); + int mark = 1; + if (ctr > 1) { + while (!cursor.isDone()) { + + final List row = new ArrayList<>(); + + for (Supplier supplier : suppliers) { + row.add(supplier.get()); + } + + retVal.set(mark++, row); + cursor.advanceUninterruptibly(); + } + } + return retVal; } From dc62f10c8e3f37eea0d4d2db4e20a2e2e9cc7bd7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 00:41:50 -0700 Subject: [PATCH 44/74] more test more fix --- .../druid/segment/join/HashJoinEngine.java | 10 +++-- .../segment/RowBasedStorageAdapterTest.java | 6 +-- .../druid/segment/join/JoinTestHelper.java | 39 +++++++++++++++++++ 3 files changed, 49 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index a29dde8a9c01..18c9cbfdef96 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -247,16 +247,19 @@ public boolean isDoneOrInterrupted() @Override public void mark() { - leftCursor.mark(); joinMarkId = joinColumnSelectorFactory.getRowId(); } @Override public void resetToMark() { - leftCursor.resetToMark(); + leftCursor.reset(); joinMatcher.reset(); - joinColumnSelectorFactory.rowId = joinMarkId; + joinColumnSelectorFactory.resetRowId(); + initialize(); + while (!isDone() && joinColumnSelectorFactory.getRowId() < joinMarkId) { + advance(); + } } @Override @@ -266,6 +269,7 @@ public void reset() joinMatcher.reset(); joinColumnSelectorFactory.resetRowId(); joinMarkId = joinColumnSelectorFactory.getRowId(); + initialize(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index b6ee4a352ba9..0812daa36e3b 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -843,8 +843,9 @@ private static List> walkCursor( cursor.reset(); int ctr = 0; + int mark = 1; while (!cursor.isDone()) { - if (ctr == 1) { + if (ctr == mark) { cursor.mark(); } final List row = new ArrayList<>(); @@ -858,9 +859,8 @@ private static List> walkCursor( cursor.advanceUninterruptibly(); } - cursor.resetToMark(); - int mark = 1; if (ctr > 1) { + cursor.resetToMark(); while (!cursor.isDone()) { final List row = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 3518be0af022..e3b8b4864bec 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -343,7 +343,25 @@ public static List readCursor(final CursorHolder cursorHolder, final L final List rows = new ArrayList<>(); boolean interruptible = false; // test both advance() and advanceUninterruptibly() + while (!cursor.isDone()) { + if (interruptible) { + cursor.advance(); + } else { + cursor.advanceUninterruptibly(); + } + + interruptible = !interruptible; + } + + cursor.reset(); + + int ctr = 0; + int mark = 2; + while (!cursor.isDone()) { + if (ctr == mark) { + cursor.mark(); + } final Object[] row = new Object[columns.size()]; for (int i = 0; i < row.length; i++) { @@ -358,6 +376,27 @@ public static List readCursor(final CursorHolder cursorHolder, final L } interruptible = !interruptible; + ctr++; + } + + if (rows.size() > mark) { + cursor.resetToMark(); + while (!cursor.isDone()) { + final Object[] row = new Object[columns.size()]; + + for (int i = 0; i < row.length; i++) { + row[i] = readers.get(i).get(); + } + + rows.set(mark++, row); + if (interruptible) { + cursor.advance(); + } else { + cursor.advanceUninterruptibly(); + } + + interruptible = !interruptible; + } } return rows; From 58677427c579eeac562919f482cde9c570ebb96a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 01:30:08 -0700 Subject: [PATCH 45/74] more test --- .../segment/UnnestStorageAdapterTest.java | 70 ++++++++++++++++++- 1 file changed, 67 insertions(+), 3 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index 229e337be523..01b5c6c85967 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -62,6 +62,7 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -238,19 +239,82 @@ public void test_unnest_adapters_basic() DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); int count = 0; + + List rows = new ArrayList<>(); while (!cursor.isDone()) { - Object dimSelectorVal = dimSelector.getObject(); - if (dimSelectorVal == null) { - Assert.assertNull(dimSelectorVal); + cursor.advance(); + } + cursor.reset(); + + int mark = 2; + while (!cursor.isDone()) { + if (count == mark) { + cursor.mark(); } + Object dimSelectorVal = dimSelector.getObject(); + rows.add(dimSelectorVal); cursor.advance(); count++; + } + cursor.resetToMark(); + while (!cursor.isDone()) { + Object dimSelectorVal = dimSelector.getObject(); + rows.set(mark++, dimSelectorVal); + cursor.advance(); } /* each row has 8 entries. unnest 2 rows -> 16 rows after unnest */ Assert.assertEquals(count, 16); + Assert.assertEquals( + Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "10", "11", "12", "13", "14", "15", "8", "9"), + rows + ); + } + } + + @Test + public void test_unnest_adapters_basic_array_column() + { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(UNNEST_ARRAYS.getInterval()) + .build(); + try (final CursorHolder cursorHolder = UNNEST_ARRAYS.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + + ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector dimSelector = factory.makeColumnValueSelector("u"); + int count = 0; + + List rows = new ArrayList<>(); + while (!cursor.isDone()) { + cursor.advance(); + } + cursor.reset(); + + int mark = 2; + while (!cursor.isDone()) { + if (count == mark) { + cursor.mark(); + } + Object dimSelectorVal = dimSelector.getObject(); + rows.add(dimSelectorVal); + cursor.advance(); + count++; + } + cursor.resetToMark(); + while (!cursor.isDone()) { + Object dimSelectorVal = dimSelector.getObject(); + rows.set(mark++, dimSelectorVal); + cursor.advance(); + } + Assert.assertEquals(count, 12); + Assert.assertEquals( + Arrays.asList(2L, 3L, 1L, null, 3L, 1L, null, 2L, 9L, 1L, 2L, 3L), + rows + ); } } From 4a55d866a479843e5a236ff09f234cb56aa54702 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 02:32:59 -0700 Subject: [PATCH 46/74] fix test --- .../druid/segment/RowBasedStorageAdapterTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 0812daa36e3b..eba63db170cd 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -493,7 +493,7 @@ public void test_makeCursor_filterOnLong() } - Assert.assertEquals(3, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test @@ -550,7 +550,7 @@ public void test_makeCursor_filterOnVirtualColumn() ); } - Assert.assertEquals(3, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test @@ -596,7 +596,7 @@ public void test_makeCursor_intervalDoesNotMatch() ); } - Assert.assertEquals(3, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test @@ -617,7 +617,7 @@ public void test_makeCursor_intervalPartiallyMatches() ); } - Assert.assertEquals(3, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test @@ -821,7 +821,7 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() ); } - Assert.assertEquals(3, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } private static List> walkCursor( From dfba713cfc1dcbdf82569a907d72c29f5f6baaba Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 12:36:22 -0700 Subject: [PATCH 47/74] fix compile --- .../test/java/org/apache/druid/msq/exec/MSQWindowTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 3fbd8c72049c..2c39d1366034 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -51,6 +51,7 @@ import org.apache.druid.query.operator.window.WindowFramedAggregateProcessor; import org.apache.druid.query.operator.window.WindowOperatorFactory; import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor; +import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.segment.column.ColumnType; @@ -2135,9 +2136,9 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers(String cont .columns("d0", "d1", "d2", "w0", "w1") .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("d0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("d1", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("d2", ScanQuery.Order.ASCENDING) + OrderBy.ascending("d0"), + OrderBy.ascending("d1"), + OrderBy.ascending("d2") ) ) .columnTypes(ColumnType.STRING, ColumnType.STRING, ColumnType.STRING, ColumnType.LONG, ColumnType.LONG) From e9a64dce27845dd24b13bad7fcd6a28c3b74696d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 12:47:36 -0700 Subject: [PATCH 48/74] nicer --- .../apache/druid/msq/exec/MSQArraysTest.java | 7 ++-- .../druid/msq/exec/MSQComplexGroupByTest.java | 3 +- .../druid/msq/exec/MSQLoadedSegmentTests.java | 3 +- .../apache/druid/msq/exec/MSQSelectTest.java | 3 +- .../msq/indexing/MSQCompactionRunnerTest.java | 9 +++-- .../druid/query/scan/ScanQuerySpecTest.java | 6 ++-- .../druid/query/scan/ScanQueryTest.java | 10 +++--- .../CalciteCatalogIngestionDmlTest.java | 9 +++-- .../sql/calcite/CalciteInsertDmlTest.java | 33 +++++++++---------- .../sql/calcite/CalciteReplaceDmlTest.java | 19 ++++------- 10 files changed, 45 insertions(+), 57 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index 17e01b4fb4df..3a3b00202979 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -38,7 +38,6 @@ import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.Query; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -863,7 +862,7 @@ public void testScanWithOrderByOnStringArray(String contextName, Map cont .intervals(querySegmentSpec(Filtration.eternity())) .columns("obj") .context(defaultScanQueryContext(context, rowSignature)) - .orderBy(Collections.singletonList(new OrderBy("obj", Order.ASCENDING))) + .orderBy(Collections.singletonList(OrderBy.ascending("obj"))) .build() ) .columnMappings(new ColumnMappings(ImmutableList.of( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index c3d617545eba..c1d4b064f81f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -37,7 +37,6 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -191,7 +190,7 @@ public void testSelectWithLoadedSegmentsOnFooWithOrderBy() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .columns("cnt", "dim1") - .orderBy(ImmutableList.of(new OrderBy("dim1", Order.ASCENDING))) + .orderBy(ImmutableList.of(OrderBy.ascending("dim1"))) .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) .build() ) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index b1681e5c717e..a2193e1a391d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -67,7 +67,6 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; @@ -465,7 +464,7 @@ public void testSelectAndOrderByOnFooWhereMatchesNoData(String contextName, Map< .columns("cnt", "dim1") .filters(equality("dim2", "nonexistent", ColumnType.STRING)) .context(defaultScanQueryContext(context, resultSignature)) - .orderBy(ImmutableList.of(new OrderBy("dim1", Order.ASCENDING))) + .orderBy(ImmutableList.of(OrderBy.ascending("dim1"))) .build() ) .columnMappings(ColumnMappings.identity(resultSignature)) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 8077c0b55556..f6d7fca2e8ad 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -54,7 +54,6 @@ import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.IndexSpec; @@ -305,10 +304,10 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce ); Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); - Assert.assertEquals(PARTITION_DIMENSIONS.stream().map(col -> new OrderBy( - col, - Order.ASCENDING - )).collect(Collectors.toList()), ((ScanQuery) actualMSQSpec.getQuery()).getOrderBys()); + Assert.assertEquals( + PARTITION_DIMENSIONS.stream().map(OrderBy::ascending).collect(Collectors.toList()), + ((ScanQuery) actualMSQSpec.getQuery()).getOrderBys() + ); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index 82886c705a3a..06b530bbdbe9 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -98,7 +98,7 @@ public void testSerializationWithTimeOrder() throws Exception Assert.assertEquals(expectedQuery, JSON_MAPPER.readValue(originalJson, ScanQuery.class)); Assert.assertEquals(Order.ASCENDING, expectedQuery.getTimeOrder()); Assert.assertEquals( - Collections.singletonList(new OrderBy("__time", Order.ASCENDING)), + Collections.singletonList(OrderBy.ascending("__time")), expectedQuery.getOrderBys() ); } @@ -118,7 +118,7 @@ public void testSerializationWithOrderBy() throws Exception 0, 3, null, - Collections.singletonList(new OrderBy("quality", Order.ASCENDING)), + Collections.singletonList(OrderBy.ascending("quality")), null, Arrays.asList("market", "quality", "index", "__time"), null, @@ -130,7 +130,7 @@ public void testSerializationWithOrderBy() throws Exception Assert.assertEquals(expectedQuery, JSON_MAPPER.readValue(originalJson, ScanQuery.class)); Assert.assertEquals(Order.NONE, expectedQuery.getTimeOrder()); Assert.assertEquals( - Collections.singletonList(new OrderBy("quality", Order.ASCENDING)), + Collections.singletonList(OrderBy.ascending("quality")), expectedQuery.getOrderBys() ); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index 54d736ec26db..5995a640a6e1 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -153,8 +153,8 @@ public void testConflictingOrderByAndTimeOrder() .orderBy( // Not ok, even though it starts with __time ASC, because it also has non-time component. ImmutableList.of( - new OrderBy("__time", Order.ASCENDING), - new OrderBy("quality", Order.DESCENDING) + OrderBy.ascending("__time"), + OrderBy.descending("quality") ) ) .columns(ImmutableList.of("__time", "quality")) @@ -170,7 +170,7 @@ public void testCompatibleOrderByAndTimeOrder() Assert.assertNotNull( Druids.newScanQueryBuilder() .order(Order.ASCENDING) - .orderBy(ImmutableList.of(new OrderBy("__time", Order.ASCENDING))) + .orderBy(ImmutableList.of(OrderBy.ascending("__time"))) .columns(ImmutableList.of("__time", "quality")) .dataSource("source") .intervals(intervalSpec) @@ -340,7 +340,7 @@ public void testGetResultOrderingWithTimeBasedOrderBy() final ScanQuery scanQuery = Druids.newScanQueryBuilder() .columns("__time") - .orderBy(Collections.singletonList(new OrderBy("__time", Order.DESCENDING))) + .orderBy(Collections.singletonList(OrderBy.descending("__time"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -356,7 +356,7 @@ public void testGetResultOrderingWithNonTimeOrderBy() final ScanQuery scanQuery = Druids.newScanQueryBuilder() .columns("quality") - .orderBy(Collections.singletonList(new OrderBy("quality", Order.ASCENDING))) + .orderBy(Collections.singletonList(OrderBy.ascending("quality"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java index 0ec629c8626f..a7a30cd4ca6e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java @@ -42,7 +42,6 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -582,8 +581,8 @@ public void testInsertTableWithClusteringWithClusteringFromCatalog() ) .orderBy( ImmutableList.of( - new OrderBy("b", Order.ASCENDING), - new OrderBy("d", Order.ASCENDING) + OrderBy.ascending("b"), + OrderBy.ascending("d") ) ) // Scan query lists columns in alphabetical order independent of the @@ -644,7 +643,7 @@ public void testInsertTableWithClusteringWithClusteringFromQuery() ) .orderBy( ImmutableList.of( - new OrderBy("b", Order.ASCENDING) + OrderBy.ascending("b") ) ) // Scan query lists columns in alphabetical order independent of the @@ -709,7 +708,7 @@ public void testInsertTableWithClusteringWithClusteringOnNewColumnFromQuery() ) .orderBy( ImmutableList.of( - new OrderBy("e", Order.ASCENDING) + OrderBy.ascending("e") ) ) // Scan query lists columns in alphabetical order independent of the diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index e8cffd3cac9c..13cd9a42dd8b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -38,7 +38,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; @@ -729,9 +728,9 @@ public void testExplainPlanInsertWithClusteredBy() throws JsonProcessingExceptio ) .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING), - new OrderBy("v1", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context( @@ -932,9 +931,9 @@ public void testInsertWithClusteredBy() ) .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING), - new OrderBy("v1", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -973,9 +972,9 @@ public void testInsertPeriodFormGranularityWithClusteredBy() ) .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING), - new OrderBy("v1", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -1023,8 +1022,8 @@ public void testInsertWithPartitionedByAndClusteredBy() .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT)) .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1") ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -1212,9 +1211,9 @@ public void testExplainPlanForInsertWithClusteredBy() throws JsonProcessingExcep ) .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING), - new OrderBy("v1", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context( @@ -1327,8 +1326,8 @@ public void testInsertFromExternalProjectSort() .columns("v0", "z") .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("z", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("z") ) ) .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 5161fa321148..d75e2a16e1b7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -34,7 +34,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; @@ -587,8 +586,8 @@ public void testReplaceWithClusteredBy() .virtualColumns(expressionVirtualColumn("v0", "floor(\"m1\")", ColumnType.FLOAT)) .orderBy( ImmutableList.of( - new OrderBy("v0", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1") ) ) .context( @@ -759,11 +758,7 @@ public void testExplainReplaceTimeChunksWithPartitioningAndClustering() throws I .dataSource("foo") .intervals(querySegmentSpec(Filtration.eternity())) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") - .orderBy( - ImmutableList.of( - new OrderBy("dim1", Order.ASCENDING) - ) - ) + .orderBy(ImmutableList.of(OrderBy.ascending("dim1"))) .columnTypes(LONG, LONG, STRING, STRING, STRING, FLOAT, DOUBLE, ofComplex("hyperUnique")) .context( queryJsonMapper.readValue( @@ -844,10 +839,10 @@ public void testExplainReplaceWithLimitAndClusteredByOrdinals() throws IOExcepti .limit(10) .orderBy( ImmutableList.of( - new OrderBy("__time", Order.ASCENDING), - new OrderBy("dim1", Order.ASCENDING), - new OrderBy("dim3", Order.ASCENDING), - new OrderBy("dim2", Order.ASCENDING) + OrderBy.ascending("__time"), + OrderBy.ascending("dim1"), + OrderBy.ascending("dim3"), + OrderBy.ascending("dim2") ) ) .columnTypes(LONG, LONG, STRING, STRING, STRING, FLOAT, DOUBLE, ColumnType.ofComplex("hyperUnique")) From 54fce97d1ad31a499619cc197f33dc8ed07b5483 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 13:10:34 -0700 Subject: [PATCH 49/74] fixup merge style --- .../src/main/java/org/apache/druid/frame/read/FrameReader.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index 49cdab7697a6..655b56547a02 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -32,7 +32,6 @@ import org.apache.druid.frame.segment.row.FrameCursorMakerFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.CursorMakerFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; From 397e1f5ca3ea347d6f958fbe17c05a607ba42c68 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 6 Aug 2024 18:50:14 -0700 Subject: [PATCH 50/74] add test comments --- .../org/apache/druid/segment/RowBasedStorageAdapterTest.java | 4 +++- .../org/apache/druid/segment/UnnestStorageAdapterTest.java | 4 ++++ .../java/org/apache/druid/segment/join/JoinTestHelper.java | 3 ++- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index eba63db170cd..cf303fa609a1 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -836,12 +836,14 @@ private static List> walkCursor( final List> retVal = new ArrayList<>(); + // test cursor reset while (!cursor.isDone()) { cursor.advanceUninterruptibly(); } cursor.reset(); + // test cursor mark/resetToMark int ctr = 0; int mark = 1; while (!cursor.isDone()) { @@ -859,7 +861,7 @@ private static List> walkCursor( cursor.advanceUninterruptibly(); } - if (ctr > 1) { + if (ctr > mark) { cursor.resetToMark(); while (!cursor.isDone()) { diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index 01b5c6c85967..a7da2c02f964 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -241,11 +241,13 @@ public void test_unnest_adapters_basic() int count = 0; List rows = new ArrayList<>(); + // test cursor reset while (!cursor.isDone()) { cursor.advance(); } cursor.reset(); + // test cursor mark/resetToMark int mark = 2; while (!cursor.isDone()) { if (count == mark) { @@ -289,11 +291,13 @@ public void test_unnest_adapters_basic_array_column() int count = 0; List rows = new ArrayList<>(); + // test cursor reset while (!cursor.isDone()) { cursor.advance(); } cursor.reset(); + // test cursor mark/resetToMark int mark = 2; while (!cursor.isDone()) { if (count == mark) { diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index e3b8b4864bec..b623d0d8de93 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -343,7 +343,7 @@ public static List readCursor(final CursorHolder cursorHolder, final L final List rows = new ArrayList<>(); boolean interruptible = false; // test both advance() and advanceUninterruptibly() - + // test cursor reset while (!cursor.isDone()) { if (interruptible) { cursor.advance(); @@ -356,6 +356,7 @@ public static List readCursor(final CursorHolder cursorHolder, final L cursor.reset(); + // test cursor mark/resetToMark int ctr = 0; int mark = 2; while (!cursor.isDone()) { From 05217df712995fcee65783e98c6e39db5056c0d4 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 7 Aug 2024 10:54:44 -0700 Subject: [PATCH 51/74] cleanup, fix segment analyzer bug --- .../ExpressionAggregationBenchmark.java | 5 +-- .../benchmark/ExpressionFilterBenchmark.java | 2 - .../ExpressionSelectorBenchmark.java | 32 ++----------- .../ExpressionVectorSelectorBenchmark.java | 1 - .../hadoop/DatasourceRecordReader.java | 1 - .../indexing/input/DruidSegmentReader.java | 1 - .../common/task/CompactionTaskRunTest.java | 15 ++----- .../indexing/common/task/IndexTaskTest.java | 10 +---- .../druid/query/metadata/SegmentAnalyzer.java | 9 +--- .../segment/QueryableIndexStorageAdapter.java | 2 +- .../segment/incremental/IncrementalIndex.java | 10 ++--- .../incremental/IncrementalIndexSchema.java | 18 ++++---- .../table/BroadcastSegmentIndexedTable.java | 9 +--- .../metadata/SegmentMetadataQueryTest.java | 45 +++++++------------ .../SegmentMetadataUnionQueryTest.java | 4 +- .../segment/AutoTypeColumnIndexerTest.java | 14 +++--- .../NestedDataColumnIndexerV4Test.java | 12 ++--- .../segment/UnnestStorageAdapterTest.java | 30 +++---------- .../druid/segment/filter/BaseFilterTest.java | 10 ++--- .../NestedFieldColumnSelectorsTest.java | 1 - .../virtual/ExpressionSelectorsTest.java | 20 ++------- .../ExpressionVectorSelectorsCastTest.java | 1 - .../ExpressionVectorSelectorsTest.java | 1 - .../org/apache/druid/cli/DumpSegment.java | 4 -- 24 files changed, 74 insertions(+), 183 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java index 2e6931637ece..d65d13a324d8 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -163,10 +163,7 @@ private double compute(final Function a { final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java index ac79a7bfd66a..06f0952b51f0 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -147,7 +147,6 @@ public void expressionFilter(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(expressionFilter.toFilter()) - .setInterval(index.getDataInterval()) .build(); try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); @@ -166,7 +165,6 @@ public void nativeFilter(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(nativeFilter.toFilter()) - .setInterval(index.getDataInterval()) .build(); try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); 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 7047896ae7ce..e09f91f1ce96 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -147,7 +147,6 @@ public void tearDown() throws Exception public void timeFloorUsingExpression(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -170,10 +169,7 @@ public void timeFloorUsingExpression(Blackhole blackhole) @Benchmark public void timeFloorUsingExtractionFn(Blackhole blackhole) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor @@ -192,11 +188,8 @@ public void timeFloorUsingExtractionFn(Blackhole blackhole) @Benchmark public void timeFloorUsingCursor(Blackhole blackhole) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .build(); final StorageAdapter adapter = new QueryableIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final CursorGranularizer granularizer = CursorGranularizer.create( adapter, @@ -234,7 +227,6 @@ public void timeFloorUsingCursor(Blackhole blackhole) public void timeFormatUsingExpression(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -259,10 +251,7 @@ public void timeFormatUsingExpression(Blackhole blackhole) @Benchmark public void timeFormatUsingExtractionFn(Blackhole blackhole) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -281,7 +270,6 @@ public void timeFormatUsingExtractionFn(Blackhole blackhole) public void strlenUsingExpressionAsLong(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -305,7 +293,6 @@ public void strlenUsingExpressionAsLong(Blackhole blackhole) public void strlenUsingExpressionAsString(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -331,11 +318,7 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) @Benchmark public void strlenUsingExtractionFn(Blackhole blackhole) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) - .build(); - - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -349,7 +332,6 @@ public void strlenUsingExtractionFn(Blackhole blackhole) public void arithmeticOnLong(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -373,7 +355,6 @@ public void arithmeticOnLong(Blackhole blackhole) public void stringConcatAndCompareOnLong(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -396,7 +377,6 @@ public void stringConcatAndCompareOnLong(Blackhole blackhole) public void caseSearched1(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -419,7 +399,6 @@ public void caseSearched1(Blackhole blackhole) public void caseSearched2(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -454,7 +433,6 @@ public void caseSearched100(Blackhole blackhole) } final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -478,7 +456,6 @@ public void caseSearched100(Blackhole blackhole) public void caseSearchedWithLookup(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( new ExpressionVirtualColumn( @@ -508,7 +485,6 @@ public void caseSearchedWithLookup(Blackhole blackhole) public void caseSearchedWithLookup2(Blackhole blackhole) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns( VirtualColumns.create( ImmutableList.of( 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 695ddcc6471d..5604f7fe8834 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -152,7 +152,6 @@ public void scan(Blackhole blackhole) ) ); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns(virtualColumns) .build(); final CursorHolder cursorHolder = closer.register( diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index d1ceccc96a35..271858de20e9 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -216,7 +216,6 @@ public Sequence apply(WindowedStorageAdapter adapter) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) - .setInterval(adapter.getInterval()) .build(); final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec); final Cursor cursor = cursorHolder.asCursor(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index a85fa73349df..94ffd953d2d5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -126,7 +126,6 @@ protected CloseableIterator> intermediateRowIterator() throw ); final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) - .setInterval(storageAdapter.getInterval()) .build(); final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 279cd937a1e6..6fbf9e5deed4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -1702,10 +1702,7 @@ public void testRunWithSpatialDimensions() throws Exception new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(segment.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); @@ -1833,10 +1830,7 @@ public void testRunWithAutoCastDimensions() throws Exception new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(segment.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); @@ -2061,10 +2055,7 @@ private List getCSVFormatRowsFromSegments(List segments) th new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(segment.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index edb1d68c012b..c54446fc4df0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -540,10 +540,7 @@ public void testTransformSpec() throws Exception new QueryableIndexStorageAdapter(indexIO.loadIndex(segmentFile)), segment.getInterval() ); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(segment.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final List> transforms = new ArrayList<>(); @@ -773,10 +770,7 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception segment.getInterval() ); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(segment.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final List hashes = new ArrayList<>(); final DimensionSelector selector = cursor.getColumnSelectorFactory() diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index 2e5283344392..6d0629c4dcec 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -50,8 +50,6 @@ import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; -import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.IOException; @@ -272,12 +270,7 @@ private ColumnAnalysis analyzeStringColumn( } if (analyzingSize()) { - final DateTime start = storageAdapter.getMinTime(); - final DateTime end = storageAdapter.getMaxTime(); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(new Interval(start, end)) - .build(); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor != null) { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 4dbe532fb013..7d860465cb6f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -174,7 +174,7 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec) { return new QueryableIndexCursorHolder( index, - CursorBuildSpec.builder(spec).setInterval(spec.getInterval()).build() + CursorBuildSpec.builder(spec).build() ); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index 06fc7195d526..5265f4f2c448 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -222,7 +222,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) } private final long minTimestamp; - private final Granularity gran; + private final Granularity queryGranularity; private final boolean rollup; private final List> rowTransformers; private final VirtualColumns virtualColumns; @@ -266,7 +266,7 @@ protected IncrementalIndex( ) { this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); - this.gran = incrementalIndexSchema.getGran(); + this.queryGranularity = incrementalIndexSchema.getQueryGranularity(); this.rollup = incrementalIndexSchema.isRollup(); this.virtualColumns = incrementalIndexSchema.getVirtualColumns(); this.metrics = incrementalIndexSchema.getMetrics(); @@ -284,7 +284,7 @@ protected IncrementalIndex( null, getCombiningAggregators(metrics), incrementalIndexSchema.getTimestampSpec(), - this.gran, + this.queryGranularity, this.rollup ); @@ -658,7 +658,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) long truncated = 0; if (row.getTimestamp() != null) { - truncated = gran.bucketStart(row.getTimestampFromEpoch()); + truncated = queryGranularity.bucketStart(row.getTimestampFromEpoch()); } IncrementalIndexRow incrementalIndexRow = IncrementalIndexRow.createTimeAndDimswithDimsKeySize( Math.max(truncated, minTimestamp), @@ -837,7 +837,7 @@ public ColumnValueSelector makeMetricColumnValueSelector(String metric, Incre public Interval getInterval() { DateTime min = DateTimes.utc(minTimestamp); - return new Interval(min, isEmpty() ? min : gran.increment(DateTimes.utc(getMaxTimeMillis()))); + return new Interval(min, isEmpty() ? min : queryGranularity.increment(DateTimes.utc(getMaxTimeMillis()))); } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index d127f3d89bb7..8866c413dc74 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -40,7 +40,7 @@ public static IncrementalIndexSchema.Builder builder() private final long minTimestamp; private final TimestampSpec timestampSpec; - private final Granularity gran; + private final Granularity queryGranularity; private final VirtualColumns virtualColumns; private final DimensionsSpec dimensionsSpec; private final AggregatorFactory[] metrics; @@ -49,7 +49,7 @@ public static IncrementalIndexSchema.Builder builder() public IncrementalIndexSchema( long minTimestamp, TimestampSpec timestampSpec, - Granularity gran, + Granularity queryGranularity, VirtualColumns virtualColumns, DimensionsSpec dimensionsSpec, AggregatorFactory[] metrics, @@ -58,7 +58,7 @@ public IncrementalIndexSchema( { this.minTimestamp = minTimestamp; this.timestampSpec = timestampSpec; - this.gran = gran; + this.queryGranularity = queryGranularity; this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns); this.dimensionsSpec = dimensionsSpec; this.metrics = metrics; @@ -75,9 +75,9 @@ public TimestampSpec getTimestampSpec() return timestampSpec; } - public Granularity getGran() + public Granularity getQueryGranularity() { - return gran; + return queryGranularity; } public VirtualColumns getVirtualColumns() @@ -104,7 +104,7 @@ public static class Builder { private long minTimestamp; private TimestampSpec timestampSpec; - private Granularity gran; + private Granularity queryGranularity; private VirtualColumns virtualColumns; private DimensionsSpec dimensionsSpec; private AggregatorFactory[] metrics; @@ -113,7 +113,7 @@ public static class Builder public Builder() { this.minTimestamp = 0L; - this.gran = Granularities.NONE; + this.queryGranularity = Granularities.NONE; this.virtualColumns = VirtualColumns.EMPTY; this.dimensionsSpec = DimensionsSpec.EMPTY; this.metrics = new AggregatorFactory[]{}; @@ -134,7 +134,7 @@ public Builder withTimestampSpec(TimestampSpec timestampSpec) public Builder withQueryGranularity(Granularity gran) { - this.gran = gran; + this.queryGranularity = gran; return this; } @@ -181,7 +181,7 @@ public IncrementalIndexSchema build() return new IncrementalIndexSchema( minTimestamp, timestampSpec, - gran, + queryGranularity, virtualColumns, dimensionsSpec, metrics, diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index 8807dc5b4a7a..716decb04d99 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -43,7 +43,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.timeline.SegmentId; -import org.joda.time.chrono.ISOChronology; import javax.annotation.Nullable; import java.io.Closeable; @@ -113,13 +112,7 @@ public BroadcastSegmentIndexedTable( indexBuilders.add(m); } - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval( - queryableIndex.getDataInterval() - .withChronology(ISOChronology.getInstanceUTC()) - ) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { this.keyColumnsIndexes = Collections.emptyList(); diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index e919799deb56..8e39a640acb3 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -205,15 +205,11 @@ public SegmentMetadataQueryTest( .merge(true) .build(); - int preferedSize1 = 0; - int placementSize2 = 0; - int overallSize1 = 153543; - int overallSize2 = 153543; + int placementSize = 0; + int overallSize = 153543; if (bitmaps) { - preferedSize1 = mmap1 ? 10881 : 10764; - placementSize2 = mmap2 ? 10881 : 0; - overallSize1 = mmap1 ? 201345 : 200831; - overallSize2 = mmap2 ? 201345 : 200831; + placementSize = 10881; + overallSize = 201345; } final Map expectedAggregators = new HashMap<>(); @@ -256,7 +252,7 @@ public SegmentMetadataQueryTest( ValueType.STRING.toString(), false, false, - preferedSize1, + placementSize, 1, "preferred", "preferred", @@ -264,7 +260,7 @@ public SegmentMetadataQueryTest( ) ) ), - overallSize1, + overallSize, 1209, expectedAggregators, null, @@ -306,7 +302,7 @@ public SegmentMetadataQueryTest( ValueType.STRING.toString(), false, false, - placementSize2, + placementSize, 1, null, null, @@ -314,8 +310,7 @@ public SegmentMetadataQueryTest( ) ) ), - // null_column will be included only for incremental index, which makes a little bigger result than expected - overallSize2, + overallSize, 1209, expectedAggregators, null, @@ -558,18 +553,16 @@ public void testSegmentMetadataQueryWithComplexColumnMerge() @Test public void testSegmentMetadataQueryWithDefaultAnalysisMerge() { - int size1 = 0; - int size2 = 0; + int size = 0; if (bitmaps) { - size1 = mmap1 ? 10881 : 10764; - size2 = mmap2 ? 10881 : 10764; + size = 10881; } ColumnAnalysis analysis = new ColumnAnalysis( ColumnType.STRING, ValueType.STRING.toString(), false, false, - size1 + size2, + size * 2, 1, "preferred", "preferred", @@ -581,18 +574,16 @@ public void testSegmentMetadataQueryWithDefaultAnalysisMerge() @Test public void testSegmentMetadataQueryWithDefaultAnalysisMerge2() { - int size1 = 0; - int size2 = 0; + int size = 0; if (bitmaps) { - size1 = mmap1 ? 6882 : 6808; - size2 = mmap2 ? 6882 : 6808; + size = 6882; } ColumnAnalysis analysis = new ColumnAnalysis( ColumnType.STRING, ValueType.STRING.toString(), false, false, - size1 + size2, + size * 2, 3, "spot", "upfront", @@ -604,18 +595,16 @@ public void testSegmentMetadataQueryWithDefaultAnalysisMerge2() @Test public void testSegmentMetadataQueryWithDefaultAnalysisMerge3() { - int size1 = 0; - int size2 = 0; + int size = 0; if (bitmaps) { - size1 = mmap1 ? 9765 : 9660; - size2 = mmap2 ? 9765 : 9660; + size = 9765; } ColumnAnalysis analysis = new ColumnAnalysis( ColumnType.STRING, ValueType.STRING.toString(), false, false, - size1 + size2, + size * 2, 9, "automotive", "travel", diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java index 9f37a47aae43..b1db91a9cb85 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java @@ -109,7 +109,7 @@ public void testSegmentMetadataUnionQuery() ValueType.STRING.toString(), false, false, - mmap ? 43524 : 43056, + 43524, 1, "preferred", "preferred", @@ -117,7 +117,7 @@ public void testSegmentMetadataUnionQuery() ) ) ), - mmap ? 805380 : 803324, + 805380, 4836, null, null, diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index 2951cf80f5a5..788181859ffa 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -148,7 +148,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -201,7 +201,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -284,7 +284,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -367,7 +367,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec( STRING_ARRAY_COL, @@ -415,7 +415,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -459,7 +459,7 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -518,7 +518,7 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index e00458c5d9f1..f13b6c5632a6 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -148,7 +148,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -199,7 +199,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -280,7 +280,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -361,7 +361,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec( STRING_ARRAY_COL, @@ -404,7 +404,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -447,7 +447,7 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.builder().build())) { + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index a7da2c02f964..b46fc8671c3a 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -229,10 +229,7 @@ public void test_unnest_adapter_column_capabilities() @Test public void test_unnest_adapters_basic() { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(UNNEST_STORAGE_ADAPTER.getInterval()) - .build(); - try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -279,10 +276,7 @@ public void test_unnest_adapters_basic() @Test public void test_unnest_adapters_basic_array_column() { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(UNNEST_ARRAYS.getInterval()) - .build(); - try (final CursorHolder cursorHolder = UNNEST_ARRAYS.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = UNNEST_ARRAYS.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -325,10 +319,7 @@ public void test_unnest_adapters_basic_array_column() @Test public void test_two_levels_of_unnest_adapters() { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval()) - .build(); - try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -383,7 +374,6 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(baseFilter) - .setInterval(unnestStorageAdapter.getInterval()) .build(); try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); @@ -429,7 +419,6 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(baseFilter) - .setInterval(unnestStorageAdapter.getInterval()) .build(); try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); @@ -706,11 +695,7 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() final Filter expectedPushDownFilter = selector(inputColumn, "1"); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(unnestStorageAdapter.getInterval()) - .build(); - - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); final Filter pushDownFilter = base.getPushDownFilter(); @@ -749,7 +734,6 @@ public void test_pushdown_filters_unnested_dimension_outside() final Filter queryFilter = new SelectorFilter(OUTPUT_COLUMN_NAME, "1", null); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(queryFilter) - .setInterval(unnestStorageAdapter.getInterval()) .build(); try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { @@ -795,10 +779,8 @@ public void testUnnestValueMatcherValueDoesntExist() new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), null ); - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(withNullsStorageAdapter.getInterval()) - .build(); - try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(buildSpec)) { + + try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); 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 f07521e07b39..1491fbd9d388 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 @@ -526,7 +526,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -554,7 +554,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -583,7 +583,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -662,7 +662,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -685,7 +685,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() 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 c458d4962318..fd89e083441a 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 @@ -346,7 +346,6 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir Assert.assertEquals(1, segments.size()); StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(storageAdapter.getInterval()) .setVirtualColumns(virtualColumns) .build(); final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 2f67dc454110..13f82e2be8ed 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -138,10 +138,7 @@ public void test_single_value_string_bindings() { final String columnName = "string3"; for (StorageAdapter adapter : ADAPTERS) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(adapter.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -206,10 +203,7 @@ public void test_multi_value_string_bindings() { final String columnName = "multi-string3"; for (StorageAdapter adapter : ADAPTERS) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(adapter.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -286,10 +280,7 @@ public void test_long_bindings() { final String columnName = "long3"; for (StorageAdapter adapter : ADAPTERS) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(adapter.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans @@ -334,10 +325,7 @@ public void test_double_bindings() { final String columnName = "double3"; for (StorageAdapter adapter : ADAPTERS) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(adapter.getInterval()) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans 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 8c1646e93a5a..6fbee67734f6 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 @@ -119,7 +119,6 @@ public static void testCast( final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) - .setInterval(index.getDataInterval()) .build(); VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor(); 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 1214987c9c98..5d284a741297 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 @@ -236,7 +236,6 @@ public static void sanityTestVectorizedExpressionSelectors( ); final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setInterval(index.getDataInterval()) .setVirtualColumns(virtualColumns) .build(); try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) { diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 3c509f65e0c2..f96a25bb6d8b 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -301,10 +301,6 @@ public static void runDump( final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(filter)) - .setInterval( - index.getDataInterval() - .withChronology(ISOChronology.getInstanceUTC()) - ) .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { From c8545ea8fd469570c2e269ef54a62b53a573bd70 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 7 Aug 2024 12:33:19 -0700 Subject: [PATCH 52/74] fix build --- .../org/apache/druid/benchmark/ExpressionSelectorBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e09f91f1ce96..5bc67bc9f729 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -195,7 +195,7 @@ public void timeFloorUsingCursor(Blackhole blackhole) adapter, cursor, Granularities.HOUR, - buildSpec.getInterval(), + adapter.getInterval(), false ); final Sequence results = From 44523cd0907dbf1e42af3395a71290890b6f27f3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 8 Aug 2024 10:31:52 -0700 Subject: [PATCH 53/74] add intervals back to windowed storage adapter for DruidSegmentReader and DatasourceRecordReader --- .../org/apache/druid/indexer/hadoop/DatasourceRecordReader.java | 1 + .../java/org/apache/druid/indexing/input/DruidSegmentReader.java | 1 + services/src/main/java/org/apache/druid/cli/DumpSegment.java | 1 - 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index 271858de20e9..d1ceccc96a35 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -216,6 +216,7 @@ public Sequence apply(WindowedStorageAdapter adapter) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) + .setInterval(adapter.getInterval()) .build(); final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec); final Cursor cursor = cursorHolder.asCursor(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index 94ffd953d2d5..a85fa73349df 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -126,6 +126,7 @@ protected CloseableIterator> intermediateRowIterator() throw ); final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) + .setInterval(storageAdapter.getInterval()) .build(); final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec); diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index f96a25bb6d8b..3f8cfcb94a45 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -97,7 +97,6 @@ import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import org.joda.time.chrono.ISOChronology; import org.roaringbitmap.IntIterator; import java.io.File; From 99af30cd01472c2c26900b45723f304bf34ebd65 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 9 Aug 2024 04:27:36 -0700 Subject: [PATCH 54/74] adjustments --- .../druid/benchmark/query/ScanBenchmark.java | 2 +- .../msq/indexing/MSQCompactionRunner.java | 4 +- .../scan/ScanQueryFrameProcessor.java | 7 +- .../druid/msq/querykit/scan/ScanQueryKit.java | 4 +- .../apache/druid/msq/exec/MSQArraysTest.java | 2 +- .../druid/msq/exec/MSQComplexGroupByTest.java | 2 +- .../druid/msq/exec/MSQLoadedSegmentTests.java | 2 +- .../apache/druid/msq/exec/MSQSelectTest.java | 2 +- .../apache/druid/msq/exec/MSQWindowTest.java | 2 +- .../msq/indexing/MSQCompactionRunnerTest.java | 2 +- .../apache/druid/frame/field/RowReader.java | 4 +- .../frame/processor/FrameProcessors.java | 4 +- .../apache/druid/frame/read/FrameReader.java | 12 +- .../druid/frame/segment/FrameCursor.java | 6 +- .../frame/segment/FrameStorageAdapter.java | 6 +- ...ory.java => FrameCursorHolderFactory.java} | 10 +- .../segment/columnar/FrameQueryableIndex.java | 2 +- ...ory.java => FrameCursorHolderFactory.java} | 10 +- .../java/org/apache/druid/query/Druids.java | 2 - .../druid/query/FilteredDataSource.java | 5 +- .../apache/druid/query/{scan => }/Order.java | 2 +- .../druid/query/{scan => }/OrderBy.java | 2 +- .../java/org/apache/druid/query/Query.java | 17 +-- .../druid/query/groupby/GroupByQuery.java | 25 +--- .../druid/query/groupby/GroupingEngine.java | 40 ++++++- .../query/operator/WindowOperatorQuery.java | 5 +- .../apache/druid/query/scan/ScanQuery.java | 28 +---- .../druid/query/scan/ScanQueryEngine.java | 17 ++- .../query/scan/ScanQueryLimitRowIterator.java | 1 + .../query/scan/ScanQueryRunnerFactory.java | 1 + .../ScanResultValueTimestampComparator.java | 1 + .../query/search/CursorOnlyStrategy.java | 13 ++- .../TimeBoundaryQueryRunnerFactory.java | 15 ++- .../query/timeseries/TimeseriesQuery.java | 36 ------ .../timeseries/TimeseriesQueryEngine.java | 51 ++++++++- .../apache/druid/query/topn/TopNQuery.java | 31 ----- .../druid/query/topn/TopNQueryEngine.java | 47 +++++++- .../java/org/apache/druid/segment/Cursor.java | 2 +- .../apache/druid/segment/CursorBuildSpec.java | 108 ++++++++++++------ .../apache/druid/segment/CursorFactory.java | 22 ++-- ...rFactory.java => CursorHolderFactory.java} | 2 +- .../druid/segment/FilteredStorageAdapter.java | 9 +- .../apache/druid/segment/StorageAdapter.java | 2 +- .../druid/segment/vector/VectorCursor.java | 3 +- .../segment/FrameStorageAdapterTest.java | 8 +- .../druid/query/CursorGranularizerTest.java | 1 - .../druid/query/groupby/GroupByQueryTest.java | 8 +- .../groupby/NestedDataGroupByQueryTest.java | 2 +- .../VectorGroupByEngineIteratorTest.java | 5 +- .../scan/ScanQueryLimitRowIteratorTest.java | 1 + .../scan/ScanQueryResultOrderingTest.java | 1 + .../scan/ScanQueryRunnerFactoryTest.java | 1 + .../druid/query/scan/ScanQueryRunnerTest.java | 1 + .../druid/query/scan/ScanQuerySpecTest.java | 2 + .../druid/query/scan/ScanQueryTest.java | 4 +- ...canResultValueTimestampComparatorTest.java | 1 + .../query/scan/UnnestScanQueryRunnerTest.java | 1 + .../NestedDataTimeseriesQueryTest.java | 2 +- .../query/timeseries/TimeseriesQueryTest.java | 4 +- .../druid/query/topn/TopNQueryTest.java | 4 +- .../segment/RowBasedStorageAdapterTest.java | 2 +- .../IncrementalIndexStorageAdapterTest.java | 9 +- .../appenderator/StreamAppenderatorTest.java | 2 +- .../druid/sql/calcite/rel/DruidQuery.java | 4 +- .../CalciteCatalogIngestionDmlTest.java | 2 +- .../sql/calcite/CalciteInsertDmlTest.java | 2 +- .../sql/calcite/CalciteJoinQueryTest.java | 2 +- .../druid/sql/calcite/CalciteQueryTest.java | 2 +- .../sql/calcite/CalciteReplaceDmlTest.java | 2 +- .../sql/calcite/CalciteSelectQueryTest.java | 2 +- .../sql/calcite/CalciteSubqueryTest.java | 2 +- 71 files changed, 382 insertions(+), 265 deletions(-) rename processing/src/main/java/org/apache/druid/frame/segment/columnar/{FrameCursorMakerFactory.java => FrameCursorHolderFactory.java} (94%) rename processing/src/main/java/org/apache/druid/frame/segment/row/{FrameCursorMakerFactory.java => FrameCursorHolderFactory.java} (91%) rename processing/src/main/java/org/apache/druid/query/{scan => }/Order.java (97%) rename processing/src/main/java/org/apache/druid/query/{scan => }/OrderBy.java (98%) rename processing/src/main/java/org/apache/druid/segment/{CursorMakerFactory.java => CursorHolderFactory.java} (96%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 884370d28c7f..8143e4e8c757 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -34,6 +34,7 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.FinalizeResultsQueryRunner; +import org.apache.druid.query.Order; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -49,7 +50,6 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryConfig; import org.apache.druid.query.scan.ScanQueryEngine; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 9b9957987e1b..795cb195a0e4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -47,6 +47,8 @@ import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; @@ -60,8 +62,6 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.scan.Order; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 13fc7ed71098..f402aa604308 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -59,8 +59,9 @@ import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.query.Druids; import org.apache.druid.query.IterableRowsCursorHelper; -import org.apache.druid.query.scan.Order; +import org.apache.druid.query.Order; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanQueryEngine; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; @@ -254,7 +255,7 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment } final CursorHolder cursorHolder = closer.register( - adapter.makeCursorHolder(query.asCursorBuildSpec(null)) + adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); final Cursor nextCursor = cursorHolder.asCursor(); @@ -299,7 +300,7 @@ protected ReturnOrAwait runWithInputChannel( } final CursorHolder cursorHolder = closer.register( - adapter.makeCursorHolder(query.asCursorBuildSpec(null)) + adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); final Cursor nextCursor = cursorHolder.asCursor(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index 038e069b76b3..5bbf9c9cbb05 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -37,9 +37,9 @@ import org.apache.druid.msq.querykit.ShuffleSpecFactories; import org.apache.druid.msq.querykit.ShuffleSpecFactory; import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; -import org.apache.druid.query.scan.Order; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index 3a3b00202979..03890d7a3263 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -36,9 +36,9 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.DataSource; import org.apache.druid.query.NestedDataTestUtils; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java index eb50146daad0..e498b5bb268a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java @@ -36,6 +36,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.DataSource; import org.apache.druid.query.NestedDataTestUtils; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; @@ -50,7 +51,6 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.nested.StructuredData; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index c1d4b064f81f..4ea4cb0344ff 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -33,11 +33,11 @@ import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.column.ColumnType; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index a2193e1a391d..b80b3844b502 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -49,6 +49,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; @@ -67,7 +68,6 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 2c39d1366034..ee7121b36787 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -32,6 +32,7 @@ import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; @@ -51,7 +52,6 @@ import org.apache.druid.query.operator.window.WindowFramedAggregateProcessor; import org.apache.druid.query.operator.window.WindowOperatorFactory; import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.segment.column.ColumnType; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index f6d7fca2e8ad..1529884d3a58 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -48,13 +48,13 @@ import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; diff --git a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java index f1563d044bfe..861609ad1882 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java @@ -57,7 +57,7 @@ public int fieldCount() /** * Read a particular field value as an object. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorMakerFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorHolderFactory} * for reading many rows out of a frame. */ public Object readField(final Memory memory, final long rowPosition, final long rowLength, final int fieldNumber) @@ -77,7 +77,7 @@ public Object readField(final Memory memory, final long rowPosition, final long /** * Read an entire row as a list of objects. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorMakerFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorHolderFactory} * for reading many rows out of a frame. */ public List readRow(final Memory memory, final long rowPosition, final long rowLength) diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index 042668884e9d..f75ced24b12d 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -117,9 +117,9 @@ public static FrameCursor makeCursor( final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - // Despite appearances of columnar FrameCursorMakerFactory with its closers, it is currently safe to never close + // Despite appearances of columnar FrameCursorHolderFactory with its closers, it is currently safe to never close // the CursorHolder that the FrameCursor comes from because it really does nothing. The row based - // FrameCursorMakerFactory has no closer stuff at all and is totally safe. If this ever changes, this method will + // FrameCursorHolderFactory has no closer stuff at all and is totally safe. If this ever changes, this method will // probably need to wrap the cursor in something closeable, or be reworked to just return the CursorHolder so that // callers can deal with closing the stuff. return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder(cursorBuildSpec) diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index 655b56547a02..9fa10f8d833f 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -29,10 +29,10 @@ import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.read.columnar.FrameColumnReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.row.FrameCursorMakerFactory; +import org.apache.druid.frame.segment.columnar.FrameCursorHolderFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.CursorMakerFactory; +import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -136,15 +136,15 @@ public ColumnCapabilities columnCapabilities(final Frame frame, final String col } /** - * Create a {@link CursorMakerFactory} for the given frame. + * Create a {@link CursorHolderFactory} for the given frame. */ - public CursorMakerFactory makeCursorMakerFactory(final Frame frame) + public CursorHolderFactory makeCursorHolderFactory(final Frame frame) { switch (frame.type()) { case COLUMNAR: - return new org.apache.druid.frame.segment.columnar.FrameCursorMakerFactory(frame, signature, columnReaders); + return new FrameCursorHolderFactory(frame, signature, columnReaders); case ROW_BASED: - return new FrameCursorMakerFactory(frame, this, fieldReaders); + return new org.apache.druid.frame.segment.row.FrameCursorHolderFactory(frame, this, fieldReaders); default: throw DruidException.defensive("Unrecognized frame type [%s]", frame.type()); } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 2155aad7247e..32dcbcd2f67b 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -19,15 +19,15 @@ package org.apache.druid.frame.segment; -import org.apache.druid.frame.segment.columnar.FrameCursorMakerFactory; +import org.apache.druid.frame.segment.columnar.FrameCursorHolderFactory; import org.apache.druid.query.BaseQuery; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; /** - * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorMakerFactory} - * and {@link FrameCursorMakerFactory}. + * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorHolderFactory} + * and {@link FrameCursorHolderFactory}. * * Adds the methods {@link #getCurrentRow()} and {@link #setCurrentRow(int)} so the cursor can be moved to * particular rows. diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java index 28162bf1028f..79dcdb7ced3c 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java @@ -23,7 +23,7 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorMakerFactory; +import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; @@ -47,14 +47,14 @@ public class FrameStorageAdapter implements StorageAdapter private final Frame frame; private final FrameReader frameReader; private final Interval interval; - private final CursorMakerFactory cursorFactory; + private final CursorHolderFactory cursorFactory; public FrameStorageAdapter(Frame frame, FrameReader frameReader, Interval interval) { this.frame = frame; this.frameReader = frameReader; this.interval = interval; - this.cursorFactory = frameReader.makeCursorMakerFactory(frame); + this.cursorFactory = frameReader.makeCursorHolderFactory(frame); } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorHolderFactory.java similarity index 94% rename from processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorHolderFactory.java index 2a3bf4b7c3aa..7c1b65ee87c8 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorHolderFactory.java @@ -33,7 +33,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorMakerFactory; +import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; @@ -51,19 +51,19 @@ import java.util.List; /** - * A {@link CursorMakerFactory} implementation based on a single columnar {@link Frame}. + * A {@link CursorHolderFactory} implementation based on a single columnar {@link Frame}. * * This class is only used for columnar frames. It is not used for row-based frames. * - * @see org.apache.druid.frame.segment.row.FrameCursorMakerFactory the row-based version + * @see org.apache.druid.frame.segment.row.FrameCursorHolderFactory the row-based version */ -public class FrameCursorMakerFactory implements CursorMakerFactory +public class FrameCursorHolderFactory implements CursorHolderFactory { private final Frame frame; private final RowSignature signature; private final List columnReaders; - public FrameCursorMakerFactory( + public FrameCursorHolderFactory( final Frame frame, final RowSignature signature, final List columnReaders diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index 94e4d637ddf5..8b2b4bef2880 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java @@ -42,7 +42,7 @@ * A {@link QueryableIndex} implementation based on a single columnar {@link Frame}. There is no internal caching * of columns here, so callers should generally wrap this in a {@link org.apache.druid.segment.ColumnCache}. * - * This class exists so {@link FrameCursorMakerFactory} can reuse code meant for regular segment-backed + * This class exists so {@link FrameCursorHolderFactory} can reuse code meant for regular segment-backed * {@link QueryableIndex}. Some methods are implemented by throwing {@link UnsupportedOperationException}, wherever * it is not expected that those methods are actually going to be needed. */ diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorHolderFactory.java similarity index 91% rename from processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorHolderFactory.java index 1f07954bac96..06e7d618b6cc 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorHolderFactory.java @@ -31,7 +31,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorMakerFactory; +import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; @@ -40,19 +40,19 @@ import java.util.List; /** - * A {@link CursorMakerFactory} implementation based on a single row-based {@link Frame}. + * A {@link CursorHolderFactory} implementation based on a single row-based {@link Frame}. * * This class is only used for row-based frames. * - * @see org.apache.druid.frame.segment.columnar.FrameCursorMakerFactory the columnar version + * @see org.apache.druid.frame.segment.columnar.FrameCursorHolderFactory the columnar version */ -public class FrameCursorMakerFactory implements CursorMakerFactory +public class FrameCursorHolderFactory implements CursorHolderFactory { private final Frame frame; private final FrameReader frameReader; private final List fieldReaders; - public FrameCursorMakerFactory( + public FrameCursorHolderFactory( final Frame frame, final FrameReader frameReader, final List fieldReaders 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 ed5029cd32d0..aad70c9ecb5d 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -39,8 +39,6 @@ import org.apache.druid.query.metadata.metadata.AggregatorMergeStrategy; import org.apache.druid.query.metadata.metadata.ColumnIncluderator; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; -import org.apache.druid.query.scan.Order; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.query.search.FragmentSearchQuerySpec; diff --git a/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java b/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java index d5e7186e9548..62b42b15c782 100644 --- a/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.WrappedSegmentReference; import org.apache.druid.utils.JvmUtils; +import javax.annotation.Nullable; import java.util.List; import java.util.Objects; import java.util.Set; @@ -67,7 +68,7 @@ public DimFilter getFilter() return filter; } - private FilteredDataSource(DataSource base, DimFilter filter) + private FilteredDataSource(DataSource base, @Nullable DimFilter filter) { this.base = base; this.filter = filter; @@ -76,7 +77,7 @@ private FilteredDataSource(DataSource base, DimFilter filter) @JsonCreator public static FilteredDataSource create( @JsonProperty("base") DataSource base, - @JsonProperty("filter") DimFilter f + @JsonProperty("filter") @Nullable DimFilter f ) { return new FilteredDataSource(base, f); diff --git a/processing/src/main/java/org/apache/druid/query/scan/Order.java b/processing/src/main/java/org/apache/druid/query/Order.java similarity index 97% rename from processing/src/main/java/org/apache/druid/query/scan/Order.java rename to processing/src/main/java/org/apache/druid/query/Order.java index 00821f5fed20..464aa9ed7d26 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/Order.java +++ b/processing/src/main/java/org/apache/druid/query/Order.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query.scan; +package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; diff --git a/processing/src/main/java/org/apache/druid/query/scan/OrderBy.java b/processing/src/main/java/org/apache/druid/query/OrderBy.java similarity index 98% rename from processing/src/main/java/org/apache/druid/query/scan/OrderBy.java rename to processing/src/main/java/org/apache/druid/query/OrderBy.java index f1ecf95277cf..5cdda9bb46ec 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/OrderBy.java +++ b/processing/src/main/java/org/apache/druid/query/OrderBy.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query.scan; +package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 6be01e0d108b..fa1f42d4c13d 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -40,10 +40,8 @@ import org.apache.druid.query.timeboundary.TimeBoundaryQuery; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQuery; -import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTimeZone; import org.joda.time.Duration; @@ -276,22 +274,17 @@ default Set getRequiredColumns() return null; } - default CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + /** + * Returns an interval if {@link #getIntervals()} has only a single interval, else explodes + */ + default Interval getSingleInterval() { - final Interval interval = CollectionUtils.getOnlyElement( + return CollectionUtils.getOnlyElement( getIntervals(), (i) -> DruidException.defensive( "This method can only be called after query is reduced to a single segment interval, got [%s]", i ) ); - - return CursorBuildSpec.builder() - .setInterval(interval) - .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setVirtualColumns(getVirtualColumns()) - .setQueryContext(context()) - .setQueryMetrics(queryMetrics) - .build(); } } 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 3ae82a9f2a12..761aa43b1b3f 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 @@ -33,7 +33,6 @@ import com.google.common.primitives.Longs; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -48,7 +47,6 @@ import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; @@ -64,15 +62,12 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -808,24 +803,10 @@ public Set getRequiredColumns() ); } - @Override - public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) + @JsonIgnore + public List getGroupingColumns() { - final Interval interval = CollectionUtils.getOnlyElement( - getIntervals(), - (i) -> DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - i - ) - ); - return CursorBuildSpec.builder() - .setInterval(interval) - .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setGroupingAndVirtualColumns(getGranularity(), groupingColumns, virtualColumns) - .setAggregators(getAggregatorSpecs()) - .setQueryContext(context()) - .setQueryMetrics(queryMetrics) - .build(); + return groupingColumns; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 64eec9f153c9..e0e25fef7677 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -53,6 +53,7 @@ import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.QueryRunner; @@ -79,11 +80,13 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.StorageAdapter; +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.ColumnType; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory; import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; @@ -92,6 +95,7 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; @@ -501,7 +505,7 @@ public Sequence process( // group by specific vectorization check: - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(groupByQueryMetrics); + final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, groupByQueryMetrics); final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); @@ -843,6 +847,40 @@ private Set getAggregatorAndPostAggregatorNames(GroupByQuery query) return aggsAndPostAggs; } + public static CursorBuildSpec makeCursorBuildSpec(GroupByQuery query, @Nullable QueryMetrics queryMetrics) + { + // virtual column is currently only used as a decorator to pass to the cursor holder to allow specializing cursor + // and vector cursors if any pre-aggregated data at the matching granularity is available + // eventually this could probably be reworked to be used by the granularizer instead of the existing method + // of creating a selector on the time column + final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query.getGranularity()); + VirtualColumns virtualColumns; + List groupingColumns; + if (granularityVirtual == null) { + virtualColumns = query.getVirtualColumns(); + groupingColumns = query.getGroupingColumns(); + } else { + virtualColumns = VirtualColumns.fromIterable( + Iterables.concat( + Collections.singletonList(granularityVirtual), + () -> Arrays.stream(query.getVirtualColumns().getVirtualColumns()).iterator() + ) + ); + groupingColumns = ImmutableList.builder() + .add(granularityVirtual.getOutputName()) + .addAll(query.getGroupingColumns()).build(); + } + return CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setVirtualColumns(virtualColumns) + .setGroupingColumns(groupingColumns) + .setAggregators(query.getAggregatorSpecs()) + .setQueryContext(query.context()) + .setQueryMetrics(queryMetrics) + .build(); + } + /** * Returns the cardinality of array needed to do array-based aggregation, or -1 if array-based aggregation diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java index ac9a2f9e4d54..aed132b745ee 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java @@ -25,18 +25,17 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.scan.Order; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.List; import java.util.Map; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index fec02576ed47..037148f50c3a 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -27,7 +27,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; @@ -36,12 +35,12 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Queries; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -49,9 +48,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature.Builder; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.utils.CollectionUtils; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Collections; @@ -418,26 +414,6 @@ public Set getRequiredColumns() } } - @Override - public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) - { - final Interval interval = CollectionUtils.getOnlyElement( - getIntervals(), - (i) -> DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - i - ) - ); - return CursorBuildSpec.builder() - .setInterval(interval) - .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setVirtualColumns(getVirtualColumns()) - .setPreferredOrdering(getOrderBys()) - .setQueryContext(context()) - .setQueryMetrics(queryMetrics) - .build(); - } - public ScanQuery withOffset(final long newOffset) { return Druids.ScanQueryBuilder.copy(this).offset(newOffset).build(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 87b316dbf0cf..c1cc76fc8925 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -29,12 +29,14 @@ import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; @@ -42,6 +44,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.filter.Filters; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -114,7 +117,7 @@ public Sequence process( // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); - final CursorHolder cursorHolder = adapter.makeCursorHolder(query.asCursorBuildSpec(queryMetrics)); + final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -236,4 +239,16 @@ private long calculateRemainingScanRowsLimit(ScanQuery query, ResponseContext re } return query.getScanRowsLimit(); } + + public static CursorBuildSpec makeCursorBuildSpec(ScanQuery query, @Nullable QueryMetrics queryMetrics) + { + return CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setVirtualColumns(query.getVirtualColumns()) + .setPreferredOrdering(query.getOrderBys()) + .setQueryContext(query.context()) + .setQueryMetrics(queryMetrics) + .build(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 69681891c998..45cd4c2624c0 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.YieldingAccumulator; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.query.Order; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 8e85476e7bbc..f09b42c27959 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.query.Order; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index c74d52464f1e..e4c353a81856 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -20,6 +20,7 @@ package org.apache.druid.query.scan; import com.google.common.primitives.Longs; +import org.apache.druid.query.Order; import java.util.Comparator; diff --git a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index af13c6cb8887..b6d0cb5f11a5 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.filter.Filters; import java.util.List; @@ -74,7 +75,17 @@ public CursorBasedExecutor( public Object2IntRBTreeMap execute(final int limit) { final StorageAdapter adapter = segment.asStorageAdapter(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter( + Filters.convertToCNFFromQueryContext( + query, + Filters.toFilter(query.getFilter()) + ) + ) + .setVirtualColumns(query.getVirtualColumns()) + .setQueryContext(query.context()) + .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 1734b5b310be..cbef078435ab 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.ChainedExecutionQueryRunner; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; @@ -37,7 +38,6 @@ import org.apache.druid.query.Result; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; @@ -45,6 +45,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.filter.Filters; import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -117,7 +118,7 @@ public Result apply(Cursor cursor) @Nullable private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legacyQuery, boolean descending) { - final CursorBuildSpec.CursorBuildSpecBuilder bob = CursorBuildSpec.builder(legacyQuery.asCursorBuildSpec(null)); + final CursorBuildSpec.CursorBuildSpecBuilder bob = CursorBuildSpec.builder(makeCursorBuildSpec(legacyQuery)); if (descending) { bob.setPreferredOrdering(Collections.singletonList(OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME))); } else { @@ -232,4 +233,14 @@ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, fi // Passed all checks. return true; } + + public static CursorBuildSpec makeCursorBuildSpec(TimeBoundaryQuery query) + { + return CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setVirtualColumns(query.getVirtualColumns()) + .setQueryContext(query.context()) + .build(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 6459003d6d45..85ddf5aaa62f 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -26,7 +26,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.commons.lang.StringUtils; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; @@ -34,22 +33,15 @@ import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec.LimitJsonIncludeFilter; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.utils.CollectionUtils; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; @@ -203,34 +195,6 @@ public Set getRequiredColumns() ); } - - @Override - public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) - { - final Interval interval = CollectionUtils.getOnlyElement( - getIntervals(), - (i) -> DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - i - ) - ); - return CursorBuildSpec.builder() - .setInterval(interval) - .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setGroupingAndVirtualColumns(getGranularity(), null, virtualColumns) - .setAggregators(getAggregatorSpecs()) - .setQueryContext(context()) - .setPreferredOrdering( - Collections.singletonList( - isDescending() ? - OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) : - OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME) - ) - ) - .setQueryMetrics(queryMetrics) - .build(); - } - @Override public TimeseriesQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index 6461bd223043..b925deb54ba4 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -27,11 +27,14 @@ import org.apache.druid.collections.StupidPool; import org.apache.druid.guice.annotations.Global; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.CursorGranularizer; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorAdapters; @@ -39,15 +42,22 @@ import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -95,7 +105,7 @@ public Sequence> process( final Granularity gran = query.getGranularity(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(query.asCursorBuildSpec(timeseriesQueryMetrics)); + final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); final Sequence> result; if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize(), cursorHolder::close)) { @@ -307,4 +317,43 @@ private Sequence> processNonVectorized( ) .filter(Objects::nonNull); } + + public static CursorBuildSpec makeCursorBuildSpec(TimeseriesQuery query, @Nullable QueryMetrics queryMetrics) + { + // virtual column is currently only used as a decorator to pass to the cursor holder to allow specializing cursor + // and vector cursors if any pre-aggregated data at the matching granularity is available + // eventually this could probably be reworked to be used by the granularizer instead of the existing method + // of creating a selector on the time column + final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query.getGranularity()); + VirtualColumns virtualColumns; + List groupingColumns; + if (granularityVirtual == null) { + virtualColumns = query.getVirtualColumns(); + groupingColumns = null; + } else { + virtualColumns = VirtualColumns.fromIterable( + Iterables.concat( + Collections.singletonList(granularityVirtual), + () -> Arrays.stream(query.getVirtualColumns().getVirtualColumns()).iterator() + ) + ); + groupingColumns = Collections.singletonList(granularityVirtual.getOutputName()); + } + return CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setGroupingColumns(groupingColumns) + .setVirtualColumns(virtualColumns) + .setAggregators(query.getAggregatorSpecs()) + .setQueryContext(query.context()) + .setPreferredOrdering( + Collections.singletonList( + query.isDescending() ? + OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) : + OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME) + ) + ) + .setQueryMetrics(queryMetrics) + .build(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 1e186bd5329d..2534c37f90f6 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -24,26 +24,20 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.utils.CollectionUtils; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; @@ -183,31 +177,6 @@ public Set getRequiredColumns() ); } - - @Override - public CursorBuildSpec asCursorBuildSpec(@Nullable QueryMetrics queryMetrics) - { - final Interval interval = CollectionUtils.getOnlyElement( - getIntervals(), - (i) -> DruidException.defensive( - "This method can only be called after query is reduced to a single segment interval, got [%s]", - i - ) - ); - return CursorBuildSpec.builder() - .setInterval(interval) - .setFilter(Filters.convertToCNFFromQueryContext(this, Filters.toFilter(getFilter()))) - .setGroupingAndVirtualColumns( - getGranularity(), - Collections.singletonList(dimensionSpec.getDimension()), - virtualColumns - ) - .setAggregators(getAggregatorSpecs()) - .setQueryContext(context()) - .setQueryMetrics(queryMetrics) - .build(); - } - public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) { if (dimensionSpec.getExtractionFn() != null) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index e6d7830e6559..fa2e7c54c018 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -20,10 +20,14 @@ package org.apache.druid.query.topn; import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.CursorGranularizer; +import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.extraction.ExtractionFn; @@ -32,13 +36,19 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; +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.Types; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.Filters; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; /** */ @@ -72,7 +82,7 @@ public Sequence> query( final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(queryMetrics); + final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { @@ -201,6 +211,41 @@ private static boolean canUsePooledAlgorithm( } } + public static CursorBuildSpec makeCursorBuildSpec(TopNQuery query, @Nullable QueryMetrics queryMetrics) + { + // virtual column is currently only used as a decorator to pass to the cursor holder to allow specializing cursor + // and vector cursors if any pre-aggregated data at the matching granularity is available + // eventually this could probably be reworked to be used by the granularizer instead of the existing method + // of creating a selector on the time column + final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query.getGranularity()); + VirtualColumns virtualColumns; + List groupingColumns; + if (granularityVirtual == null) { + virtualColumns = query.getVirtualColumns(); + groupingColumns = null; + } else { + virtualColumns = VirtualColumns.fromIterable( + Iterables.concat( + Collections.singletonList(granularityVirtual), + () -> Arrays.stream(query.getVirtualColumns().getVirtualColumns()).iterator() + ) + ); + groupingColumns = ImmutableList.of( + granularityVirtual.getOutputName(), + query.getDimensionSpec().getDimension() + ); + } + return CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setGroupingColumns(groupingColumns) + .setVirtualColumns(virtualColumns) + .setAggregators(query.getAggregatorSpecs()) + .setQueryContext(query.context()) + .setQueryMetrics(queryMetrics) + .build(); + } + /** * {@link ExtractionFn} which are one to one may have their execution deferred until as late as possible, since * which value is used as the grouping key itself doesn't particularly matter. For top-n, this method allows the diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index d53b87567536..5cb8f978295a 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -24,7 +24,7 @@ /** * Cursor is an interface for iteration over a range of data points, used during query execution. Cursors are available - * from {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. + * from {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. * * A typical usage pattern might look something like this: *
diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
index 50a1a39a6282..d8f680da016f 100644
--- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
+++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
@@ -19,23 +19,17 @@
 
 package org.apache.druid.segment;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.query.Order;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.QueryContext;
 import org.apache.druid.query.QueryMetrics;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.filter.Filter;
-import org.apache.druid.query.scan.Order;
-import org.apache.druid.query.scan.OrderBy;
 import org.apache.druid.segment.column.ColumnHolder;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
 
 public class CursorBuildSpec
@@ -89,55 +83,100 @@ public CursorBuildSpec(
     this.queryMetrics = queryMetrics;
   }
 
+  /**
+   * {@link Filter} to supply to the {@link CursorHolder}. Only rows which match will be available through the
+   * selectors created from the {@link Cursor} or {@link org.apache.druid.segment.vector.VectorCursor}
+   */
   @Nullable
   public Filter getFilter()
   {
     return filter;
   }
 
+  /**
+   * {@link Interval} filter to supply to the {@link CursorHolder}. Only rows whose timestamps fall within this range
+   * will be available through the selectors created from the {@link Cursor} or
+   * {@link org.apache.druid.segment.vector.VectorCursor}
+   */
   public Interval getInterval()
   {
     return interval;
   }
 
+  /**
+   * Any columns which will be used for grouping by a query engine for the {@link CursorHolder}, useful for
+   * specializing the {@link Cursor} or {@link org.apache.druid.segment.vector.VectorCursor} if any pre-aggregated
+   * data is available.
+   */
   @Nullable
   public List getGroupingColumns()
   {
     return groupingColumns;
   }
 
+  /**
+   * Any {@link VirtualColumns} which are used by a query engine to assist in
+   * determining if {@link CursorHolder#canVectorize()}
+   */
   public VirtualColumns getVirtualColumns()
   {
     return virtualColumns;
   }
 
+  /**
+   * Any {@link AggregatorFactory} which will be used by a query engine for the {@link CursorHolder}, useful
+   * to assist in determining if {@link CursorHolder#canVectorize()}, as well as specializing the {@link Cursor} or
+   * {@link org.apache.druid.segment.vector.VectorCursor} if any pre-aggregated data is available.
+   */
   @Nullable
   public List getAggregators()
   {
     return aggregators;
   }
 
+  /**
+   * List of all {@link OrderBy} columns which a query engine will use to sort its results to supply to the
+   * {@link CursorHolder}, which can allow optimization of the provided {@link Cursor} or
+   * {@link org.apache.druid.segment.vector.VectorCursor} if data matching the preferred ordering is available.
+   * 

+ * If not specified, the cursor will advance in the native order of the underlying data. + */ @Nullable public List getPreferredOrdering() { return orderByColumns; } + /** + * {@link QueryContext} for the {@link CursorHolder} to provide a mechanism to push various data into + * {@link Cursor} and {@link org.apache.druid.segment.vector.VectorCursor} such as + * {@link org.apache.druid.query.QueryContexts#VECTORIZE_KEY} and + * {@link org.apache.druid.query.QueryContexts#VECTOR_SIZE_KEY} + */ public QueryContext getQueryContext() { return queryContext; } + /** + * {@link QueryMetrics} to use for measuring things involved with {@link Cursor} and + * {@link org.apache.druid.segment.vector.VectorCursor} creation. + */ @Nullable public QueryMetrics getQueryMetrics() { return queryMetrics; } + /** + * Check if the first {@link OrderBy} column of {@link CursorBuildSpec#getPreferredOrdering()} is + * {@link Order#DESCENDING}, which allow {@link Cursor} on time ordered data to advance in descending order if + * possible. + */ public static boolean preferDescendingTimeOrder(@Nullable List preferredOrdering) { - if (preferredOrdering != null && preferredOrdering.size() == 1) { - final OrderBy orderBy = Iterables.getOnlyElement(preferredOrdering); + if (preferredOrdering != null && !preferredOrdering.isEmpty()) { + final OrderBy orderBy = preferredOrdering.get(0); return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()) && Order.DESCENDING == orderBy.getOrder(); } return false; @@ -178,69 +217,74 @@ private CursorBuildSpecBuilder(CursorBuildSpec buildSpec) this.queryMetrics = buildSpec.queryMetrics; } + /** + * @see CursorBuildSpec#getFilter() + */ public CursorBuildSpecBuilder setFilter(@Nullable Filter filter) { this.filter = filter; return this; } + /** + * @see CursorBuildSpec#getInterval() + */ public CursorBuildSpecBuilder setInterval(Interval interval) { this.interval = interval; return this; } - public CursorBuildSpecBuilder setGroupingAndVirtualColumns( - Granularity granularity, - @Nullable List groupingColumns, - VirtualColumns virtualColumns + /** + * @see CursorBuildSpec#getGroupingColumns() + */ + public CursorBuildSpecBuilder setGroupingColumns( + @Nullable List groupingColumns ) { - final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(granularity); - if (granularityVirtual == null) { - this.virtualColumns = virtualColumns; - this.groupingColumns = groupingColumns; - } else { - this.virtualColumns = VirtualColumns.fromIterable( - Iterables.concat( - Collections.singletonList(granularityVirtual), - () -> Arrays.stream(virtualColumns.getVirtualColumns()).iterator() - ) - ); - ImmutableList.Builder bob = ImmutableList.builder() - .add(granularityVirtual.getOutputName()); - if (groupingColumns != null) { - bob.addAll(groupingColumns); - } - this.groupingColumns = bob.build(); - } + this.groupingColumns = groupingColumns; return this; } + /** + * @see CursorBuildSpec#getVirtualColumns() + */ public CursorBuildSpecBuilder setVirtualColumns(VirtualColumns virtualColumns) { this.virtualColumns = virtualColumns; return this; } + /** + * @see CursorBuildSpec#getAggregators() + */ public CursorBuildSpecBuilder setAggregators(@Nullable List aggregators) { this.aggregators = aggregators; return this; } + /** + * @see CursorBuildSpec#getPreferredOrdering() + */ public CursorBuildSpecBuilder setPreferredOrdering(@Nullable List orderBy) { this.preferredOrdering = orderBy; return this; } + /** + * @see CursorBuildSpec#getQueryContext() + */ public CursorBuildSpecBuilder setQueryContext(QueryContext queryContext) { this.queryContext = queryContext; return this; } + /** + * @see CursorBuildSpec#getQueryMetrics() + */ public CursorBuildSpecBuilder setQueryMetrics(@Nullable QueryMetrics queryMetrics) { this.queryMetrics = queryMetrics; diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java index 7a259d83d488..551430a00e33 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -35,8 +35,8 @@ * @see StorageAdapter * * @deprecated This interface is deprecated and no longer implemented by any built-in {@link StorageAdapter}. Callers - * should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. Implementors should implement - * {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. + * should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. Implementors should implement + * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. */ @Deprecated public interface CursorFactory @@ -47,9 +47,9 @@ public interface CursorFactory * Query engines should use this before running in vectorized mode, and be prepared to fall back to non-vectorized * mode if this method returns false. * - * @deprecated Callers should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} and call + * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call * {@link CursorHolder#canVectorize()}. - * Implementors should implement {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is + * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is * no longer implemented by any built-in factories. */ @Deprecated @@ -60,16 +60,16 @@ default boolean canVectorize( ) { throw DruidException.defensive( - "CursorFactory.canVectorize is no longer supported, use CursorMakerFactory.makeCursorHolder instead" + "CursorFactory.canVectorize is no longer supported, use CursorHolderFactory.makeCursorHolder instead" ); } /** * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity). * - * @deprecated Callers should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} and call + * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call * {@link CursorHolder#asCursor()}. - * Implementors should implement {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. + * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. * This method is no longer implemented by any built-in factories. */ @Deprecated @@ -83,7 +83,7 @@ default Sequence makeCursors( ) { throw DruidException.defensive( - "CursorFactory.makeCursors is no longer supported, use CursorMakerFactory.makeCursorHolder instead" + "CursorFactory.makeCursors is no longer supported, use CursorHolderFactory.makeCursorHolder instead" ); } @@ -95,9 +95,9 @@ default Sequence makeCursors( * Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval * of this segment). * - * @deprecated Callers should use {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} and call + * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call * {@link CursorHolder#asVectorCursor()}. Implementors should implement - * {@link CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is no longer implemented by any + * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is no longer implemented by any * built-in factories. */ @Deprecated @@ -112,7 +112,7 @@ default VectorCursor makeVectorCursor( ) { throw DruidException.defensive( - "CursorFactory.makeVectorCursor is no longer supported, use CursorMakerFactory.makeCursorHolder instead" + "CursorFactory.makeVectorCursor is no longer supported, use CursorHolderFactory.makeCursorHolder instead" ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java similarity index 96% rename from processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java rename to processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java index 89eb0f88e87d..e191f033f593 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorMakerFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java @@ -19,7 +19,7 @@ package org.apache.druid.segment; -public interface CursorMakerFactory +public interface CursorHolderFactory { CursorHolder makeCursorHolder(CursorBuildSpec spec); } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index b1b19b6a032b..b4b0ddfbb60b 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -32,10 +32,11 @@ public class FilteredStorageAdapter implements StorageAdapter { + @Nullable private final DimFilter filterOnDataSource; private final StorageAdapter baseStorageAdapter; - public FilteredStorageAdapter(final StorageAdapter adapter, final DimFilter filter) + public FilteredStorageAdapter(final StorageAdapter adapter, @Nullable final DimFilter filter) { this.baseStorageAdapter = adapter; this.filterOnDataSource = filter; @@ -53,7 +54,11 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec) newFilter = null; } } else { - newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter())); + if (filterOnDataSource != null) { + newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter())); + } else { + newFilter = spec.getFilter(); + } } buildSpecBuilder.setFilter(newFilter); return baseStorageAdapter.makeCursorHolder(buildSpecBuilder.build()); diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index 1cfc56b352d1..4c0635ee91d2 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -35,7 +35,7 @@ /** */ @PublicApi -public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorMakerFactory +public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHolderFactory { /** diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index 9ecfa77e54a5..57b23ebdbfae 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java @@ -20,10 +20,11 @@ package org.apache.druid.segment.vector; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolderFactory; /** * Vectorized cursor used during query execution. VectorCursors are available from - * {@link org.apache.druid.segment.CursorMakerFactory#makeCursorHolder(CursorBuildSpec)} via + * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via * {@link org.apache.druid.segment.CursorHolder#asVectorCursor()}. * * See {@link org.apache.druid.query.vector.VectorCursorGranularizer} for a helper that makes it easier for query diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index acce8020acef..3ae83e8c895b 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.extraction.TimeFormatExtractionFn; @@ -35,7 +36,6 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; @@ -384,9 +384,9 @@ private void assertCursorMatch(final Function call try (final CursorHolder queryableMaker = call.apply(queryableAdapter); final CursorHolder frameMaker = call.apply(frameAdapter)) { final Sequence> queryableRows = - FrameTestUtil.readRowsFromCursor(queryableMaker.asCursor(), signature); + FrameTestUtil.readRowsFromCursor(advanceAndReset(queryableMaker.asCursor()), signature); final Sequence> frameRows = - FrameTestUtil.readRowsFromCursor(frameMaker.asCursor(), signature); + FrameTestUtil.readRowsFromCursor(advanceAndReset(frameMaker.asCursor()), signature); FrameTestUtil.assertRowsEqual(queryableRows, frameRows); } } @@ -398,7 +398,7 @@ private void assertVectorCursorsMatch(final Function> queryableRows = - FrameTestUtil.readRowsFromVectorCursor(cursorHolder.asVectorCursor(), signature).withBaggage(cursorHolder); + FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(cursorHolder.asVectorCursor()), signature).withBaggage(cursorHolder); final Sequence> frameRows = FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameCursorHolder.asVectorCursor()), signature) .withBaggage(frameCursorHolder); diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java index 0f6dbe60c792..d8b2539b0cca 100644 --- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java index ccaad1fa984d..59865f47d634 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java @@ -171,7 +171,7 @@ public void testAsCursorBuildSpecAllGranularity() new ExpressionVirtualColumn("v0", "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil()) ); final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index"); - Query query = GroupByQuery + GroupByQuery query = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -200,7 +200,7 @@ public void testAsCursorBuildSpecAllGranularity() ) .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); Assert.assertEquals(ImmutableList.of("quality", "market", "v0"), buildSpec.getGroupingColumns()); Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); @@ -214,7 +214,7 @@ public void testAsCursorBuildSpecDayGranularity() new ExpressionVirtualColumn("v0", "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil()) ); final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index"); - Query query = GroupByQuery + GroupByQuery query = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -243,7 +243,7 @@ public void testAsCursorBuildSpecDayGranularity() ) .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); Assert.assertEquals( ImmutableList.of(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME, "quality", "market", "v0"), diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index 6b80e817392f..adbcbc187bb4 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -655,7 +655,7 @@ private void runResults( List segments = segmentsGenerator.apply(tempFolder, closer); Supplier> runner = () -> helper.runQueryOnSegmentsObjs(segments, groupQuery).toList(); - final CursorBuildSpec spec = groupQuery.asCursorBuildSpec(null); + final CursorBuildSpec spec = GroupingEngine.makeCursorBuildSpec(groupQuery, null); boolean allCanVectorize = segments.stream() .allMatch( s -> { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java index 47e6dde69b0d..ce38427728c9 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java @@ -27,6 +27,7 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.epinephelinae.VectorGrouper; import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine.VectorGroupByEngineIterator; import org.apache.druid.segment.ColumnProcessors; @@ -61,7 +62,9 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException .setAggregatorSpecs(factory) .build(); final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); - final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(query.asCursorBuildSpec(null)); + final CursorHolder cursorHolder = storageAdapter.makeCursorHolder( + GroupingEngine.makeCursorBuildSpec(query, null) + ); final ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[4096]); final VectorCursor cursor = cursorHolder.asVectorCursor(); final List dimensions = query.getDimensions().stream().map( diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java index a3390bef347b..e187ddfd630a 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.context.ResponseContext; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index e6c915b66c57..2bfd47fb6990 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.context.ResponseContext; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 4c841a35e27b..794656198d00 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerTestHelper; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index dec9eb70879d..8dd183906189 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -41,6 +41,7 @@ import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.Druids; import org.apache.druid.query.MetricsEmittingQueryRunner; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index 06b530bbdbe9..f5ce99e5a3ef 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.TableDataSource; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index 5995a640a6e1..f5a784b3841e 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -29,6 +29,8 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -441,7 +443,7 @@ public void testAsCursorBuildSpec() .columns("foo", "bar") .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = ScanQueryEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); Assert.assertNull(buildSpec.getGroupingColumns()); Assert.assertNull(buildSpec.getAggregators()); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index f2d8ebb28d02..91f0ef0c4a69 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -21,6 +21,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.column.ColumnHolder; diff --git a/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java index a70eae07ec46..a69e42bcd4df 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerTestHelper; diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index d7421690b500..727086b7be3f 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -662,7 +662,7 @@ private void runResults( List segments = segmentsGenerator.apply(tempFolder, closer); Supplier>> runner = () -> helper.runQueryOnSegmentsObjs(segments, timeseriesQuery).toList(); - final CursorBuildSpec spec = timeseriesQuery.asCursorBuildSpec(null); + final CursorBuildSpec spec = TimeseriesQueryEngine.makeCursorBuildSpec(timeseriesQuery, null); boolean allCanVectorize = segments.stream() .allMatch( s -> { diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java index 04f0ae9e4825..2174841006ef 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java @@ -136,7 +136,7 @@ public void testAsCursorBuildSpecAllGranularity() .descending(descending) .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FULL_ON_INTERVAL, buildSpec.getInterval()); Assert.assertNull(buildSpec.getGroupingColumns()); Assert.assertEquals( @@ -179,7 +179,7 @@ public void testAsCursorBuildSpecDayGranularity() .descending(descending) .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FULL_ON_INTERVAL, buildSpec.getInterval()); Assert.assertEquals( Collections.singletonList(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME), diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java index 303ea85efce9..d9493278ad26 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java @@ -290,7 +290,7 @@ public void testAsCursorBuildSpecAllGranularity() .threshold(100) .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = TopNQueryEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); Assert.assertEquals(ImmutableList.of("v"), buildSpec.getGroupingColumns()); Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); @@ -316,7 +316,7 @@ public void testAsCursorBuildSpecDayGranularity() .threshold(100) .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = TopNQueryEngine.makeCursorBuildSpec(query, null); Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval()); Assert.assertEquals( ImmutableList.of(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME, "v"), diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index cf303fa609a1..ad631e68728e 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -33,9 +33,9 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.CursorGranularizer; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index cd8f5d5004be..b2755687c4e2 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.JavaScriptAggregatorFactory; @@ -51,9 +52,9 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; @@ -144,7 +145,7 @@ public void testSanity() throws Exception .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .addOrderByColumn("billy") .build(); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); try ( CloseableStupidPool pool = new CloseableStupidPool<>( @@ -221,7 +222,7 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception .addOrderByColumn("billy") .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); try ( CloseableStupidPool pool = new CloseableStupidPool<>( "GroupByQueryEngine-bufferPool", @@ -488,7 +489,7 @@ public void testFilterByNull() throws Exception .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); try ( CloseableStupidPool pool = new CloseableStupidPool<>( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index d1a25b54e954..46580a5eeeed 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -35,12 +35,12 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index b1226c3d8454..dc127fa7f861 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -51,6 +51,8 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.FilteredDataSource; import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; @@ -74,8 +76,6 @@ import org.apache.druid.query.operator.ScanOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.ordering.StringComparator; -import org.apache.druid.query.scan.Order; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.query.timeboundary.TimeBoundaryQuery; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java index a7a30cd4ca6e..813722e6990e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java @@ -36,13 +36,13 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.CalciteCatalogIngestionDmlTest.CatalogIngestionDmlComponentSupplier; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 13cd9a42dd8b..ed4aa6d91ebe 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -33,12 +33,12 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index ddc2b69bcb27..f407cff22d84 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -41,6 +41,7 @@ import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.Order; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; @@ -71,7 +72,6 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.timeboundary.TimeBoundaryQuery; import org.apache.druid.query.topn.DimensionTopNMetricSpec; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 3d19408f16df..7566d7f77863 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -44,6 +44,7 @@ import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.OperatorFactoryBuilders; +import org.apache.druid.query.Order; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; @@ -106,7 +107,6 @@ import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.operator.window.WindowFrame; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQuery.ResultFormat; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index d75e2a16e1b7..cb819c3634a1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -30,11 +30,11 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.scan.OrderBy; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index 9e003eac3a46..6e06c5d4b5d2 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -40,7 +41,6 @@ import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.topn.DimensionTopNMetricSpec; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index 83f76de749bb..03d9357c7d48 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -37,6 +37,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -66,7 +67,6 @@ import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.query.scan.Order; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.topn.DimensionTopNMetricSpec; import org.apache.druid.query.topn.TopNQueryBuilder; From 6e2ed9647aaf4fe93daaed24b556e17c7e3e5f64 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 9 Aug 2024 12:30:37 -0700 Subject: [PATCH 55/74] fix mistake --- .../main/java/org/apache/druid/query/topn/TopNQueryEngine.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index fa2e7c54c018..b660cb1d5ebe 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -222,7 +222,7 @@ public static CursorBuildSpec makeCursorBuildSpec(TopNQuery query, @Nullable Que List groupingColumns; if (granularityVirtual == null) { virtualColumns = query.getVirtualColumns(); - groupingColumns = null; + groupingColumns = Collections.singletonList(query.getDimensionSpec().getDimension()); } else { virtualColumns = VirtualColumns.fromIterable( Iterables.concat( From b4a48a632d34ba62f286b58dde1d433891bc347a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 9 Aug 2024 22:40:15 -0700 Subject: [PATCH 56/74] loosen contract of mark resetToMark, add tests --- .../java/org/apache/druid/segment/Cursor.java | 56 +- .../apache/druid/segment/RowBasedCursor.java | 20 +- .../druid/segment/UnnestDimensionCursor.java | 4 + .../druid/segment/join/HashJoinEngine.java | 16 +- .../druid/segment/vector/VectorCursor.java | 4 +- .../segment/RowBasedStorageAdapterTest.java | 205 ++++++- .../HashJoinSegmentStorageAdapterTest.java | 506 +++++++++++++----- .../druid/segment/join/JoinTestHelper.java | 118 +++- 8 files changed, 762 insertions(+), 167 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 5cb8f978295a..352a0874164c 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -25,7 +25,7 @@ /** * Cursor is an interface for iteration over a range of data points, used during query execution. Cursors are available * from {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. - * + *

* A typical usage pattern might look something like this: *

  *   try (CursorHolder cursorHolder = adapter.makeCursorHolder(...)) {
@@ -45,7 +45,7 @@
  * {@link QueryableIndexCursorHolder.QueryableIndexCursor} is an implementation for historical segments, and
  * {@link IncrementalIndexCursorHolder.IncrementalIndexCursor} is an implementation for
  * {@link org.apache.druid.segment.incremental.IncrementalIndex}.
- *
+ * 

* Cursor is conceptually similar to {@link TimeAndDimsPointer}, but the latter is used for historical segment creation * rather than query execution (as Cursor). If those abstractions could be collapsed (and if it is worthwhile) is yet to * be determined. @@ -61,19 +61,21 @@ public interface Cursor ColumnSelectorFactory getColumnSelectorFactory(); /** - * Advance to the next row in the cursor, checking if thread has been interrupted after advancing and possibly - * throwing {@link QueryInterruptedException} if so. + * Advance the cursor to the next position, checking if thread has been interrupted after advancing and possibly + * throwing {@link QueryInterruptedException} if so. Callers should check {@link #isDone()} or + * {@link #isDoneOrInterrupted()} before getting the next value from a selector. */ void advance(); /** - * Advance to the next row in the cursor + * Advance to the cursor to the next position. Callers should check {@link #isDone()} or + * {@link #isDoneOrInterrupted()} before getting the next value from a selector. */ void advanceUninterruptibly(); /** - * Check if the current cursor position is valid. If true, any selectors created via - * {@link #getColumnSelectorFactory()} will no longer produce values. + * Check if the current cursor position is valid, returning false if there are values to read from selectors created + * by {@link #getColumnSelectorFactory()}. If true, any such selectors will no longer produce values. */ boolean isDone(); @@ -85,17 +87,51 @@ public interface Cursor boolean isDoneOrInterrupted(); /** - * Mark a position on the cursor at the current row, which can recalled with {@link #resetToMark()}. + * Mark a position on the cursor at the current row, which can recalled with {@link #resetToMark()}. This position is + * only guaranteed to have the same timestamp as the current row when this method is called. + *

+ * In most cursors, this is able to mark and return to an exact row, but currently the only caller of this method does + * not require this contract ({@link org.apache.druid.query.topn.TopNQueryEngine}) rather it only requires that the + * first timestamp of the marked position can be returned to, since TopN uses this to make multiple passes over a + * cursor within a granularity bucket interval (and so this method only really needs to work correctly if the data is + * time ordered). + *

+ * Additionally, not all built-in implementations can be guaranteed to be able to mark an exact row due to potential + * underlying data mutability: + * - {@link RowBasedCursor} being potentially backed by mutable data, such as lookups when queried as segments + * - the cursor provided by {@link org.apache.druid.segment.join.HashJoinEngine} due to + * {@link org.apache.druid.segment.join.JoinMatcher} being backed by potentially mutable iterables, such as lookups. + * If this ever changes so that we always have completely immutable data backing cursor, then we could potentially + * strengthen this contract to be able to mark and reset to exact cursor positions, but until that point do not count + * on this behavior without examining the specific type of cursors being used. */ void mark(); /** - * Reset to position set by {@link #mark()} + * Reset to position set by {@link #mark()}. This is only guaranteed to be the first ocurring row with the same + * timestamp as the row when {@link #mark()} was called. + *

+ * In most cursors, this is able to mark and return to an exact row, but currently the only caller of this method does + * not require this contract ({@link org.apache.druid.query.topn.TopNQueryEngine}) rather it only requires that the + * first timestamp of the marked position can be returned to, since TopN uses this to make multiple passes over a + * cursor within a granularity bucket interval (and so this method only really needs to work correctly if the data is + * time ordered). + *

+ * Additionally, not all built-in implementations can be guaranteed to be able to mark an exact row due to potential + * underlying data mutability: + * - {@link RowBasedCursor} being potentially backed by mutable data, such as lookups when queried as segments + * - the cursor provided by {@link org.apache.druid.segment.join.HashJoinEngine} due to + * {@link org.apache.druid.segment.join.JoinMatcher} being backed by potentially mutable iterables, such as lookups. + * If this ever changes so that we always have completely immutable data backing cursor, then we could potentially + * strengthen this contract to be able to mark and reset to exact cursor positions, but until that point do not count + * on this behavior without examining the specific type of cursors being used. */ void resetToMark(); /** - * Reset to start of cursor and discard mark. + * Reset to start of cursor and discard mark. Most cursor implementations are backed by immutable data, but there is + * generically no guarantee that advancing through a cursor again will read exactly the same data or even number of + * rows, since the underlying data might be mutable in some cases. */ void reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index 281d22bc33fc..a43f7086e6f0 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -46,7 +47,7 @@ public class RowBasedCursor implements Cursor private final ValueMatcher valueMatcher; private long rowId = 0; - private long markId = 0; + private DateTime markTime; public RowBasedCursor( final RowWalker rowWalker, @@ -63,6 +64,7 @@ public RowBasedCursor( this.interval = interval; this.descending = descending; this.startTime = descending ? interval.getEnd().minus(1) : interval.getStart(); + this.markTime = startTime; this.columnSelectorFactory = virtualColumns.wrap( new RowBasedColumnSelectorFactory<>( rowWalker::currentRow, @@ -119,7 +121,7 @@ public boolean isDoneOrInterrupted() @Override public void mark() { - markId = rowId; + markTime = DateTimes.utc(timestampFunction.applyAsLong(rowWalker.currentRow())); } @Override @@ -127,15 +129,15 @@ public void resetToMark() { rowId = 0; rowWalker.reset(); - rowWalker.skipToDateTime(startTime, descending); - advanceToMatchingMarkRow(); + rowWalker.skipToDateTime(markTime, descending); + advanceToMatchingRow(); } @Override public void reset() { rowId = 0; - markId = 0; + markTime = startTime; rowWalker.reset(); rowWalker.skipToDateTime(startTime, descending); advanceToMatchingRow(); @@ -148,12 +150,4 @@ private void advanceToMatchingRow() rowId++; } } - - private void advanceToMatchingMarkRow() - { - while (!isDone() && rowId < markId) { - rowWalker.advance(); - rowId++; - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 7f4a8aa3806e..4958e62c5853 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -305,6 +305,10 @@ public void resetToMark() if (!baseCursor.isDone()) { indexIntsForRow = new SingleIndexInts(); indexedIntsForCurrentRow = dimSelector.getRow(); + // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid + if (index > indexIntsForRow.size()) { + index = 0; + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index 18c9cbfdef96..fc88b80bed27 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -147,8 +147,6 @@ void resetRowId() class JoinCursor implements Cursor { - private long joinMarkId = 0; - public void initialize() { matchCurrentPosition(); @@ -247,19 +245,20 @@ public boolean isDoneOrInterrupted() @Override public void mark() { - joinMarkId = joinColumnSelectorFactory.getRowId(); + // join cursor handles mark and resetToMark imperfectly. The left cursor will be marked, but the right matchers + // will be completely reset. This is fine for left and inner joins per the contract of mark and resetToMark + // since the left cursor defines the rows timestamp. However for right and full joins, in an ideal world we + // would mark the right side, so we would have a way to reset to those positions. However, the contract of + // JoinMatcher is too loose to be able to guarantee that the underlying data behind the right side cursors + leftCursor.mark(); } @Override public void resetToMark() { - leftCursor.reset(); + leftCursor.resetToMark(); joinMatcher.reset(); - joinColumnSelectorFactory.resetRowId(); initialize(); - while (!isDone() && joinColumnSelectorFactory.getRowId() < joinMarkId) { - advance(); - } } @Override @@ -268,7 +267,6 @@ public void reset() leftCursor.reset(); joinMatcher.reset(); joinColumnSelectorFactory.resetRowId(); - joinMarkId = joinColumnSelectorFactory.getRowId(); initialize(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index 57b23ebdbfae..103d769fc477 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java @@ -26,10 +26,10 @@ * Vectorized cursor used during query execution. VectorCursors are available from * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via * {@link org.apache.druid.segment.CursorHolder#asVectorCursor()}. - * + *

* See {@link org.apache.druid.query.vector.VectorCursorGranularizer} for a helper that makes it easier for query * engines to do time granularization. - * + *

* An example of how to use the methods in this class: * *

diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
index ad631e68728e..2eec513d949d 100644
--- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
@@ -22,6 +22,7 @@
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.math.LongMath;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.common.guava.GuavaUtils;
 import org.apache.druid.java.util.common.DateTimes;
@@ -36,17 +37,20 @@
 import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.filter.SelectorDimFilter;
+import org.apache.druid.query.filter.TypedInFilter;
 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;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.math.RoundingMode;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -209,14 +213,52 @@ public Function columnFunction(String columnName)
         }
       };
 
+  private static final RowAdapter SAME_TIME_ROW_ADAPTER =
+      new RowAdapter()
+      {
+        private DateTime startTime = DateTimes.nowUtc();
+        @Override
+        public ToLongFunction timestampFunction()
+        {
+          return i -> {
+            long div = LongMath.divide(i, 3, RoundingMode.FLOOR);
+            return startTime.plus(div).getMillis();
+          };
+        }
+
+        @Override
+        public Function columnFunction(String columnName)
+        {
+          if (UNKNOWN_TYPE_NAME.equals(columnName)) {
+            return i -> i;
+          } else {
+            final ValueType valueType = GuavaUtils.getEnumIfPresent(ValueType.class, columnName);
+
+            if (valueType == null || valueType == ValueType.COMPLEX) {
+              return i -> null;
+            } else {
+              return i -> DimensionHandlerUtils.convertObjectToType(
+                  i,
+                  ROW_SIGNATURE.getColumnType(columnName).orElse(null)
+              );
+            }
+          }
+        }
+      };
+
   public final AtomicLong numCloses = new AtomicLong();
 
   private RowBasedStorageAdapter createIntAdapter(final int... ints)
+  {
+    return createIntAdapter(ROW_ADAPTER, ints);
+  }
+
+  private RowBasedStorageAdapter createIntAdapter(RowAdapter adapter, final int... ints)
   {
     return new RowBasedStorageAdapter<>(
         Sequences.simple(Arrays.stream(ints).boxed().collect(Collectors.toList()))
                  .withBaggage(numCloses::incrementAndGet),
-        ROW_ADAPTER,
+        adapter,
         ROW_SIGNATURE
     );
   }
@@ -516,7 +558,7 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNull()
       );
     }
 
-    Assert.assertEquals(3, numCloses.get());
+    Assert.assertEquals(2, numCloses.get());
   }
 
   @Test
@@ -553,6 +595,7 @@ public void test_makeCursor_filterOnVirtualColumn()
     Assert.assertEquals(2, numCloses.get());
   }
 
+
   @Test
   public void test_makeCursor_descending()
   {
@@ -801,7 +844,7 @@ public void test_makeCursor_allProcessors()
       );
     }
 
-    Assert.assertEquals(3, numCloses.get());
+    Assert.assertEquals(2, numCloses.get());
   }
 
   @Test
@@ -824,6 +867,104 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull()
     Assert.assertEquals(2, numCloses.get());
   }
 
+  @Test
+  public void test_makeCursor_mark_resets_to_different_row()
+  {
+    final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 0, 1, 2, 3, 4, 5, 6, 7);
+
+    final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
+                                                     .build();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
+      Assert.assertEquals(
+          ImmutableList.of(
+              ImmutableList.of("0"),
+              ImmutableList.of("1"),
+              ImmutableList.of("2"),
+              ImmutableList.of("3"),
+              // duplicate row since mark at 4 but resets to 3 since same timestamp
+              ImmutableList.of("3"),
+              ImmutableList.of("4"),
+              ImmutableList.of("5"),
+              ImmutableList.of("6"),
+              ImmutableList.of("7")
+          ),
+          walkCursorMarkResetDifferentRow(cursor, READ_STRING, 4)
+      );
+    }
+
+    Assert.assertEquals(3, numCloses.get());
+  }
+
+  @Test
+  public void test_makeCursor_mark_resets_to_different_row_descending()
+  {
+    final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 0, 1, 2, 3, 4, 5, 6, 7);
+
+    final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
+                                                     .setPreferredOrdering(
+                                                         Collections.singletonList(
+                                                             OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
+                                                         )
+                                                     )
+                                                     .build();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
+      Assert.assertEquals(
+          ImmutableList.of(
+              ImmutableList.of("7"),
+              ImmutableList.of("6"),
+              ImmutableList.of("5"),
+              ImmutableList.of("4"),
+              // duplicate rows since mark at 4 but resets to 5 since same timestamp
+              ImmutableList.of("5"),
+              ImmutableList.of("4"),
+              ImmutableList.of("3"),
+              ImmutableList.of("2"),
+              ImmutableList.of("1"),
+              ImmutableList.of("0")
+          ),
+          walkCursorMarkResetDifferentRow(cursor, READ_STRING, 4)
+      );
+    }
+
+    Assert.assertEquals(1, numCloses.get());
+  }
+
+  @Test
+  public void test_makeCursor_filterOnLong_resets_to_different_row()
+  {
+    final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 1, 2, 3, 4, 5, 6, 7);
+
+    final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
+                                                     .setFilter(
+                                                         new TypedInFilter(
+                                                             ValueType.LONG.name(),
+                                                             ColumnType.LONG,
+                                                             null,
+                                                             Arrays.asList(3L, 4L),
+                                                             null
+                                                         )
+                                                     )
+                                                     .build();
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
+
+      Assert.assertEquals(
+          ImmutableList.of(
+              ImmutableList.of("3"),
+              // duplicate rows since cursor is marked at 1, but resets to 0 since same timestamp
+              ImmutableList.of("3"),
+              ImmutableList.of("4")
+          ),
+          walkCursorMarkResetDifferentRow(cursor, READ_STRING, 1)
+      );
+    }
+
+
+    Assert.assertEquals(3, numCloses.get());
+  }
+
   private static List> walkCursor(
       final Cursor cursor,
       final List>> processors
@@ -845,7 +986,7 @@ private static List> walkCursor(
 
     // test cursor mark/resetToMark
     int ctr = 0;
-    int mark = 1;
+    int mark = 2;
     while (!cursor.isDone()) {
       if (ctr == mark) {
         cursor.mark();
@@ -879,6 +1020,62 @@ private static List> walkCursor(
     return retVal;
   }
 
+  private static List> walkCursorMarkResetDifferentRow(
+      final Cursor cursor,
+      final List>> processors,
+      int mark
+  )
+  {
+    final List> suppliers = new ArrayList<>();
+    for (Function> processor : processors) {
+      suppliers.add(processor.apply(cursor));
+    }
+
+    final List> retVal = new ArrayList<>();
+
+    // test cursor reset
+    while (!cursor.isDone()) {
+      cursor.advanceUninterruptibly();
+    }
+
+    cursor.reset();
+
+    // test cursor mark/resetToMark
+    int ctr = 0;
+    while (!cursor.isDone()) {
+      if (ctr == mark) {
+        cursor.mark();
+      }
+      final List row = new ArrayList<>();
+
+      for (Supplier supplier : suppliers) {
+        row.add(supplier.get());
+      }
+
+      retVal.add(row);
+      ctr++;
+      cursor.advanceUninterruptibly();
+    }
+
+    if (ctr > mark) {
+      cursor.resetToMark();
+      retVal.removeAll(retVal.subList(mark, retVal.size()));
+      while (!cursor.isDone()) {
+
+        final List row = new ArrayList<>();
+
+        for (Supplier supplier : suppliers) {
+          row.add(supplier.get());
+        }
+
+        retVal.add(row);
+        cursor.advanceUninterruptibly();
+      }
+    }
+
+    return retVal;
+  }
+
   private static List> walkCursorGranularized(
       final StorageAdapter adapter,
       final Cursor cursor,
diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
index 4ddbe36c68db..59755da0c38b 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java
@@ -696,27 +696,39 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            "countryIsoCode",
-            "countryNumber",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "countryIsoCode",
+        "countryNumber",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L},
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L}
         )
     );
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L},
+            // duplicate row, right join reset to mark starts at beginning
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L},
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L}
+        ),
+        1
+    );
   }
 
   @Test
@@ -729,26 +741,38 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup(
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            "countryIsoCode",
-            "countryNumber",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "countryIsoCode",
+        "countryNumber",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"},
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"}
         )
     );
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"},
+            // duplicate row, right join reset to mark starts at beginning
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"},
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"}
+        ),
+        1
+    );
   }
 
   @Test
@@ -761,27 +785,39 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            "countryIsoCode",
-            "countryNumber",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "countryIsoCode",
+        "countryNumber",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L},
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L}
         )
     );
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L},
+            // duplicate row, full join reset to mark starts at beginning
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L},
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L}
+        ),
+        1
+    );
   }
 
   @Test
@@ -794,26 +830,38 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            "countryIsoCode",
-            "countryNumber",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "countryIsoCode",
+        "countryNumber",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"},
             new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"}
         )
     );
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"},
+            // duplicate row, full join reset to mark starts at beginning
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"},
+            new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"}
+        ),
+        1
+    );
   }
 
   @Test
@@ -1240,18 +1288,19 @@ public void test_makeCursor_factToCountryAlwaysTrue()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{"Diskussion:Sebastian Schulz", "Australia"},
             new Object[]{"Diskussion:Sebastian Schulz", "Canada"},
@@ -1273,6 +1322,36 @@ public void test_makeCursor_factToCountryAlwaysTrue()
             new Object[]{"Diskussion:Sebastian Schulz", "Fourems"}
         )
     );
+
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{"Diskussion:Sebastian Schulz", "Australia"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Canada"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Chile"},
+            // duplicate rows start here, reset to mark resets right
+            new Object[]{"Diskussion:Sebastian Schulz", "Australia"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Canada"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Chile"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Germany"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Ecuador"},
+            new Object[]{"Diskussion:Sebastian Schulz", "France"},
+            new Object[]{"Diskussion:Sebastian Schulz", "United Kingdom"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Italy"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Japan"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Republic of Korea"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Mexico"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Norway"},
+            new Object[]{"Diskussion:Sebastian Schulz", "El Salvador"},
+            new Object[]{"Diskussion:Sebastian Schulz", "United States"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Atlantis"},
+            new Object[]{"Diskussion:Sebastian Schulz", "States United"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Usca"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Fourems"}
+        ),
+        3
+    );
   }
 
   @Test
@@ -1340,18 +1419,19 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{"Diskussion:Sebastian Schulz", "Australia"},
             new Object[]{"Diskussion:Sebastian Schulz", "Canada"},
@@ -1373,6 +1453,36 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup()
             new Object[]{"Diskussion:Sebastian Schulz", "Fourems"}
         )
     );
+
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{"Diskussion:Sebastian Schulz", "Australia"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Canada"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Chile"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Germany"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Australia"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Canada"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Chile"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Germany"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Ecuador"},
+            new Object[]{"Diskussion:Sebastian Schulz", "France"},
+            new Object[]{"Diskussion:Sebastian Schulz", "United Kingdom"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Italy"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Japan"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Republic of Korea"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Mexico"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Norway"},
+            new Object[]{"Diskussion:Sebastian Schulz", "El Salvador"},
+            new Object[]{"Diskussion:Sebastian Schulz", "United States"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Atlantis"},
+            new Object[]{"Diskussion:Sebastian Schulz", "States United"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Usca"},
+            new Object[]{"Diskussion:Sebastian Schulz", "Fourems"}
+        ),
+        4
+    );
   }
 
   @Test
@@ -1635,21 +1745,22 @@ public void test_makeCursor_factToRegionTheWrongWay()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(
-            CursorBuildSpec.builder().setFilter(filter).build()
-        ),
-        ImmutableList.of(
-            "page",
-            "regionIsoCode",
-            "countryIsoCode",
-            FACT_TO_REGION_PREFIX + "regionName",
-            FACT_TO_REGION_PREFIX + "countryIsoCode"
-        ),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "regionIsoCode",
+        "countryIsoCode",
+        FACT_TO_REGION_PREFIX + "regionName",
+        FACT_TO_REGION_PREFIX + "countryIsoCode"
+    );
+    CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build();
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
         ImmutableList.of(
             new Object[]{"Giusy Ferreri discography", "VA", "IT", "Provincia di Varese", "IT"},
             new Object[]{"Giusy Ferreri discography", "VA", "IT", "Virginia", "US"},
@@ -1659,6 +1770,22 @@ public void test_makeCursor_factToRegionTheWrongWay()
             new Object[]{"Old Anatolian Turkish", "VA", "US", "Virginia", "US"}
         )
     );
+
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(buildSpec),
+        columns,
+        ImmutableList.of(
+            new Object[]{"Giusy Ferreri discography", "VA", "IT", "Provincia di Varese", "IT"},
+            new Object[]{"Giusy Ferreri discography", "VA", "IT", "Virginia", "US"},
+            new Object[]{"Roma-Bangkok", "VA", "IT", "Provincia di Varese", "IT"},
+            new Object[]{"Roma-Bangkok", "VA", "IT", "Virginia", "US"},
+            new Object[]{"Old Anatolian Turkish", "VA", "US", "Provincia di Varese", "IT"},
+            // duplicate row from mark/reset to mark
+            new Object[]{"Old Anatolian Turkish", "VA", "US", "Provincia di Varese", "IT"},
+            new Object[]{"Old Anatolian Turkish", "VA", "US", "Virginia", "US"}
+        ),
+        5
+    );
   }
 
   @Test
@@ -1967,26 +2094,87 @@ public void test_makeCursor_factToCountryRightWithBaseFilter()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            baseFilter,
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        baseFilter,
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "countryIsoCode",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
+    );
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+        columns,
         ImmutableList.of(
-            "page",
-            "countryIsoCode",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
+            new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L},
+            new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L},
+            new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L},
+            new Object[]{null, null, "AU", "Australia", 0L},
+            new Object[]{null, null, "CL", "Chile", 2L},
+            new Object[]{null, null, "DE", "Germany", 3L},
+            new Object[]{null, null, "EC", "Ecuador", 4L},
+            new Object[]{null, null, "FR", "France", 5L},
+            new Object[]{null, null, "GB", "United Kingdom", 6L},
+            new Object[]{null, null, "IT", "Italy", 7L},
+            new Object[]{null, null, "JP", "Japan", 8L},
+            new Object[]{null, null, "KR", "Republic of Korea", 9L},
+            new Object[]{null, null, "MX", "Mexico", 10L},
+            new Object[]{null, null, "NO", "Norway", 11L},
+            new Object[]{null, null, "SV", "El Salvador", 12L},
+            new Object[]{null, null, "US", "United States", 13L},
+            new Object[]{null, null, "AX", "Atlantis", 14L},
+            new Object[]{null, null, "SU", "States United", 15L},
+            new Object[]{null, null, "USCA", "Usca", 16L},
+            new Object[]{null, null, "MMMM", "Fourems", 205L}
+        )
+    );
+
+    // no dupes because mark/reset happens while advancing left
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+        columns,
+        ImmutableList.of(
+            new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L},
+            new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L},
+            new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L},
+            new Object[]{null, null, "AU", "Australia", 0L},
+            new Object[]{null, null, "CL", "Chile", 2L},
+            new Object[]{null, null, "DE", "Germany", 3L},
+            new Object[]{null, null, "EC", "Ecuador", 4L},
+            new Object[]{null, null, "FR", "France", 5L},
+            new Object[]{null, null, "GB", "United Kingdom", 6L},
+            new Object[]{null, null, "IT", "Italy", 7L},
+            new Object[]{null, null, "JP", "Japan", 8L},
+            new Object[]{null, null, "KR", "Republic of Korea", 9L},
+            new Object[]{null, null, "MX", "Mexico", 10L},
+            new Object[]{null, null, "NO", "Norway", 11L},
+            new Object[]{null, null, "SV", "El Salvador", 12L},
+            new Object[]{null, null, "US", "United States", 13L},
+            new Object[]{null, null, "AX", "Atlantis", 14L},
+            new Object[]{null, null, "SU", "States United", 15L},
+            new Object[]{null, null, "USCA", "Usca", 16L},
+            new Object[]{null, null, "MMMM", "Fourems", 205L}
         ),
+        2
+    );
+
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+        columns,
         ImmutableList.of(
             new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L},
             new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L},
             new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L},
             new Object[]{null, null, "AU", "Australia", 0L},
             new Object[]{null, null, "CL", "Chile", 2L},
+            // dupes start here from mark/reset
+            new Object[]{null, null, "AU", "Australia", 0L},
+            new Object[]{null, null, "CL", "Chile", 2L},
             new Object[]{null, null, "DE", "Germany", 3L},
             new Object[]{null, null, "EC", "Ecuador", 4L},
             new Object[]{null, null, "FR", "France", 5L},
@@ -2002,7 +2190,8 @@ public void test_makeCursor_factToCountryRightWithBaseFilter()
             new Object[]{null, null, "SU", "States United", 15L},
             new Object[]{null, null, "USCA", "Usca", 16L},
             new Object[]{null, null, "MMMM", "Fourems", 205L}
-        )
+        ),
+        5
     );
   }
 
@@ -2020,26 +2209,88 @@ public void test_makeCursor_factToCountryFullWithBaseFilter()
         joinableClauses,
         VirtualColumns.EMPTY
     );
-    JoinTestHelper.verifyCursor(
-        new HashJoinSegmentStorageAdapter(
-            factSegment.asStorageAdapter(),
-            baseFilter,
-            joinableClauses,
-            joinFilterPreAnalysis
-        ).makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+    HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
+        factSegment.asStorageAdapter(),
+        baseFilter,
+        joinableClauses,
+        joinFilterPreAnalysis
+    );
+    List columns = ImmutableList.of(
+        "page",
+        "countryIsoCode",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
+        FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
+    );
+    JoinTestHelper.verifyCursorNoMarkReset(
+        adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+        columns,
         ImmutableList.of(
-            "page",
-            "countryIsoCode",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName",
-            FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber"
+            new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L},
+            new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L},
+            new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L},
+            new Object[]{"Orange Soda", "MatchNothing", null, null, NullHandling.sqlCompatible() ? null : 0L},
+            new Object[]{null, null, "AU", "Australia", 0L},
+            new Object[]{null, null, "CL", "Chile", 2L},
+            new Object[]{null, null, "DE", "Germany", 3L},
+            new Object[]{null, null, "EC", "Ecuador", 4L},
+            new Object[]{null, null, "FR", "France", 5L},
+            new Object[]{null, null, "GB", "United Kingdom", 6L},
+            new Object[]{null, null, "IT", "Italy", 7L},
+            new Object[]{null, null, "JP", "Japan", 8L},
+            new Object[]{null, null, "KR", "Republic of Korea", 9L},
+            new Object[]{null, null, "MX", "Mexico", 10L},
+            new Object[]{null, null, "NO", "Norway", 11L},
+            new Object[]{null, null, "SV", "El Salvador", 12L},
+            new Object[]{null, null, "US", "United States", 13L},
+            new Object[]{null, null, "AX", "Atlantis", 14L},
+            new Object[]{null, null, "SU", "States United", 15L},
+            new Object[]{null, null, "USCA", "Usca", 16L},
+            new Object[]{null, null, "MMMM", "Fourems", 205L}
+        )
+    );
+
+    // no dupes because mark/reset happens while advancing left
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+        columns,
+        ImmutableList.of(
+            new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L},
+            new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L},
+            new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L},
+            new Object[]{"Orange Soda", "MatchNothing", null, null, NullHandling.sqlCompatible() ? null : 0L},
+            new Object[]{null, null, "AU", "Australia", 0L},
+            new Object[]{null, null, "CL", "Chile", 2L},
+            new Object[]{null, null, "DE", "Germany", 3L},
+            new Object[]{null, null, "EC", "Ecuador", 4L},
+            new Object[]{null, null, "FR", "France", 5L},
+            new Object[]{null, null, "GB", "United Kingdom", 6L},
+            new Object[]{null, null, "IT", "Italy", 7L},
+            new Object[]{null, null, "JP", "Japan", 8L},
+            new Object[]{null, null, "KR", "Republic of Korea", 9L},
+            new Object[]{null, null, "MX", "Mexico", 10L},
+            new Object[]{null, null, "NO", "Norway", 11L},
+            new Object[]{null, null, "SV", "El Salvador", 12L},
+            new Object[]{null, null, "US", "United States", 13L},
+            new Object[]{null, null, "AX", "Atlantis", 14L},
+            new Object[]{null, null, "SU", "States United", 15L},
+            new Object[]{null, null, "USCA", "Usca", 16L},
+            new Object[]{null, null, "MMMM", "Fourems", 205L}
         ),
+        1
+    );
+
+    JoinTestHelper.verifyCursorMarkReset(
+        adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN),
+        columns,
         ImmutableList.of(
             new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L},
             new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L},
             new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L},
             new Object[]{"Orange Soda", "MatchNothing", null, null, NullHandling.sqlCompatible() ? null : 0L},
             new Object[]{null, null, "AU", "Australia", 0L},
+            // dupes start here
+            new Object[]{null, null, "AU", "Australia", 0L},
             new Object[]{null, null, "CL", "Chile", 2L},
             new Object[]{null, null, "DE", "Germany", 3L},
             new Object[]{null, null, "EC", "Ecuador", 4L},
@@ -2056,7 +2307,8 @@ public void test_makeCursor_factToCountryFullWithBaseFilter()
             new Object[]{null, null, "SU", "States United", 15L},
             new Object[]{null, null, "USCA", "Usca", 16L},
             new Object[]{null, null, "MMMM", "Fourems", 205L}
-        )
+        ),
+        5
     );
   }
 
diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java
index b623d0d8de93..74c9e56fa8e7 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java
@@ -68,6 +68,7 @@
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.Function;
 import java.util.function.Supplier;
 import java.util.function.ToLongFunction;
@@ -356,9 +357,64 @@ public static List readCursor(final CursorHolder cursorHolder, final L
 
       cursor.reset();
 
+      while (!cursor.isDone()) {
+        final Object[] row = new Object[columns.size()];
+
+        for (int i = 0; i < row.length; i++) {
+          row[i] = readers.get(i).get();
+        }
+
+        rows.add(row);
+        if (interruptible) {
+          cursor.advance();
+        } else {
+          cursor.advanceUninterruptibly();
+        }
+
+        interruptible = !interruptible;
+      }
+
+      return rows;
+    }
+    finally {
+      cursorHolder.close();
+    }
+  }
+
+  public static List readCursorMarkReset(final CursorHolder cursorHolder, final List columns, int mark)
+  {
+    try {
+      final Cursor cursor = cursorHolder.asCursor();
+      final List> readers = columns
+          .stream()
+          .map(
+              column ->
+                  ColumnProcessors.makeProcessor(
+                      column,
+                      SIMPLE_READER,
+                      cursor.getColumnSelectorFactory()
+                  )
+          )
+          .collect(Collectors.toList());
+
+      final List rows = new ArrayList<>();
+      boolean interruptible = false; // test both advance() and advanceUninterruptibly()
+
+      // test cursor reset
+      while (!cursor.isDone()) {
+        if (interruptible) {
+          cursor.advance();
+        } else {
+          cursor.advanceUninterruptibly();
+        }
+
+        interruptible = !interruptible;
+      }
+
+      cursor.reset();
+
       // test cursor mark/resetToMark
       int ctr = 0;
-      int mark = 2;
       while (!cursor.isDone()) {
         if (ctr == mark) {
           cursor.mark();
@@ -382,6 +438,7 @@ public static List readCursor(final CursorHolder cursorHolder, final L
 
       if (rows.size() > mark) {
         cursor.resetToMark();
+        rows.removeAll(rows.subList(mark, rows.size()));
         while (!cursor.isDone()) {
           final Object[] row = new Object[columns.size()];
 
@@ -389,7 +446,7 @@ public static List readCursor(final CursorHolder cursorHolder, final L
             row[i] = readers.get(i).get();
           }
 
-          rows.set(mark++, row);
+          rows.add(row);
           if (interruptible) {
             cursor.advance();
           } else {
@@ -412,6 +469,38 @@ public static void verifyCursor(
       final List columns,
       final List expectedRows
   )
+  {
+    final int max = expectedRows.size();
+    final List rows;
+    int mark = -1;
+    if (max > 1) {
+      mark = ThreadLocalRandom.current().nextInt(1, max);
+      rows = readCursorMarkReset(cursorHolder, columns, mark);
+    } else {
+      rows = readCursor(cursorHolder, columns);
+    }
+
+    for (int i = 0; i < rows.size(); i++) {
+      try {
+        log.info("Row #%-2d: %s", i, TestHelper.JSON_MAPPER.writeValueAsString(rows.get(i)));
+      }
+      catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    Assert.assertEquals("number of rows" + (mark > 0 ? " (mark: " + mark + ")" : ""), expectedRows.size(), rows.size());
+
+    for (int i = 0; i < rows.size(); i++) {
+      Assert.assertArrayEquals("row #" + i, expectedRows.get(i), rows.get(i));
+    }
+  }
+
+  public static void verifyCursorNoMarkReset(
+      final CursorHolder cursorHolder,
+      final List columns,
+      final List expectedRows
+  )
   {
     final List rows = readCursor(cursorHolder, columns);
 
@@ -431,6 +520,31 @@ public static void verifyCursor(
     }
   }
 
+  public static void verifyCursorMarkReset(
+      final CursorHolder cursorHolder,
+      final List columns,
+      final List expectedRows,
+      int mark
+  )
+  {
+    final List rows = readCursorMarkReset(cursorHolder, columns, mark);
+
+    for (int i = 0; i < rows.size(); i++) {
+      try {
+        log.info("Row #%-2d: %s", i, TestHelper.JSON_MAPPER.writeValueAsString(rows.get(i)));
+      }
+      catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    Assert.assertEquals("number of rows", expectedRows.size(), rows.size());
+
+    for (int i = 0; i < rows.size(); i++) {
+      Assert.assertArrayEquals("row #" + i, expectedRows.get(i), rows.get(i));
+    }
+  }
+
   public static  T withRowsFromResource(
       final String resource,
       final Function>, T> f

From fd4232cf37c9b0ea166f810bc6e3241d16c104a7 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Fri, 9 Aug 2024 23:15:25 -0700
Subject: [PATCH 57/74] a bit more test

---
 .../UnnestColumnValueSelectorCursor.java      |  4 +
 .../druid/segment/UnnestDimensionCursor.java  |  8 +-
 .../segment/UnnestStorageAdapterTest.java     | 83 +++++++++++++++++++
 3 files changed, 91 insertions(+), 4 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
index 76e30b4a9270..c7f843c79787 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
@@ -257,6 +257,10 @@ public void resetToMark()
     if (unnestListForCurrentRow.isEmpty()) {
       moveToNextNonEmptyRow();
     }
+    // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid
+    if (index >= unnestListForCurrentRow.size()) {
+      index = 0;
+    }
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
index 4958e62c5853..8c2ee41b4fe1 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
@@ -305,10 +305,10 @@ public void resetToMark()
     if (!baseCursor.isDone()) {
       indexIntsForRow = new SingleIndexInts();
       indexedIntsForCurrentRow = dimSelector.getRow();
-      // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid
-      if (index > indexIntsForRow.size()) {
-        index = 0;
-      }
+    }
+    // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid
+    if (index >= indexIntsForRow.size()) {
+      index = 0;
     }
   }
 
diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
index b46fc8671c3a..37a90d6ecb10 100644
--- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
@@ -25,6 +25,7 @@
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.NestedDataTestUtils;
@@ -35,6 +36,7 @@
 import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.filter.AndFilter;
 import org.apache.druid.segment.filter.OrFilter;
@@ -65,6 +67,8 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.Function;
+import java.util.function.ToLongFunction;
 
 import static org.apache.druid.segment.filter.FilterTestUtils.not;
 import static org.apache.druid.segment.filter.FilterTestUtils.selector;
@@ -316,6 +320,85 @@ public void test_unnest_adapters_basic_array_column()
     }
   }
 
+  @Test
+  public void test_unnest_adapters_basic_row_based_array_column()
+  {
+    StorageAdapter adapter = new UnnestStorageAdapter(
+        new RowBasedStorageAdapter<>(
+            Sequences.simple(
+                Arrays.asList(
+                    new Object[]{1L, new Object[]{1L, 2L}},
+                    new Object[]{1L, new Object[]{3L, 4L, 5L}},
+                    new Object[]{2L, new Object[]{6L, null, 7L}},
+                    new Object[]{2L, null},
+                    new Object[]{3L, new Object[]{8L, 9L, 10L}}
+                )
+            ),
+            new RowAdapter()
+            {
+              @Override
+              public ToLongFunction timestampFunction()
+              {
+                return value -> (long) value[0];
+              }
+
+              @Override
+              public Function columnFunction(String columnName)
+              {
+                if (columnName.equals("a")) {
+                  return objects -> objects[1];
+                }
+                return null;
+              }
+            },
+            RowSignature.builder().add("arrayLongNulls", ColumnType.LONG_ARRAY).build()
+        ),
+        new ExpressionVirtualColumn("u", "\"a\"", ColumnType.LONG, ExprMacroTable.nil()),
+        null
+    );
+    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      Cursor cursor = cursorHolder.asCursor();
+
+      ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
+
+      ColumnValueSelector dimSelector = factory.makeColumnValueSelector("u");
+      int count = 0;
+
+      List rows = new ArrayList<>();
+      // test cursor reset
+      while (!cursor.isDone()) {
+        cursor.advance();
+      }
+      cursor.reset();
+
+      // test cursor mark/resetToMark
+      int mark = 4;
+      while (!cursor.isDone()) {
+        if (count == mark) {
+          cursor.mark();
+        }
+        Object dimSelectorVal = dimSelector.getObject();
+        rows.add(dimSelectorVal);
+        cursor.advance();
+        count++;
+      }
+      cursor.resetToMark();
+      rows.removeAll(rows.subList(mark, rows.size()));
+      while (!cursor.isDone()) {
+        Object dimSelectorVal = dimSelector.getObject();
+        rows.add(dimSelectorVal);
+        cursor.advance();
+      }
+      Assert.assertEquals(count, 11);
+      Assert.assertEquals(
+          // marked at position 4, however first row has the same timestamp so we effectively restart cursor after
+          // position 4
+          Arrays.asList(1, 2, 3, 4, 1, 2, 3, 4, 5, 6, null, 7, 8, 9, 10),
+          rows
+      );
+    }
+  }
+
   @Test
   public void test_two_levels_of_unnest_adapters()
   {

From dae82e2f5901b6fae39afd6b59468b01c9b23ebe Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Sat, 10 Aug 2024 02:13:47 -0700
Subject: [PATCH 58/74] fix

---
 .../apache/druid/segment/UnnestColumnValueSelectorCursor.java | 4 +++-
 .../java/org/apache/druid/segment/UnnestDimensionCursor.java  | 3 ++-
 .../org/apache/druid/segment/UnnestStorageAdapterTest.java    | 2 +-
 3 files changed, 6 insertions(+), 3 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
index c7f843c79787..cb2f33f5b929 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
@@ -24,6 +24,7 @@
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.column.ColumnCapabilities;
+import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
@@ -61,6 +62,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
   private final String outputName;
   private int index;
   private Object currentVal;
+  @MonotonicNonNull
   private List unnestListForCurrentRow;
   private boolean needInitialization;
   private int markIndex = 0;
@@ -258,7 +260,7 @@ public void resetToMark()
       moveToNextNonEmptyRow();
     }
     // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid
-    if (index >= unnestListForCurrentRow.size()) {
+    if (unnestListForCurrentRow == null || index >= unnestListForCurrentRow.size()) {
       index = 0;
     }
   }
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
index 8c2ee41b4fe1..064d94cb01c8 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
@@ -73,6 +73,7 @@ public class UnnestDimensionCursor implements Cursor
   @MonotonicNonNull
   private IndexedInts indexedIntsForCurrentRow;
   private boolean needInitialization;
+  @MonotonicNonNull
   private SingleIndexInts indexIntsForRow;
   private final int nullId;
   private final int idOffset;
@@ -307,7 +308,7 @@ public void resetToMark()
       indexedIntsForCurrentRow = dimSelector.getRow();
     }
     // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid
-    if (index >= indexIntsForRow.size()) {
+    if (indexedIntsForCurrentRow == null || index >= indexedIntsForCurrentRow.size()) {
       index = 0;
     }
   }
diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
index 37a90d6ecb10..b8bb8bac9f33 100644
--- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
@@ -393,7 +393,7 @@ public Function columnFunction(String columnName)
       Assert.assertEquals(
           // marked at position 4, however first row has the same timestamp so we effectively restart cursor after
           // position 4
-          Arrays.asList(1, 2, 3, 4, 1, 2, 3, 4, 5, 6, null, 7, 8, 9, 10),
+          Arrays.asList(1L, 2L, 3L, 4L, 1L, 2L, 3L, 4L, 5L, 6L, null, 7L, 8L, 9L, 10L),
           rows
       );
     }

From f41a80ac39f0a77eb1b5e607d15a02f1449f7be4 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Sat, 10 Aug 2024 11:53:16 -0700
Subject: [PATCH 59/74] style

---
 .../java/org/apache/druid/segment/UnnestStorageAdapterTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
index b8bb8bac9f33..8d475476f19b 100644
--- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java
@@ -345,7 +345,7 @@ public ToLongFunction timestampFunction()
               @Override
               public Function columnFunction(String columnName)
               {
-                if (columnName.equals("a")) {
+                if ("a".equals(columnName)) {
                   return objects -> objects[1];
                 }
                 return null;

From 444c5b1ee7bcb7e71b570f61422e66626f8f16ff Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Mon, 12 Aug 2024 16:15:40 -0700
Subject: [PATCH 60/74] handle virtual gran name conflicts

---
 .../common/granularity/Granularities.java     | 26 ++++++++-
 .../druid/query/groupby/GroupingEngine.java   |  2 +-
 .../timeseries/TimeseriesQueryEngine.java     |  2 +-
 .../druid/query/topn/TopNQueryEngine.java     |  2 +-
 .../granularity/QueryGranularityTest.java     | 30 +++++-----
 .../druid/query/groupby/GroupByQueryTest.java | 55 ++++++++++++++++++-
 .../query/timeseries/TimeseriesQueryTest.java |  2 +-
 .../druid/query/topn/TopNQueryTest.java       |  2 +-
 8 files changed, 98 insertions(+), 23 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
index 084c87b33830..82819c85d5f8 100644
--- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
+++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
@@ -19,7 +19,9 @@
 
 package org.apache.druid.java.util.common.granularity;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Query;
 import org.apache.druid.query.expression.TimestampFloorExprMacro;
 import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.ColumnType;
@@ -27,6 +29,7 @@
 import org.joda.time.chrono.ISOChronology;
 
 import javax.annotation.Nullable;
+import java.util.Set;
 
 /**
  * This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
@@ -58,6 +61,24 @@ public static Granularity nullToAll(Granularity granularity)
     return granularity == null ? Granularities.ALL : granularity;
   }
 
+  @Nullable
+  public static ExpressionVirtualColumn toVirtualColumn(Query query)
+  {
+    ImmutableSet.Builder builder = ImmutableSet.builder();
+    final Set requiredColumns = query.getRequiredColumns();
+    if (requiredColumns != null) {
+      builder.addAll(requiredColumns);
+    }
+    builder.addAll(query.getVirtualColumns().getColumnNames());
+    final Set columnNamesForConflictResolution = builder.build();
+    String virtualColumnName = GRANULARITY_VIRTUAL_COLUMN_NAME;
+    int ctr = 0;
+    while (columnNamesForConflictResolution.contains(virtualColumnName)) {
+      virtualColumnName = virtualColumnName + ctr++;
+    }
+    return toVirtualColumn(query.getGranularity(), virtualColumnName);
+  }
+
   /**
    * Translates a {@link Granularity} to a {@link ExpressionVirtualColumn} on {@link ColumnHolder#TIME_COLUMN_NAME} of
    * the equivalent grouping column. If granularity is {@link #ALL}, this method returns null since we are not grouping
@@ -68,7 +89,7 @@ public static Granularity nullToAll(Granularity granularity)
    * {@link ColumnHolder#TIME_COLUMN_NAME} directly.
    */
   @Nullable
-  public static ExpressionVirtualColumn toVirtualColumn(Granularity granularity)
+  public static ExpressionVirtualColumn toVirtualColumn(Granularity granularity, String virtualColumnName)
   {
     if (ALL.equals(granularity)) {
       return null;
@@ -84,8 +105,9 @@ public static ExpressionVirtualColumn toVirtualColumn(Granularity granularity)
         expression = TimestampFloorExprMacro.forQueryGranularity(period.getPeriod());
       }
     }
+
     return new ExpressionVirtualColumn(
-        GRANULARITY_VIRTUAL_COLUMN_NAME,
+        virtualColumnName,
         expression,
         ColumnType.LONG,
         ExprMacroTable.granularity()
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
index e0e25fef7677..6a6b935dd86c 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
@@ -853,7 +853,7 @@ public static CursorBuildSpec makeCursorBuildSpec(GroupByQuery query, @Nullable
     // and vector cursors if any pre-aggregated data at the matching granularity is available
     // eventually this could probably be reworked to be used by the granularizer instead of the existing method
     // of creating a selector on the time column
-    final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query.getGranularity());
+    final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query);
     VirtualColumns virtualColumns;
     List groupingColumns;
     if (granularityVirtual == null) {
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
index b925deb54ba4..94e1d6f839b9 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
@@ -324,7 +324,7 @@ public static CursorBuildSpec makeCursorBuildSpec(TimeseriesQuery query, @Nullab
     // and vector cursors if any pre-aggregated data at the matching granularity is available
     // eventually this could probably be reworked to be used by the granularizer instead of the existing method
     // of creating a selector on the time column
-    final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query.getGranularity());
+    final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query);
     VirtualColumns virtualColumns;
     List groupingColumns;
     if (granularityVirtual == null) {
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
index b660cb1d5ebe..bbbafd579c73 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
@@ -217,7 +217,7 @@ public static CursorBuildSpec makeCursorBuildSpec(TopNQuery query, @Nullable Que
     // and vector cursors if any pre-aggregated data at the matching granularity is available
     // eventually this could probably be reworked to be used by the granularizer instead of the existing method
     // of creating a selector on the time column
-    final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query.getGranularity());
+    final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query);
     VirtualColumns virtualColumns;
     List groupingColumns;
     if (granularityVirtual == null) {
diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
index 16fa189e1893..d73c50533ae0 100644
--- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
+++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
@@ -1036,23 +1036,23 @@ public void testToVirtualColumn()
         origin
     );
 
-    ExpressionVirtualColumn column = Granularities.toVirtualColumn(hour);
-    Assert.assertEquals("timestamp_floor(__time,'PT1H')", column.getExpression());
-    column = Granularities.toVirtualColumn(hourWithOrigin);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
-    column = Granularities.toVirtualColumn(hourWithTz);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
-    column = Granularities.toVirtualColumn(duration);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
-    column = Granularities.toVirtualColumn(Granularities.NONE);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
-    column = Granularities.toVirtualColumn(Granularities.ALL);
+    ExpressionVirtualColumn column = Granularities.toVirtualColumn(hour, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals("timestamp_floor(__time,'PT1H')", Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    column = Granularities.toVirtualColumn(hourWithOrigin, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    column = Granularities.toVirtualColumn(hourWithTz, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    column = Granularities.toVirtualColumn(duration, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    column = Granularities.toVirtualColumn(Granularities.NONE, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    column = Granularities.toVirtualColumn(Granularities.ALL, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
     Assert.assertNull(column);
-    column = Granularities.toVirtualColumn(Granularities.HOUR);
-    Assert.assertEquals("timestamp_floor(__time,'PT1H')", column.getExpression());
-    column = Granularities.toVirtualColumn(Granularities.MINUTE);
+    column = Granularities.toVirtualColumn(Granularities.HOUR, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals("timestamp_floor(__time,'PT1H')", Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    column = Granularities.toVirtualColumn(Granularities.MINUTE, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
     Assert.assertEquals("timestamp_floor(__time,'PT1M')", column.getExpression());
-    column = Granularities.toVirtualColumn(Granularities.FIFTEEN_MINUTE);
+    column = Granularities.toVirtualColumn(Granularities.FIFTEEN_MINUTE, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
     Assert.assertEquals("timestamp_floor(__time,'PT15M')", column.getExpression());
   }
 
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java
index 59865f47d634..07205872089d 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java
@@ -252,7 +252,60 @@ public void testAsCursorBuildSpecDayGranularity()
     Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
     Assert.assertEquals(
         VirtualColumns.create(
-            Granularities.toVirtualColumn(Granularities.DAY),
+            Granularities.toVirtualColumn(query),
+            virtualColumns.getVirtualColumns()[0]
+        ),
+        buildSpec.getVirtualColumns()
+    );
+  }
+
+  @Test
+  public void testAsCursorBuildSpecDayGranularityNameConflict()
+  {
+    // make conflicting column name to force artificial granularity column to have a different name
+    final VirtualColumns virtualColumns = VirtualColumns.create(
+        new ExpressionVirtualColumn(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME, "concat(placement, 'foo')", ColumnType.STRING, ExprMacroTable.nil())
+    );
+    final LongSumAggregatorFactory longSum = new LongSumAggregatorFactory("idx", "index");
+    GroupByQuery query = GroupByQuery
+        .builder()
+        .setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
+        .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
+        .setDimensions(
+            new DefaultDimensionSpec(QueryRunnerTestHelper.QUALITY_DIMENSION, "alias"),
+            new DefaultDimensionSpec(
+                QueryRunnerTestHelper.MARKET_DIMENSION,
+                "market",
+                ColumnType.STRING_ARRAY
+            ),
+            new DefaultDimensionSpec("v0", "v0", ColumnType.STRING)
+        )
+        .setVirtualColumns(virtualColumns)
+        .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, longSum)
+        .setGranularity(Granularities.DAY)
+        .setPostAggregatorSpecs(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
+        .setLimitSpec(
+            new DefaultLimitSpec(
+                ImmutableList.of(new OrderByColumnSpec(
+                    "alias",
+                    OrderByColumnSpec.Direction.ASCENDING,
+                    StringComparators.LEXICOGRAPHIC
+                )),
+                100
+            )
+        )
+        .build();
+
+    final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null);
+    Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
+    Assert.assertEquals(
+        ImmutableList.of(Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME + "0", "quality", "market", "v0"),
+        buildSpec.getGroupingColumns()
+    );
+    Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
+    Assert.assertEquals(
+        VirtualColumns.create(
+            Granularities.toVirtualColumn(query.getGranularity(), Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME + "0"),
             virtualColumns.getVirtualColumns()[0]
         ),
         buildSpec.getVirtualColumns()
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
index 2174841006ef..40e2f6ec9499 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
@@ -196,7 +196,7 @@ public void testAsCursorBuildSpecDayGranularity()
     );
     Assert.assertEquals(
         VirtualColumns.create(
-            Granularities.toVirtualColumn(Granularities.DAY),
+            Granularities.toVirtualColumn(query),
             virtualColumns.getVirtualColumns()[0]
         ),
         buildSpec.getVirtualColumns()
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
index d9493278ad26..39371a8747a2 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
@@ -325,7 +325,7 @@ public void testAsCursorBuildSpecDayGranularity()
     Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
     Assert.assertEquals(
         VirtualColumns.create(
-            Granularities.toVirtualColumn(Granularities.DAY),
+            Granularities.toVirtualColumn(query),
             virtualColumns.getVirtualColumns()[0]
         ),
         buildSpec.getVirtualColumns()

From 212af82bef081213d257f9136bdcd78207704e3d Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Mon, 12 Aug 2024 17:35:41 -0700
Subject: [PATCH 61/74] oops, fix test

---
 .../druid/granularity/QueryGranularityTest.java      | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
index d73c50533ae0..41834d448fba 100644
--- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
+++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
@@ -1037,19 +1037,19 @@ public void testToVirtualColumn()
     );
 
     ExpressionVirtualColumn column = Granularities.toVirtualColumn(hour, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
-    Assert.assertEquals("timestamp_floor(__time,'PT1H')", Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals("timestamp_floor(__time,'PT1H')", column.getExpression());
     column = Granularities.toVirtualColumn(hourWithOrigin, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
     column = Granularities.toVirtualColumn(hourWithTz, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
     column = Granularities.toVirtualColumn(duration, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
     column = Granularities.toVirtualColumn(Granularities.NONE, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
-    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, column.getExpression());
     column = Granularities.toVirtualColumn(Granularities.ALL, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
     Assert.assertNull(column);
     column = Granularities.toVirtualColumn(Granularities.HOUR, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
-    Assert.assertEquals("timestamp_floor(__time,'PT1H')", Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
+    Assert.assertEquals("timestamp_floor(__time,'PT1H')", column.getExpression());
     column = Granularities.toVirtualColumn(Granularities.MINUTE, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);
     Assert.assertEquals("timestamp_floor(__time,'PT1M')", column.getExpression());
     column = Granularities.toVirtualColumn(Granularities.FIFTEEN_MINUTE, Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME);

From 3a785589f6cd54ce96c0213278c63888c06b2db2 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Tue, 13 Aug 2024 19:45:23 -0700
Subject: [PATCH 62/74] topN with better reset to not be wasteful, add
 getOrdering to CursorHolder

---
 .../apache/druid/frame/read/FrameReader.java  |   7 +-
 .../druid/frame/segment/FrameCursor.java      |   7 +-
 ... => ColumnarFrameCursorHolderFactory.java} |  29 ++++-
 .../segment/columnar/FrameQueryableIndex.java |   2 +-
 ....java => RowFrameCursorHolderFactory.java} |  28 ++++-
 .../druid/query/scan/ScanQueryEngine.java     |  15 +++
 .../TimeBoundaryQueryRunnerFactory.java       |   9 +-
 .../timeseries/TimeseriesQueryEngine.java     |   3 +
 .../AggregateTopNMetricFirstAlgorithm.java    |   2 +
 .../druid/query/topn/BaseTopNAlgorithm.java   |   5 +-
 .../StringTopNColumnAggregatesProcessor.java  |   3 +-
 .../apache/druid/segment/CursorBuildSpec.java |  18 +---
 .../apache/druid/segment/CursorHolder.java    |  12 +++
 .../org/apache/druid/segment/Cursors.java     | 100 ++++++++++++++++++
 .../segment/QueryableIndexCursorHolder.java   |  12 ++-
 .../druid/segment/RowBasedStorageAdapter.java |  26 +++--
 .../apache/druid/segment/StorageAdapter.java  |  25 ++++-
 .../druid/segment/UnnestStorageAdapter.java   |  17 ++-
 .../IncrementalIndexCursorHolder.java         |  21 +++-
 .../join/HashJoinSegmentStorageAdapter.java   |  33 +++---
 .../druid/frame/TestArrayStorageAdapter.java  |   9 ++
 .../segment/FrameStorageAdapterTest.java      |  10 +-
 .../druid/frame/testutil/FrameTestUtil.java   |   7 ++
 .../druid/query/CursorGranularizerTest.java   |   9 +-
 .../druid/query/topn/TopNQueryRunnerTest.java |  49 +++++++++
 .../segment/RowBasedStorageAdapterTest.java   |  22 +---
 .../IncrementalIndexStorageAdapterTest.java   |  15 +--
 .../segment/join/PostJoinCursorTest.java      |   8 ++
 28 files changed, 391 insertions(+), 112 deletions(-)
 rename processing/src/main/java/org/apache/druid/frame/segment/columnar/{FrameCursorHolderFactory.java => ColumnarFrameCursorHolderFactory.java} (86%)
 rename processing/src/main/java/org/apache/druid/frame/segment/row/{FrameCursorHolderFactory.java => RowFrameCursorHolderFactory.java} (79%)
 create mode 100644 processing/src/main/java/org/apache/druid/segment/Cursors.java

diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java
index 9fa10f8d833f..ce169b72b1db 100644
--- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java
+++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java
@@ -29,7 +29,8 @@
 import org.apache.druid.frame.key.KeyColumn;
 import org.apache.druid.frame.read.columnar.FrameColumnReader;
 import org.apache.druid.frame.read.columnar.FrameColumnReaders;
-import org.apache.druid.frame.segment.columnar.FrameCursorHolderFactory;
+import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory;
+import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory;
 import org.apache.druid.frame.write.FrameWriterUtils;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.segment.CursorHolderFactory;
@@ -142,9 +143,9 @@ public CursorHolderFactory makeCursorHolderFactory(final Frame frame)
   {
     switch (frame.type()) {
       case COLUMNAR:
-        return new FrameCursorHolderFactory(frame, signature, columnReaders);
+        return new ColumnarFrameCursorHolderFactory(frame, signature, columnReaders);
       case ROW_BASED:
-        return new org.apache.druid.frame.segment.row.FrameCursorHolderFactory(frame, this, fieldReaders);
+        return new RowFrameCursorHolderFactory(frame, this, fieldReaders);
       default:
         throw DruidException.defensive("Unrecognized frame type [%s]", frame.type());
     }
diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java
index 32dcbcd2f67b..1c99dee30b31 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java
@@ -19,15 +19,16 @@
 
 package org.apache.druid.frame.segment;
 
-import org.apache.druid.frame.segment.columnar.FrameCursorHolderFactory;
+import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory;
+import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory;
 import org.apache.druid.query.BaseQuery;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.SimpleSettableOffset;
 
 /**
- * An implementation of {@link Cursor} used by {@link org.apache.druid.frame.segment.row.FrameCursorHolderFactory}
- * and {@link FrameCursorHolderFactory}.
+ * An implementation of {@link Cursor} used by {@link RowFrameCursorHolderFactory}
+ * and {@link ColumnarFrameCursorHolderFactory}.
  *
  * Adds the methods {@link #getCurrentRow()} and {@link #setCurrentRow(int)} so the cursor can be moved to
  * particular rows.
diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java
similarity index 86%
rename from processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorHolderFactory.java
rename to processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java
index 7c1b65ee87c8..7c97530e37f1 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorHolderFactory.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java
@@ -25,8 +25,10 @@
 import org.apache.druid.frame.segment.FrameCursor;
 import org.apache.druid.frame.segment.FrameCursorUtils;
 import org.apache.druid.frame.segment.FrameFilteredOffset;
+import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.vector.VectorValueMatcher;
 import org.apache.druid.segment.ColumnCache;
@@ -34,6 +36,7 @@
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.CursorHolderFactory;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.QueryableIndexColumnSelectorFactory;
 import org.apache.druid.segment.SimpleAscendingOffset;
 import org.apache.druid.segment.SimpleDescendingOffset;
@@ -55,15 +58,15 @@
  *
  * This class is only used for columnar frames. It is not used for row-based frames.
  *
- * @see org.apache.druid.frame.segment.row.FrameCursorHolderFactory the row-based version
+ * @see RowFrameCursorHolderFactory the row-based version
  */
-public class FrameCursorHolderFactory implements CursorHolderFactory
+public class ColumnarFrameCursorHolderFactory implements CursorHolderFactory
 {
   private final Frame frame;
   private final RowSignature signature;
   private final List columnReaders;
 
-  public FrameCursorHolderFactory(
+  public ColumnarFrameCursorHolderFactory(
       final Frame frame,
       final RowSignature signature,
       final List columnReaders
@@ -78,6 +81,16 @@ public FrameCursorHolderFactory(
   public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
     final Closer closer = Closer.create();
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    final List ordering;
+    final boolean descending;
+    if (Cursors.preferDescendingTimeOrdering(spec)) {
+      ordering = Cursors.descendingTimeOrder();
+      descending = true;
+    } else {
+      ordering = Cursors.ascendingTimeOrder();
+      descending = false;
+    }
     return new CursorHolder()
     {
       @Override
@@ -85,7 +98,7 @@ public boolean canVectorize()
       {
         return (spec.getFilter() == null || spec.getFilter().canVectorizeMatcher(signature))
                && spec.getVirtualColumns().canVectorize(signature)
-               && !CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering());
+               && !descending;
       }
 
       @Override
@@ -94,7 +107,7 @@ public Cursor asCursor()
         final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders);
         final ColumnCache columnCache = new ColumnCache(index, closer);
         final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval());
-        final boolean descendingTimeOrder = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering());
+        final boolean descendingTimeOrder = Cursors.preferDescendingTimeOrdering(spec);
         final SimpleSettableOffset baseOffset = descendingTimeOrder
                                                 ? new SimpleDescendingOffset(frame.numRows())
                                                 : new SimpleAscendingOffset(frame.numRows());
@@ -116,6 +129,12 @@ public Cursor asCursor()
         return new FrameCursor(offset, columnSelectorFactory);
       }
 
+      @Override
+      public List getOrdering()
+      {
+        return ordering;
+      }
+
       @Nullable
       @Override
       public VectorCursor asVectorCursor()
diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java
index 8b2b4bef2880..86bf7eb51263 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java
@@ -42,7 +42,7 @@
  * A {@link QueryableIndex} implementation based on a single columnar {@link Frame}. There is no internal caching
  * of columns here, so callers should generally wrap this in a {@link org.apache.druid.segment.ColumnCache}.
  *
- * This class exists so {@link FrameCursorHolderFactory} can reuse code meant for regular segment-backed
+ * This class exists so {@link ColumnarFrameCursorHolderFactory} can reuse code meant for regular segment-backed
  * {@link QueryableIndex}. Some methods are implemented by throwing {@link UnsupportedOperationException}, wherever
  * it is not expected that those methods are actually going to be needed.
  */
diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java
similarity index 79%
rename from processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorHolderFactory.java
rename to processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java
index 06e7d618b6cc..2d0d6567d5f1 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorHolderFactory.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java
@@ -26,12 +26,15 @@
 import org.apache.druid.frame.segment.FrameCursor;
 import org.apache.druid.frame.segment.FrameCursorUtils;
 import org.apache.druid.frame.segment.FrameFilteredOffset;
+import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.CursorHolderFactory;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.SimpleAscendingOffset;
 import org.apache.druid.segment.SimpleDescendingOffset;
 import org.apache.druid.segment.SimpleSettableOffset;
@@ -44,15 +47,15 @@
  *
  * This class is only used for row-based frames.
  *
- * @see org.apache.druid.frame.segment.columnar.FrameCursorHolderFactory the columnar version
+ * @see ColumnarFrameCursorHolderFactory the columnar version
  */
-public class FrameCursorHolderFactory implements CursorHolderFactory
+public class RowFrameCursorHolderFactory implements CursorHolderFactory
 {
   private final Frame frame;
   private final FrameReader frameReader;
   private final List fieldReaders;
 
-  public FrameCursorHolderFactory(
+  public RowFrameCursorHolderFactory(
       final Frame frame,
       final FrameReader frameReader,
       final List fieldReaders
@@ -66,6 +69,16 @@ public FrameCursorHolderFactory(
   @Override
   public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    final List ordering;
+    final boolean descending;
+    if (Cursors.preferDescendingTimeOrdering(spec)) {
+      ordering = Cursors.descendingTimeOrder();
+      descending = true;
+    } else {
+      ordering = Cursors.ascendingTimeOrder();
+      descending = false;
+    }
     return new CursorHolder()
     {
       @Nullable
@@ -74,7 +87,7 @@ public Cursor asCursor()
       {
         final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval());
 
-        final SimpleSettableOffset baseOffset = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering())
+        final SimpleSettableOffset baseOffset = descending
                                                 ? new SimpleDescendingOffset(frame.numRows())
                                                 : new SimpleAscendingOffset(frame.numRows());
 
@@ -102,6 +115,13 @@ public Cursor asCursor()
         // Currently, it assumes that closing the Sequence does nothing.
         return cursor;
       }
+
+      @Nullable
+      @Override
+      public List getOrdering()
+      {
+        return ordering;
+      }
     };
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
index c1cc76fc8925..ff70f722339e 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import org.apache.druid.error.DruidException;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.UOE;
@@ -38,6 +39,7 @@
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.VirtualColumn;
@@ -118,6 +120,19 @@ public Sequence process(
     responseContext.addRowScanCount(0);
     final long limit = calculateRemainingScanRowsLimit(query, responseContext);
     final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics));
+    if (Order.NONE != query.getTimeOrder()) {
+      final Order requiredOrder = query.getTimeOrder();
+      if (!(Cursors.isTimeOrdered(cursorHolder) && cursorHolder.getOrdering().get(0).getOrder() == requiredOrder)) {
+        String failureReason = StringUtils.format(
+            "Cursor must be ordered by [%s] with direction [%s] but was [%s] instead.",
+            ColumnHolder.TIME_COLUMN_NAME,
+            requiredOrder,
+            cursorHolder.getOrdering()
+        );
+        cursorHolder.close();
+        throw DruidException.defensive(failureReason);
+      }
+    }
     return new BaseSequence<>(
         new BaseSequence.IteratorMaker>()
         {
diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
index cbef078435ab..38b5ff6eb930 100644
--- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
@@ -27,7 +27,7 @@
 import org.apache.druid.java.util.common.guava.BaseSequence;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.ChainedExecutionQueryRunner;
-import org.apache.druid.query.OrderBy;
+import org.apache.druid.query.Order;
 import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryProcessingPool;
@@ -42,6 +42,7 @@
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.column.ColumnHolder;
@@ -51,7 +52,6 @@
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
-import java.util.Collections;
 import java.util.Iterator;
 
 /**
@@ -120,12 +120,13 @@ private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legac
     {
       final CursorBuildSpec.CursorBuildSpecBuilder bob = CursorBuildSpec.builder(makeCursorBuildSpec(legacyQuery));
       if (descending) {
-        bob.setPreferredOrdering(Collections.singletonList(OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)));
+        bob.setPreferredOrdering(Cursors.descendingTimeOrder());
       } else {
-        bob.setPreferredOrdering(Collections.singletonList(OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME)));
+        bob.setPreferredOrdering(Cursors.ascendingTimeOrder());
       }
 
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(bob.build())) {
+        Cursors.requireTimeOrdering(cursorHolder, descending ? Order.DESCENDING : Order.ASCENDING);
         final Cursor cursor = cursorHolder.asCursor();
         if (cursor == null) {
           return null;
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
index 94e1d6f839b9..04cf153048cb 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
@@ -33,6 +33,7 @@
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.CursorGranularizer;
+import org.apache.druid.query.Order;
 import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.QueryMetrics;
 import org.apache.druid.query.Result;
@@ -44,6 +45,7 @@
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.SegmentMissingException;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.VirtualColumn;
@@ -106,6 +108,7 @@ public Sequence> process(
 
 
     final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics));
+    Cursors.requireTimeOrdering(cursorHolder, query.isDescending() ? Order.DESCENDING : Order.ASCENDING);
     final Sequence> result;
 
     if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize(), cursorHolder::close)) {
diff --git a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
index 839091cb079d..104f47f99753 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
@@ -112,6 +112,8 @@ public void run(
     PooledTopNAlgorithm allMetricAlgo = new PooledTopNAlgorithm(storageAdapter, query, bufferPool);
     PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null;
     try {
+      // reset cursor to starting position (mark was set by the call to run)
+      params.getCursor().resetToMark();
       // Run topN for all metrics for top N dimension values
       allMetricsParam = allMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor(), params.getGranularizer());
       allMetricAlgo.run(
diff --git a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
index 16c6b498c996..904063edb89d 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
@@ -123,7 +123,9 @@ private void runWithCardinalityKnown(
       resetAggregators(aggregatesStore);
 
       numProcessed += numToProcess;
-      params.getCursor().resetToMark();
+      if (numProcessed < cardinality) {
+        params.getCursor().resetToMark();
+      }
     }
     if (queryMetrics != null) {
       queryMetrics.addProcessedRows(processedRows);
@@ -152,7 +154,6 @@ private void runWithCardinalityUnknown(
     long processedRows = scanAndAggregate(params, null, aggregatesStore);
     updateResults(params, null, aggregatesStore, resultBuilder);
     resetAggregators(aggregatesStore);
-    params.getCursor().resetToMark();
     if (queryMetrics != null) {
       queryMetrics.addProcessedRows(processedRows);
       queryMetrics.stopRecordingScanTime();
diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java
index 6993b6ae9ffb..d6f716f2c2b3 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java
@@ -122,7 +122,8 @@ public long scanAndAggregate(
     // we must know cardinality to use array based aggregation. in cases where the same dictionary ids map to different
     // values (1:* or *:*), results can be entirely incorrect since an aggregator for a different value might be
     // chosen from the array based on the re-used dictionary id
-    if (notUnknown && hasDictionary) {
+    // finally, 'run' passes in selector as null for unknown case, so to be safe check for null
+    if (notUnknown && hasDictionary && rowSelector != null) {
       return scanAndAggregateWithCardinalityKnown(query, cursor, granularizer, selector, rowSelector);
     } else {
       return scanAndAggregateWithCardinalityUnknown(query, cursor, granularizer, selector);
diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
index d8f680da016f..f6cbbdeffe8d 100644
--- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
+++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java
@@ -20,13 +20,11 @@
 package org.apache.druid.segment;
 
 import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.query.Order;
 import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.QueryContext;
 import org.apache.druid.query.QueryMetrics;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.filter.Filter;
-import org.apache.druid.segment.column.ColumnHolder;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
@@ -34,7 +32,7 @@
 
 public class CursorBuildSpec
 {
-  public static final CursorBuildSpec FULL_SCAN = CursorBuildSpec.builder().build();
+  public static final CursorBuildSpec FULL_SCAN = builder().build();
 
   public static CursorBuildSpecBuilder builder()
   {
@@ -168,20 +166,6 @@ public QueryMetrics getQueryMetrics()
     return queryMetrics;
   }
 
-  /**
-   * Check if the first {@link OrderBy} column of {@link CursorBuildSpec#getPreferredOrdering()} is
-   * {@link Order#DESCENDING}, which allow {@link Cursor} on time ordered data to advance in descending order if
-   * possible.
-   */
-  public static boolean preferDescendingTimeOrder(@Nullable List preferredOrdering)
-  {
-    if (preferredOrdering != null && !preferredOrdering.isEmpty()) {
-      final OrderBy orderBy = preferredOrdering.get(0);
-      return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()) && Order.DESCENDING == orderBy.getOrder();
-    }
-    return false;
-  }
-
   public static class CursorBuildSpecBuilder
   {
     @Nullable
diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java
index 85776c7214a3..d0384e265b34 100644
--- a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java
@@ -20,10 +20,13 @@
 package org.apache.druid.segment;
 
 import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.segment.vector.VectorCursor;
 
 import javax.annotation.Nullable;
 import java.io.Closeable;
+import java.util.Collections;
+import java.util.List;
 
 public interface CursorHolder extends Closeable
 {
@@ -50,6 +53,15 @@ default boolean canVectorize()
     return false;
   }
 
+  /**
+   * Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns
+   * null then the cursor has no defined ordering
+   */
+  default List getOrdering()
+  {
+    return Collections.emptyList();
+  }
+
   /**
    * Release any resources acquired by cursors.
    */
diff --git a/processing/src/main/java/org/apache/druid/segment/Cursors.java b/processing/src/main/java/org/apache/druid/segment/Cursors.java
new file mode 100644
index 000000000000..eb65d0b61d72
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/Cursors.java
@@ -0,0 +1,100 @@
+/*
+ * 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.segment;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.query.Order;
+import org.apache.druid.query.OrderBy;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import java.util.Collections;
+import java.util.List;
+
+public class Cursors
+{
+  private static final List TIME_ASCENDING_ORDER = Collections.singletonList(
+      OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME)
+  );
+
+  private static final List TIME_DESCENDING_ORDER = Collections.singletonList(
+      OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
+  );
+
+  /**
+   * Check if the first {@link OrderBy} column of {@link CursorBuildSpec#getPreferredOrdering()} is
+   * {@link Order#DESCENDING}, which allow {@link Cursor} on time ordered data to advance in descending order if
+   * possible.
+   */
+  public static boolean preferDescendingTimeOrdering(CursorBuildSpec buildSpec)
+  {
+    final List preferredOrdering = buildSpec.getPreferredOrdering();
+    if (preferredOrdering != null && !preferredOrdering.isEmpty()) {
+      final OrderBy orderBy = preferredOrdering.get(0);
+      return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()) && Order.DESCENDING == orderBy.getOrder();
+    }
+    return false;
+  }
+
+  /**
+   * Check if the first {@link OrderBy} of {@link CursorHolder#getOrdering()} is {@link ColumnHolder#TIME_COLUMN_NAME}
+   */
+  public static boolean isTimeOrdered(CursorHolder holder)
+  {
+    final List ordering = holder.getOrdering();
+    if (ordering.isEmpty()) {
+      return false;
+    }
+    final OrderBy orderBy = ordering.get(0);
+    return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName());
+  }
+
+  /**
+   * Require the first {@link OrderBy} of {@link CursorHolder#getOrdering()} is {@link ColumnHolder#TIME_COLUMN_NAME}.
+   * Throws {@link DruidException} if the order does
+   */
+  public static void requireTimeOrdering(CursorHolder holder, Order order)
+  {
+    DruidException.conditionalDefensive(
+        isTimeOrdered(holder) && holder.getOrdering().get(0).getOrder() == order,
+        "Cursor must be ordered by [%s] with direction [%s] but was [%s] instead.",
+        ColumnHolder.TIME_COLUMN_NAME,
+        order,
+        holder.getOrdering()
+    );
+  }
+
+  /**
+   * Get a {@link CursorHolder} {@link OrderBy} list that contains only a {@link ColumnHolder#TIME_COLUMN_NAME} as
+   * {@link Order#ASCENDING}, classic Druid segment order.
+   */
+  public static List ascendingTimeOrder()
+  {
+    return TIME_ASCENDING_ORDER;
+  }
+
+  /**
+   * Get a {@link CursorHolder} {@link OrderBy} list that contains only a {@link ColumnHolder#TIME_COLUMN_NAME} as
+   * {@link Order#DESCENDING}, classic Druid segment order in reverse.
+   */
+  public static List descendingTimeOrder()
+  {
+    return TIME_DESCENDING_ORDER;
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
index 487a50319286..9f636e05494a 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
@@ -29,6 +29,7 @@
 import org.apache.druid.query.BaseQuery;
 import org.apache.druid.query.BitmapResultFactory;
 import org.apache.druid.query.DefaultBitmapResultFactory;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryContext;
 import org.apache.druid.query.QueryMetrics;
@@ -56,6 +57,7 @@
 import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
@@ -72,6 +74,7 @@ public class QueryableIndexCursorHolder implements CursorHolder
   private final Filter filter;
   @Nullable
   private final QueryMetrics metrics;
+  private final List ordering;
   private final boolean descending;
   private final QueryContext queryContext;
   private final int vectorSize;
@@ -87,7 +90,14 @@ public QueryableIndexCursorHolder(
     this.virtualColumns = cursorBuildSpec.getVirtualColumns();
     this.aggregatorFactories = cursorBuildSpec.getAggregators();
     this.filter = cursorBuildSpec.getFilter();
-    this.descending = CursorBuildSpec.preferDescendingTimeOrder(cursorBuildSpec.getPreferredOrdering());
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    if (Cursors.preferDescendingTimeOrdering(cursorBuildSpec)) {
+      this.ordering = Collections.singletonList(OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME));
+      this.descending = true;
+    } else {
+      this.ordering = Collections.singletonList(OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME));
+      this.descending = false;
+    }
     this.queryContext = cursorBuildSpec.getQueryContext();
     this.vectorSize = cursorBuildSpec.getQueryContext().getVectorSize();
     this.metrics = cursorBuildSpec.getQueryMetrics();
diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
index 946ae9970b8f..f4330709a44b 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
@@ -28,6 +28,7 @@
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.guava.SimpleSequence;
 import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.data.Indexed;
@@ -168,6 +169,16 @@ public Metadata getMetadata()
   @Override
   public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    final List ordering;
+    final boolean descending;
+    if (Cursors.preferDescendingTimeOrdering(spec)) {
+      ordering = Cursors.descendingTimeOrder();
+      descending = true;
+    } else {
+      ordering = Cursors.ascendingTimeOrder();
+      descending = false;
+    }
     return new CursorHolder()
     {
       final Closer closer = Closer.create();
@@ -175,12 +186,8 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       @Override
       public Cursor asCursor()
       {
-        final boolean descendingTimeOrder = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering());
         final RowWalker rowWalker = closer.register(
-            new RowWalker<>(
-                 descendingTimeOrder ? reverse(rowSequence) : rowSequence,
-                rowAdapter
-            )
+            new RowWalker<>(descending ? reverse(rowSequence) : rowSequence, rowAdapter)
         );
         return new RowBasedCursor<>(
             rowWalker,
@@ -188,11 +195,18 @@ public Cursor asCursor()
             spec.getFilter(),
             spec.getInterval(),
             spec.getVirtualColumns(),
-            descendingTimeOrder,
+            descending,
             rowSignature
         );
       }
 
+      @Nullable
+      @Override
+      public List getOrdering()
+      {
+        return ordering;
+      }
+
       @Override
       public void close()
       {
diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
index 4c0635ee91d2..fba947f73d2a 100644
--- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
@@ -22,6 +22,7 @@
 import com.google.common.collect.Iterables;
 import org.apache.druid.guice.annotations.PublicApi;
 import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.data.Indexed;
@@ -30,6 +31,7 @@
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
+import java.util.List;
 import java.util.Optional;
 
 /**
@@ -47,6 +49,16 @@ public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHo
   @Override
   default CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    final List ordering;
+    final boolean descending;
+    if (Cursors.preferDescendingTimeOrdering(spec)) {
+      ordering = Cursors.descendingTimeOrder();
+      descending = true;
+    } else {
+      ordering = Cursors.ascendingTimeOrder();
+      descending = false;
+    }
     return new CursorHolder()
     {
       @Override
@@ -55,7 +67,7 @@ public boolean canVectorize()
         return StorageAdapter.this.canVectorize(
             spec.getFilter(),
             spec.getVirtualColumns(),
-            CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering())
+            Cursors.preferDescendingTimeOrdering(spec)
         );
       }
 
@@ -68,7 +80,7 @@ public Cursor asCursor()
                 spec.getInterval(),
                 spec.getVirtualColumns(),
                 Granularities.ALL,
-                CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()),
+                descending,
                 spec.getQueryMetrics()
             ).toList()
         );
@@ -81,12 +93,19 @@ public VectorCursor asVectorCursor()
             spec.getFilter(),
             spec.getInterval(),
             spec.getVirtualColumns(),
-            CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()),
+            descending,
             spec.getQueryContext().getVectorSize(),
             spec.getQueryMetrics()
         );
       }
 
+      @Nullable
+      @Override
+      public List getOrdering()
+      {
+        return ordering;
+      }
+
       @Override
       public void close()
       {
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
index e15b4d34b3c6..a9f215892d1b 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
@@ -20,10 +20,12 @@
 package org.apache.druid.segment;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.filter.BooleanFilter;
 import org.apache.druid.query.filter.DimFilter;
 import org.apache.druid.query.filter.EqualityFilter;
@@ -56,6 +58,7 @@
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Supplier;
 
 /**
  * This class serves as the Storage Adapter for the Unnest Segment and is responsible for creating the cursors
@@ -108,11 +111,14 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
     return new CursorHolder()
     {
       final Closer closer = Closer.create();
+      final Supplier cursorHolderSupplier = Suppliers.memoize(
+          () -> closer.register(baseAdapter.makeCursorHolder(unnestBuildSpec))
+      );
+
       @Override
       public Cursor asCursor()
       {
-        final CursorHolder cursorHolder = closer.register(baseAdapter.makeCursorHolder(unnestBuildSpec));
-        final Cursor cursor = cursorHolder.asCursor();
+        final Cursor cursor = cursorHolderSupplier.get().asCursor();
         if (cursor == null) {
           return null;
         }
@@ -144,6 +150,13 @@ public Cursor asCursor()
         );
       }
 
+      @Nullable
+      @Override
+      public List getOrdering()
+      {
+        return cursorHolderSupplier.get().getOrdering();
+      }
+
       @Override
       public void close()
       {
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
index 8b90b367ec4c..31b15c62733a 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
@@ -21,24 +21,29 @@
 
 import com.google.common.collect.Iterators;
 import org.apache.druid.query.BaseQuery;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.filter.ValueMatchers;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.util.Iterator;
+import java.util.List;
 
 public class IncrementalIndexCursorHolder implements CursorHolder
 {
   private final IncrementalIndexStorageAdapter storageAdapter;
   private final IncrementalIndex index;
   private final CursorBuildSpec spec;
+  private final List ordering;
+  private final boolean descending;
 
   public IncrementalIndexCursorHolder(
       IncrementalIndexStorageAdapter storageAdapter,
@@ -49,6 +54,14 @@ public IncrementalIndexCursorHolder(
     this.storageAdapter = storageAdapter;
     this.index = index;
     this.spec = spec;
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    if (Cursors.preferDescendingTimeOrdering(spec)) {
+      this.ordering = Cursors.descendingTimeOrder();
+      this.descending = true;
+    } else {
+      this.ordering = Cursors.ascendingTimeOrder();
+      this.descending = false;
+    }
   }
 
   @Override
@@ -67,12 +80,18 @@ public Cursor asCursor()
         storageAdapter,
         index,
         spec.getVirtualColumns(),
-        CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering()),
+        descending,
         spec.getFilter(),
         spec.getInterval()
     );
   }
 
+  @Override
+  public List getOrdering()
+  {
+    return ordering;
+  }
+
   static class IncrementalIndexCursor implements Cursor
   {
     private IncrementalIndexRowHolder currEntry;
diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
index d53478b9df31..186352b14b1c 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
@@ -24,10 +24,12 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.Metadata;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.VirtualColumns;
@@ -39,7 +41,6 @@
 import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
 import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
 import org.apache.druid.segment.join.filter.JoinFilterSplit;
-import org.apache.druid.segment.vector.VectorCursor;
 import org.apache.druid.utils.CloseableUtils;
 import org.joda.time.DateTime;
 import org.joda.time.Interval;
@@ -231,6 +232,17 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       return baseAdapter.makeCursorHolder(newSpec);
     }
 
+    // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary
+    final List ordering;
+    final boolean descending;
+    if (Cursors.preferDescendingTimeOrdering(spec)) {
+      ordering = Cursors.descendingTimeOrder();
+      descending = true;
+    } else {
+      ordering = Cursors.ascendingTimeOrder();
+      descending = false;
+    }
+
     return new CursorHolder()
     {
       final Closer joinablesCloser = Closer.create();
@@ -290,10 +302,9 @@ public Cursor asCursor()
         }
 
         Cursor retVal = baseCursor;
-        final boolean isDescendingTimeOrdering = CursorBuildSpec.preferDescendingTimeOrder(spec.getPreferredOrdering());
 
         for (JoinableClause clause : clauses) {
-          retVal = HashJoinEngine.makeJoinCursor(retVal, clause, isDescendingTimeOrdering, joinablesCloser);
+          retVal = HashJoinEngine.makeJoinCursor(retVal, clause, descending, joinablesCloser);
         }
 
         return PostJoinCursor.wrap(
@@ -303,23 +314,17 @@ public Cursor asCursor()
         );
       }
 
+      @Nullable
       @Override
-      public void close()
-      {
-        CloseableUtils.closeAndWrapExceptions(joinablesCloser);
-      }
-
-      @Override
-      public boolean canVectorize()
+      public List getOrdering()
       {
-        return CursorHolder.super.canVectorize();
+        return ordering;
       }
 
-      @Nullable
       @Override
-      public VectorCursor asVectorCursor()
+      public void close()
       {
-        return CursorHolder.super.asVectorCursor();
+        CloseableUtils.closeAndWrapExceptions(joinablesCloser);
       }
     };
   }
diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
index 6a4e1e616833..f53793d692b1 100644
--- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
+++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
@@ -20,6 +20,7 @@
 package org.apache.druid.frame;
 
 import com.google.common.collect.Iterables;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
@@ -40,6 +41,7 @@
 import org.apache.druid.segment.data.IndexedInts;
 
 import javax.annotation.Nullable;
+import java.util.List;
 
 /**
  * Storage adapter around {@link QueryableIndex} that transforms all multi-value strings columns into string arrays.
@@ -64,6 +66,13 @@ public Cursor asCursor()
         return new DecoratedCursor(delegate.asCursor());
       }
 
+      @Nullable
+      @Override
+      public List getOrdering()
+      {
+        return delegate.getOrdering();
+      }
+
       @Override
       public void close()
       {
diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
index 3ae83e8c895b..b9abf2f81b14 100644
--- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java
@@ -28,7 +28,6 @@
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.math.expr.ExprMacroTable;
-import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.QueryContext;
 import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.extraction.TimeFormatExtractionFn;
@@ -39,6 +38,7 @@
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.StorageAdapter;
@@ -271,13 +271,7 @@ public CursorTests(
                                       .setFilter(this.filter)
                                       .setInterval(this.interval)
                                       .setVirtualColumns(this.virtualColumns)
-                                      .setPreferredOrdering(
-                                          descending ?
-                                          Collections.singletonList(
-                                              OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                          ) :
-                                          null
-                                      )
+                                      .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : null)
                                       .setQueryContext(queryContext)
                                       .build();
     }
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 d3a2ce42832a..e1981f3618b0 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
@@ -35,6 +35,7 @@
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.segment.ColumnProcessors;
 import org.apache.druid.segment.ColumnSelectorFactory;
@@ -311,6 +312,12 @@ public Cursor asCursor()
           return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn);
         }
 
+        @Override
+        public List getOrdering()
+        {
+          return cursorHolder.getOrdering();
+        }
+
         @Override
         public void close()
         {
diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
index d8b2539b0cca..8aeaf6260705 100644
--- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java
@@ -31,10 +31,10 @@
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.IndexBuilder;
 import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.StorageAdapter;
-import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
@@ -47,7 +47,6 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 public class CursorGranularizerTest extends InitializedNullHandlingTest
@@ -205,11 +204,7 @@ public void testGranularizeFullScan()
   public void testGranularizeFullScanDescending()
   {
     final CursorBuildSpec descending = CursorBuildSpec.builder()
-                                                      .setPreferredOrdering(
-                                                          Collections.singletonList(
-                                                              OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                                          )
-                                                      )
+                                                      .setPreferredOrdering(Cursors.descendingTimeOrder())
                                                       .build();
     try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) {
       final Cursor cursor = cursorHolder.asCursor();
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
index 79adf89fb86b..98355777fe2c 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
@@ -6193,6 +6193,55 @@ public void testExpressionAggregatorComplex()
     assertExpectedResults(expectedResults, query);
   }
 
+  @Test
+  public void testTopNAggregateTopnMetricFirst()
+  {
+    TopNQuery query = new TopNQueryBuilder()
+            .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
+            .granularity(QueryRunnerTestHelper.ALL_GRAN)
+            .dimension(QueryRunnerTestHelper.MARKET_DIMENSION)
+            .metric(QueryRunnerTestHelper.INDEX_METRIC)
+            .threshold(4)
+            .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD)
+            .aggregators(commonAggregators)
+            .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT)
+            .context(ImmutableMap.of("doAggregateTopNMetricFirst", true))
+            .build();
+
+
+    List> expectedResults = Collections.singletonList(
+            new Result<>(
+                    DateTimes.of("2011-04-01T00:00:00.000Z"),
+                    TopNResultValue.create(
+                            Arrays.>asList(
+                                    ImmutableMap.of(
+                                            QueryRunnerTestHelper.MARKET_DIMENSION, "total_market",
+                                            "rows", 4L,
+                                            "index", 5351.814783D,
+                                            "addRowsIndexConstant", 5356.814783D,
+                                            "uniques", QueryRunnerTestHelper.UNIQUES_2
+                                    ),
+                                    ImmutableMap.of(
+                                            QueryRunnerTestHelper.MARKET_DIMENSION, "upfront",
+                                            "rows", 4L,
+                                            "index", 4875.669692D,
+                                            "addRowsIndexConstant", 4880.669692D,
+                                            "uniques", QueryRunnerTestHelper.UNIQUES_2
+                                    ),
+                                    ImmutableMap.of(
+                                            QueryRunnerTestHelper.MARKET_DIMENSION, "spot",
+                                            "rows", 18L,
+                                            "index", 2231.876812D,
+                                            "addRowsIndexConstant", 2250.876812D,
+                                            "uniques", QueryRunnerTestHelper.UNIQUES_9
+                                    )
+                            )
+                    )
+            )
+    );
+    assertExpectedResults(expectedResults, query);
+  }
+
   private static Map makeRowWithNulls(
       String dimName,
       @Nullable Object dimValue,
diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
index 2eec513d949d..26e776aa74bc 100644
--- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
@@ -34,12 +34,10 @@
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.query.CursorGranularizer;
-import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.filter.TypedInFilter;
 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;
 import org.apache.druid.segment.column.ValueType;
@@ -602,11 +600,7 @@ public void test_makeCursor_descending()
     final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2);
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setPreferredOrdering(
-                                                         Collections.singletonList(
-                                                             OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                                         )
-                                                     )
+                                                     .setPreferredOrdering(Cursors.descendingTimeOrder())
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -720,11 +714,7 @@ public void test_makeCursor_hourGranularityWithIntervalDescending()
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
                                                      .setInterval(Intervals.of("1970-01-01T01/PT2H"))
-                                                     .setPreferredOrdering(
-                                                         Collections.singletonList(
-                                                             OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                                         )
-                                                     )
+                                                     .setPreferredOrdering(Cursors.descendingTimeOrder())
                                                      .build();
 
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
@@ -902,11 +892,7 @@ public void test_makeCursor_mark_resets_to_different_row_descending()
     final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 0, 1, 2, 3, 4, 5, 6, 7);
 
     final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
-                                                     .setPreferredOrdering(
-                                                         Collections.singletonList(
-                                                             OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                                         )
-                                                     )
+                                                     .setPreferredOrdering(Cursors.descendingTimeOrder())
                                                      .build();
     try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
       final Cursor cursor = cursorHolder.asCursor();
@@ -1089,7 +1075,7 @@ private static List> walkCursorGranularized(
         cursor,
         granularity,
         buildSpec.getInterval(),
-        CursorBuildSpec.preferDescendingTimeOrder(buildSpec.getPreferredOrdering())
+        Cursors.preferDescendingTimeOrdering(buildSpec)
     );
 
     final List> suppliers = new ArrayList<>();
diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
index b2755687c4e2..4df2e2c0407e 100644
--- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java
@@ -35,7 +35,6 @@
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.js.JavaScriptConfig;
 import org.apache.druid.query.DruidProcessingConfig;
-import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.Result;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.JavaScriptAggregatorFactory;
@@ -63,9 +62,9 @@
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.CursorBuildSpec;
 import org.apache.druid.segment.CursorHolder;
+import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
-import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.data.IndexedInts;
 import org.apache.druid.segment.filter.Filters;
 import org.apache.druid.segment.filter.OrFilter;
@@ -287,11 +286,7 @@ public void testResetSanity() throws IOException
                                                        .setFilter(new SelectorFilter("sally", "bo"))
                                                        .setInterval(interval)
                                                        .setPreferredOrdering(
-                                                           descending ?
-                                                           Collections.singletonList(
-                                                               OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                                           ) :
-                                                           null
+                                                           descending ? Cursors.descendingTimeOrder() : null
                                                        )
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
@@ -374,11 +369,7 @@ public void testMarkResetMarkSanity() throws IOException
                                                        )
                                                        .setInterval(interval)
                                                        .setPreferredOrdering(
-                                                           descending ?
-                                                           Collections.singletonList(
-                                                               OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)
-                                                           ) :
-                                                           null
+                                                           descending ? Cursors.descendingTimeOrder() : null
                                                        )
                                                        .build();
       try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) {
diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
index 80e7c9c87322..1435a7f27378 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
@@ -20,6 +20,7 @@
 package org.apache.druid.segment.join;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.druid.query.OrderBy;
 import org.apache.druid.query.QueryInterruptedException;
 import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
@@ -81,6 +82,13 @@ public Cursor asCursor()
             return new CursorNoAdvance(delegate.asCursor(), countDownLatch);
           }
 
+          @Nullable
+          @Override
+          public List getOrdering()
+          {
+            return delegate.getOrdering();
+          }
+
           @Override
           public void close()
           {

From a04c17e28884a3f9dd12d958d26f0b366b5d7ab1 Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Tue, 13 Aug 2024 21:10:18 -0700
Subject: [PATCH 63/74] oops

---
 .../apache/druid/segment/QueryableIndexCursorHolder.java    | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
index 9f636e05494a..2fbf0d63f44f 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java
@@ -284,6 +284,12 @@ public VectorCursor asVectorCursor()
     }
   }
 
+  @Override
+  public List getOrdering()
+  {
+    return ordering;
+  }
+
   @Override
   public void close()
   {

From 242c91874083512cbbbee57f0cd20e5e5403fcbf Mon Sep 17 00:00:00 2001
From: Clint Wylie 
Date: Wed, 14 Aug 2024 01:32:08 -0700
Subject: [PATCH 64/74] remove Cursor.mark, Cursor.resetToMark, force TopN
 algorithm to use heap unless can be computed in a single pass

---
 .../druid/frame/segment/FrameCursor.java      |  12 -
 .../AggregateTopNMetricFirstAlgorithm.java    |   2 +-
 .../druid/query/topn/BaseTopNAlgorithm.java   |   2 +-
 .../druid/query/topn/TopNQueryEngine.java     |  47 +++-
 .../java/org/apache/druid/segment/Cursor.java |  42 ---
 .../segment/QueryableIndexCursorHolder.java   |  12 -
 .../apache/druid/segment/RowBasedCursor.java  |  16 --
 .../UnnestColumnValueSelectorCursor.java      |  22 --
 .../druid/segment/UnnestDimensionCursor.java  |  22 --
 .../IncrementalIndexCursorHolder.java         |  44 ---
 .../druid/segment/join/HashJoinEngine.java    |  19 --
 .../druid/segment/join/PostJoinCursor.java    |  13 -
 .../druid/frame/TestArrayStorageAdapter.java  |  12 -
 .../testutil/RowNumberUpdatingCursor.java     |  14 -
 .../druid/query/topn/TopNQueryRunnerTest.java | 110 ++++++++
 .../org/apache/druid/segment/ListCursor.java  |  12 -
 .../segment/RowBasedStorageAdapterTest.java   | 175 +-----------
 .../segment/UnnestStorageAdapterTest.java     |  38 +--
 .../IncrementalIndexStorageAdapterTest.java   |  93 -------
 .../HashJoinSegmentStorageAdapterTest.java    | 259 +-----------------
 .../druid/segment/join/JoinTestHelper.java    | 140 ----------
 .../segment/join/PostJoinCursorTest.java      |  12 -
 .../selector/TestColumnValueSelector.java     |  16 --
 23 files changed, 161 insertions(+), 973 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java
index 1c99dee30b31..83bd1e673d98 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java
@@ -79,18 +79,6 @@ public boolean isDoneOrInterrupted()
     return isDone() || Thread.currentThread().isInterrupted();
   }
 
-  @Override
-  public void mark()
-  {
-    markOffset = offset.getOffset();
-  }
-
-  @Override
-  public void resetToMark()
-  {
-    offset.setCurrentOffset(markOffset);
-  }
-
   @Override
   public void reset()
   {
diff --git a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
index 104f47f99753..9c71a3842292 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
@@ -113,7 +113,7 @@ public void run(
     PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null;
     try {
       // reset cursor to starting position (mark was set by the call to run)
-      params.getCursor().resetToMark();
+      params.getCursor().reset();
       // Run topN for all metrics for top N dimension values
       allMetricsParam = allMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor(), params.getGranularizer());
       allMetricAlgo.run(
diff --git a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
index 904063edb89d..c8bcbc1a0944 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
@@ -124,7 +124,7 @@ private void runWithCardinalityKnown(
 
       numProcessed += numToProcess;
       if (numProcessed < cardinality) {
-        params.getCursor().resetToMark();
+        params.getCursor().reset();
       }
     }
     if (queryMetrics != null) {
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
index bbbafd579c73..af98381fe733 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import org.apache.druid.collections.NonBlockingPool;
+import org.apache.druid.collections.ResourceHolder;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
@@ -106,7 +107,6 @@ public Sequence> query(
         Sequences.simple(granularizer.getBucketIterable())
                  .map(bucketInterval -> {
                    granularizer.advanceToBucket(bucketInterval);
-                   cursor.mark();
                    return mapFn.apply(cursor, granularizer, queryMetrics);
                  }),
                  Predicates.notNull()
@@ -141,13 +141,13 @@ private TopNMapFn getMapFn(
 
 
     final TopNAlgorithm topNAlgorithm;
-    if (canUsePooledAlgorithm(selector, query, columnCapabilities)) {
+    if (canUsePooledAlgorithm(selector, query, columnCapabilities, bufferPool, cardinality, numBytesPerRecord)) {
       // pool based algorithm selection, if we can
       if (selector.isAggregateAllMetrics()) {
         // if sorted by dimension we should aggregate all metrics in a single pass, use the regular pooled algorithm for
         // this
         topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool);
-      } else if (selector.isAggregateTopNMetricFirst() || query.context().getBoolean("doAggregateTopNMetricFirst", false)) {
+      } else if (shouldUseAggregateMetricFirstAlgorithm(query, selector)) {
         // for high cardinality dimensions with larger result sets we aggregate with only the ordering aggregation to
         // compute the first 'n' values, and then for the rest of the metrics but for only the 'n' values
         topNAlgorithm = new AggregateTopNMetricFirstAlgorithm(adapter, query, bufferPool);
@@ -182,7 +182,7 @@ private TopNMapFn getMapFn(
    * algorithm) are optimized off-heap algorithms for aggregating dictionary encoded string columns. These algorithms
    * rely on dictionary ids being unique so to aggregate on the dictionary ids directly and defer
    * {@link org.apache.druid.segment.DimensionSelector#lookupName(int)} until as late as possible in query processing.
-   *
+   * 

* When these conditions are not true, we have an on-heap fall-back algorithm, the {@link HeapBasedTopNAlgorithm} * (and {@link TimeExtractionTopNAlgorithm} for a specialized form for long columns) which aggregates on values of * selectors. @@ -190,7 +190,10 @@ private TopNMapFn getMapFn( private static boolean canUsePooledAlgorithm( final TopNAlgorithmSelector selector, final TopNQuery query, - final ColumnCapabilities capabilities + final ColumnCapabilities capabilities, + final NonBlockingPool bufferPool, + final int cardinality, + final int numBytesPerRecord ) { if (selector.isHasExtractionFn()) { @@ -202,13 +205,39 @@ private static boolean canUsePooledAlgorithm( // non-string output cannot use the pooled algorith, even if the underlying selector supports it return false; } - if (Types.is(capabilities, ValueType.STRING)) { - // string columns must use the on heap algorithm unless they have the following capabilites - return capabilities.isDictionaryEncoded().isTrue() && capabilities.areDictionaryValuesUnique().isTrue(); - } else { + if (!Types.is(capabilities, ValueType.STRING)) { // non-strings are not eligible to use the pooled algorithm, and should use a heap algorithm return false; } + + // string columns must use the on heap algorithm unless they have the following capabilites + if (!capabilities.isDictionaryEncoded().isTrue() || !capabilities.areDictionaryValuesUnique().isTrue()) { + return false; + } + if (Granularities.ALL.equals(query.getGranularity())) { + // all other requirements have been satisfied, ALL granularity can always use the pooled algorithms + return true; + } + // if not using ALL granularity, we can still potentially use the pooled algorithm if we are certain it doesn't + // need to make multiple passes (e.g. reset the cursor) + try (final ResourceHolder resultsBufHolder = bufferPool.take()) { + final ByteBuffer resultsBuf = resultsBufHolder.get(); + resultsBuf.clear(); + + final int numBytesToWorkWith = resultsBuf.remaining(); + final int numValuesPerPass = numBytesPerRecord > 0 ? numBytesToWorkWith / numBytesPerRecord : cardinality; + + return numValuesPerPass <= cardinality; + } + } + + private static boolean shouldUseAggregateMetricFirstAlgorithm(TopNQuery query, TopNAlgorithmSelector selector) + { + // must be using ALL granularity since it makes multiple passes and must reset the cursor + if (Granularities.ALL.equals(query.getGranularity())) { + return selector.isAggregateTopNMetricFirst() || query.context().getBoolean("doAggregateTopNMetricFirst", false); + } + return false; } public static CursorBuildSpec makeCursorBuildSpec(TopNQuery query, @Nullable QueryMetrics queryMetrics) diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 352a0874164c..0a7ae22bb972 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -86,48 +86,6 @@ public interface Cursor */ boolean isDoneOrInterrupted(); - /** - * Mark a position on the cursor at the current row, which can recalled with {@link #resetToMark()}. This position is - * only guaranteed to have the same timestamp as the current row when this method is called. - *

- * In most cursors, this is able to mark and return to an exact row, but currently the only caller of this method does - * not require this contract ({@link org.apache.druid.query.topn.TopNQueryEngine}) rather it only requires that the - * first timestamp of the marked position can be returned to, since TopN uses this to make multiple passes over a - * cursor within a granularity bucket interval (and so this method only really needs to work correctly if the data is - * time ordered). - *

- * Additionally, not all built-in implementations can be guaranteed to be able to mark an exact row due to potential - * underlying data mutability: - * - {@link RowBasedCursor} being potentially backed by mutable data, such as lookups when queried as segments - * - the cursor provided by {@link org.apache.druid.segment.join.HashJoinEngine} due to - * {@link org.apache.druid.segment.join.JoinMatcher} being backed by potentially mutable iterables, such as lookups. - * If this ever changes so that we always have completely immutable data backing cursor, then we could potentially - * strengthen this contract to be able to mark and reset to exact cursor positions, but until that point do not count - * on this behavior without examining the specific type of cursors being used. - */ - void mark(); - - /** - * Reset to position set by {@link #mark()}. This is only guaranteed to be the first ocurring row with the same - * timestamp as the row when {@link #mark()} was called. - *

- * In most cursors, this is able to mark and return to an exact row, but currently the only caller of this method does - * not require this contract ({@link org.apache.druid.query.topn.TopNQueryEngine}) rather it only requires that the - * first timestamp of the marked position can be returned to, since TopN uses this to make multiple passes over a - * cursor within a granularity bucket interval (and so this method only really needs to work correctly if the data is - * time ordered). - *

- * Additionally, not all built-in implementations can be guaranteed to be able to mark an exact row due to potential - * underlying data mutability: - * - {@link RowBasedCursor} being potentially backed by mutable data, such as lookups when queried as segments - * - the cursor provided by {@link org.apache.druid.segment.join.HashJoinEngine} due to - * {@link org.apache.druid.segment.join.JoinMatcher} being backed by potentially mutable iterables, such as lookups. - * If this ever changes so that we always have completely immutable data backing cursor, then we could potentially - * strengthen this contract to be able to mark and reset to exact cursor positions, but until that point do not count - * on this behavior without examining the specific type of cursors being used. - */ - void resetToMark(); - /** * Reset to start of cursor and discard mark. Most cursor implementations are backed by immutable data, but there is * generically no guarantee that advancing through a cursor again will read exactly the same data or even number of diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 2fbf0d63f44f..bf8f3f8c3405 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -469,18 +469,6 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } - @Override - public void mark() - { - cursorOffset.mark(); - } - - @Override - public void resetToMark() - { - cursorOffset.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index a43f7086e6f0..f8e3a4c5abeb 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -118,21 +117,6 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } - @Override - public void mark() - { - markTime = DateTimes.utc(timestampFunction.applyAsLong(rowWalker.currentRow())); - } - - @Override - public void resetToMark() - { - rowId = 0; - rowWalker.reset(); - rowWalker.skipToDateTime(markTime, descending); - advanceToMatchingRow(); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index cb2f33f5b929..e894298cb10b 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -243,28 +243,6 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } - @Override - public void mark() - { - markIndex = index; - baseCursor.mark(); - } - - @Override - public void resetToMark() - { - index = markIndex; - baseCursor.resetToMark(); - getNextRow(); - if (unnestListForCurrentRow.isEmpty()) { - moveToNextNonEmptyRow(); - } - // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid - if (unnestListForCurrentRow == null || index >= unnestListForCurrentRow.size()) { - index = 0; - } - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 064d94cb01c8..975566fab4f6 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -291,28 +291,6 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } - @Override - public void mark() - { - markIndex = index; - baseCursor.mark(); - } - - @Override - public void resetToMark() - { - index = markIndex; - baseCursor.resetToMark(); - if (!baseCursor.isDone()) { - indexIntsForRow = new SingleIndexInts(); - indexedIntsForCurrentRow = dimSelector.getRow(); - } - // if base cursor reset to mark was not able to return to the exact row, the marked index might no longer be valid - if (indexedIntsForCurrentRow == null || index >= indexedIntsForCurrentRow.size()) { - index = 0; - } - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 31b15c62733a..62682745bbdd 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -214,50 +214,6 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } - @Override - public void mark() - { - if (!done && currEntry.get() != null) { - IncrementalIndexRow row = currEntry.get(); - markRowId = row.getRowIndex(); - markMillis = row.getTimestamp(); - } else { - markRowId = -1; - markMillis = isDescending ? Long.MAX_VALUE : Long.MIN_VALUE; - } - } - - @Override - public void resetToMark() - { - if (markRowId < 0) { - reset(); - return; - } - baseIter = facts.timeRangeIterable( - isDescending, - isDescending ? interval.getStartMillis() : markMillis, - isDescending ? markMillis : interval.getEndMillis() - ).iterator(); - - BaseQuery.checkInterrupted(); - - boolean foundMatched = false; - while (baseIter.hasNext()) { - IncrementalIndexRow entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getRowIndex())) { - continue; - } - currEntry.set(entry); - if (entry.getRowIndex() == markRowId && filterMatcher.matches(false)) { - foundMatched = true; - break; - } - } - - done = !foundMatched && (emptyRange || !baseIter.hasNext()); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index fc88b80bed27..6567baa140d7 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -242,25 +242,6 @@ public boolean isDoneOrInterrupted() return isDone() || Thread.currentThread().isInterrupted(); } - @Override - public void mark() - { - // join cursor handles mark and resetToMark imperfectly. The left cursor will be marked, but the right matchers - // will be completely reset. This is fine for left and inner joins per the contract of mark and resetToMark - // since the left cursor defines the rows timestamp. However for right and full joins, in an ideal world we - // would mark the right side, so we would have a way to reset to those positions. However, the contract of - // JoinMatcher is too loose to be able to guarantee that the underlying data behind the right side cursors - leftCursor.mark(); - } - - @Override - public void resetToMark() - { - leftCursor.resetToMark(); - joinMatcher.reset(); - initialize(); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 264f25c67932..b8ef88996ff1 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -139,19 +139,6 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } - @Override - public void mark() - { - baseCursor.mark(); - } - - @Override - public void resetToMark() - { - baseCursor.resetToMark(); - advanceToMatch(); - } - @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index f53793d692b1..173750a74c75 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -217,18 +217,6 @@ public boolean isDoneOrInterrupted() return cursor.isDoneOrInterrupted(); } - @Override - public void mark() - { - cursor.mark(); - } - - @Override - public void resetToMark() - { - cursor.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 00001525456d..e97f420867e9 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -70,20 +70,6 @@ public boolean isDoneOrInterrupted() return baseCursor.isDoneOrInterrupted(); } - @Override - public void mark() - { - baseCursor.mark(); - markRowNumber = rowNumberVirtualColumn.getValue(); - } - - @Override - public void resetToMark() - { - rowNumberVirtualColumn.setValue(markRowNumber); - baseCursor.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index 98355777fe2c..b0f015c627fc 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -6242,6 +6242,116 @@ public void testTopNAggregateTopnMetricFirst() assertExpectedResults(expectedResults, query); } + @Test + public void testTopNAggregateTopnMetricFirstWithGranularity() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .granularity(QueryRunnerTestHelper.MONTH_GRAN) + .dimension(QueryRunnerTestHelper.MARKET_DIMENSION) + .metric("last") + .threshold(3) + .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .aggregators( + new LongFirstAggregatorFactory("first", "index", null), + new LongLastAggregatorFactory("last", "index", null) + ) + .context(ImmutableMap.of("doAggregateTopNMetricFirst", true)) + .build(); + + List> expectedResults = Arrays.asList( + new Result<>( + DateTimes.of("2011-01-01T00:00:00.000Z"), + TopNResultValue.create( + Arrays.>asList( + ImmutableMap.builder() + .put("market", "total_market") + .put("first", 1000L) + .put("last", 1127L) + .build(), + ImmutableMap.builder() + .put("market", "upfront") + .put("first", 800L) + .put("last", 943L) + .build(), + ImmutableMap.builder() + .put("market", "spot") + .put("first", 100L) + .put("last", 155L) + .build() + ) + ) + ), + new Result<>( + DateTimes.of("2011-02-01T00:00:00.000Z"), + TopNResultValue.create( + Arrays.>asList( + ImmutableMap.builder() + .put("market", "total_market") + .put("first", 1203L) + .put("last", 1292L) + .build(), + ImmutableMap.builder() + .put("market", "upfront") + .put("first", 1667L) + .put("last", 1101L) + .build(), + ImmutableMap.builder() + .put("market", "spot") + .put("first", 132L) + .put("last", 114L) + .build() + ) + ) + ), + new Result<>( + DateTimes.of("2011-03-01T00:00:00.000Z"), + TopNResultValue.create( + Arrays.>asList( + ImmutableMap.builder() + .put("market", "total_market") + .put("first", 1124L) + .put("last", 1366L) + .build(), + ImmutableMap.builder() + .put("market", "upfront") + .put("first", 1166L) + .put("last", 1063L) + .build(), + ImmutableMap.builder() + .put("market", "spot") + .put("first", 153L) + .put("last", 125L) + .build() + ) + ) + ), + new Result<>( + DateTimes.of("2011-04-01T00:00:00.000Z"), + TopNResultValue.create( + Arrays.>asList( + ImmutableMap.builder() + .put("market", "total_market") + .put("first", 1314L) + .put("last", 1029L) + .build(), + ImmutableMap.builder() + .put("market", "upfront") + .put("first", 1447L) + .put("last", 780L) + .build(), + ImmutableMap.builder() + .put("market", "spot") + .put("first", 135L) + .put("last", 120L) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + private static Map makeRowWithNulls( String dimName, @Nullable Object dimValue, diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index 30b04fbb724d..265bffa371b1 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -214,18 +214,6 @@ public boolean isDoneOrInterrupted() return false; } - @Override - public void resetToMark() - { - index = markIndex; - } - - @Override - public void mark() - { - markIndex = index; - } - @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 26e776aa74bc..dca89de18b12 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -36,7 +36,6 @@ import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.filter.TypedInFilter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -510,7 +509,7 @@ public void test_makeCursor() ); } - Assert.assertEquals(3, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test @@ -857,100 +856,6 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() Assert.assertEquals(2, numCloses.get()); } - @Test - public void test_makeCursor_mark_resets_to_different_row() - { - final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 0, 1, 2, 3, 4, 5, 6, 7); - - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - final Cursor cursor = cursorHolder.asCursor(); - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("0"), - ImmutableList.of("1"), - ImmutableList.of("2"), - ImmutableList.of("3"), - // duplicate row since mark at 4 but resets to 3 since same timestamp - ImmutableList.of("3"), - ImmutableList.of("4"), - ImmutableList.of("5"), - ImmutableList.of("6"), - ImmutableList.of("7") - ), - walkCursorMarkResetDifferentRow(cursor, READ_STRING, 4) - ); - } - - Assert.assertEquals(3, numCloses.get()); - } - - @Test - public void test_makeCursor_mark_resets_to_different_row_descending() - { - final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 0, 1, 2, 3, 4, 5, 6, 7); - - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setPreferredOrdering(Cursors.descendingTimeOrder()) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - final Cursor cursor = cursorHolder.asCursor(); - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("7"), - ImmutableList.of("6"), - ImmutableList.of("5"), - ImmutableList.of("4"), - // duplicate rows since mark at 4 but resets to 5 since same timestamp - ImmutableList.of("5"), - ImmutableList.of("4"), - ImmutableList.of("3"), - ImmutableList.of("2"), - ImmutableList.of("1"), - ImmutableList.of("0") - ), - walkCursorMarkResetDifferentRow(cursor, READ_STRING, 4) - ); - } - - Assert.assertEquals(1, numCloses.get()); - } - - @Test - public void test_makeCursor_filterOnLong_resets_to_different_row() - { - final RowBasedStorageAdapter adapter = createIntAdapter(SAME_TIME_ROW_ADAPTER, 1, 2, 3, 4, 5, 6, 7); - - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter( - new TypedInFilter( - ValueType.LONG.name(), - ColumnType.LONG, - null, - Arrays.asList(3L, 4L), - null - ) - ) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - final Cursor cursor = cursorHolder.asCursor(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("3"), - // duplicate rows since cursor is marked at 1, but resets to 0 since same timestamp - ImmutableList.of("3"), - ImmutableList.of("4") - ), - walkCursorMarkResetDifferentRow(cursor, READ_STRING, 1) - ); - } - - - Assert.assertEquals(3, numCloses.get()); - } - private static List> walkCursor( final Cursor cursor, final List>> processors @@ -970,68 +875,7 @@ private static List> walkCursor( cursor.reset(); - // test cursor mark/resetToMark - int ctr = 0; - int mark = 2; - while (!cursor.isDone()) { - if (ctr == mark) { - cursor.mark(); - } - final List row = new ArrayList<>(); - - for (Supplier supplier : suppliers) { - row.add(supplier.get()); - } - - retVal.add(row); - ctr++; - cursor.advanceUninterruptibly(); - } - - if (ctr > mark) { - cursor.resetToMark(); - while (!cursor.isDone()) { - - final List row = new ArrayList<>(); - - for (Supplier supplier : suppliers) { - row.add(supplier.get()); - } - - retVal.set(mark++, row); - cursor.advanceUninterruptibly(); - } - } - - return retVal; - } - - private static List> walkCursorMarkResetDifferentRow( - final Cursor cursor, - final List>> processors, - int mark - ) - { - final List> suppliers = new ArrayList<>(); - for (Function> processor : processors) { - suppliers.add(processor.apply(cursor)); - } - - final List> retVal = new ArrayList<>(); - - // test cursor reset - while (!cursor.isDone()) { - cursor.advanceUninterruptibly(); - } - - cursor.reset(); - - // test cursor mark/resetToMark - int ctr = 0; while (!cursor.isDone()) { - if (ctr == mark) { - cursor.mark(); - } final List row = new ArrayList<>(); for (Supplier supplier : suppliers) { @@ -1039,26 +883,9 @@ private static List> walkCursorMarkResetDifferentRow( } retVal.add(row); - ctr++; cursor.advanceUninterruptibly(); } - if (ctr > mark) { - cursor.resetToMark(); - retVal.removeAll(retVal.subList(mark, retVal.size())); - while (!cursor.isDone()) { - - final List row = new ArrayList<>(); - - for (Supplier supplier : suppliers) { - row.add(supplier.get()); - } - - retVal.add(row); - cursor.advanceUninterruptibly(); - } - } - return retVal; } diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index 8d475476f19b..8ea5791463fc 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -248,22 +248,11 @@ public void test_unnest_adapters_basic() } cursor.reset(); - // test cursor mark/resetToMark - int mark = 2; while (!cursor.isDone()) { - if (count == mark) { - cursor.mark(); - } Object dimSelectorVal = dimSelector.getObject(); rows.add(dimSelectorVal); cursor.advance(); count++; - } - cursor.resetToMark(); - while (!cursor.isDone()) { - Object dimSelectorVal = dimSelector.getObject(); - rows.set(mark++, dimSelectorVal); - cursor.advance(); } /* each row has 8 entries. @@ -295,23 +284,12 @@ public void test_unnest_adapters_basic_array_column() } cursor.reset(); - // test cursor mark/resetToMark - int mark = 2; while (!cursor.isDone()) { - if (count == mark) { - cursor.mark(); - } Object dimSelectorVal = dimSelector.getObject(); rows.add(dimSelectorVal); cursor.advance(); count++; } - cursor.resetToMark(); - while (!cursor.isDone()) { - Object dimSelectorVal = dimSelector.getObject(); - rows.set(mark++, dimSelectorVal); - cursor.advance(); - } Assert.assertEquals(count, 12); Assert.assertEquals( Arrays.asList(2L, 3L, 1L, null, 3L, 1L, null, 2L, 9L, 1L, 2L, 3L), @@ -371,29 +349,15 @@ public Function columnFunction(String columnName) } cursor.reset(); - // test cursor mark/resetToMark - int mark = 4; while (!cursor.isDone()) { - if (count == mark) { - cursor.mark(); - } Object dimSelectorVal = dimSelector.getObject(); rows.add(dimSelectorVal); cursor.advance(); count++; } - cursor.resetToMark(); - rows.removeAll(rows.subList(mark, rows.size())); - while (!cursor.isDone()) { - Object dimSelectorVal = dimSelector.getObject(); - rows.add(dimSelectorVal); - cursor.advance(); - } Assert.assertEquals(count, 11); Assert.assertEquals( - // marked at position 4, however first row has the same timestamp so we effectively restart cursor after - // position 4 - Arrays.asList(1L, 2L, 3L, 4L, 1L, 2L, 3L, 4L, 5L, 6L, null, 7L, 8L, 9L, 10L), + Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, null, 7L, 8L, 9L, 10L), rows ); } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 4df2e2c0407e..c82003e1e84c 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.incremental; import com.fasterxml.jackson.core.JsonProcessingException; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -67,7 +66,6 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.filter.OrFilter; import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; @@ -317,97 +315,6 @@ public void testResetSanity() throws IOException } } - @Test - public void testMarkResetMarkSanity() throws IOException - { - - IncrementalIndex index = indexCreator.createIndex(); - DateTime t = DateTimes.nowUtc(); - Interval interval = new Interval(t.minusMinutes(1), t.plusMinutes(1)); - - index.add( - new MapBasedInputRow( - t.minus(1).getMillis(), - Collections.singletonList("billy"), - ImmutableMap.of("billy", "hi") - ) - ); - index.add( - new MapBasedInputRow( - t.minus(1).getMillis(), - Collections.singletonList("sally"), - ImmutableMap.of("sally", "bo") - ) - ); - - index.add( - new MapBasedInputRow( - t.getMillis(), - Collections.singletonList("billy"), - ImmutableMap.of("billy", "hi2") - ) - ); - index.add( - new MapBasedInputRow( - t.getMillis(), - Collections.singletonList("sally"), - ImmutableMap.of("sally", "bo2") - ) - ); - - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - - for (boolean descending : Arrays.asList(false, true)) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter( - new OrFilter( - ImmutableList.of( - new SelectorFilter("sally", "bo"), - new SelectorFilter("sally", "bo2") - ) - ) - ) - .setInterval(interval) - .setPreferredOrdering( - descending ? Cursors.descendingTimeOrder() : null - ) - .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - Cursor cursor = cursorHolder.asCursor(); - DimensionSelector dimSelector; - dimSelector = cursor.getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals(descending ? "bo2" : "bo", dimSelector.lookupName(dimSelector.getRow().get(0))); - cursor.advance(); - cursor.mark(); - Assert.assertEquals(descending ? "bo" : "bo2", dimSelector.lookupName(dimSelector.getRow().get(0))); - index.add( - new MapBasedInputRow( - t.minus(1).getMillis(), - Collections.singletonList("sally"), - ImmutableMap.of("sally", "ah") - ) - ); - - // Cursor mark and resetToMark should not be affected by out of order values - cursor.resetToMark(); - - dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals(descending ? "bo" : "bo2", dimSelector.lookupName(dimSelector.getRow().get(0))); - - // Cursor reset should not be affected by mark/reset mark either - cursor.reset(); - - dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals(descending ? "bo2" : "bo", dimSelector.lookupName(dimSelector.getRow().get(0))); - } - } - } - @Test public void testSingleValueTopN() throws IOException { diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index 59755da0c38b..e323b2599131 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -710,7 +710,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -718,17 +718,6 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull() new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L} ) ); - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, - // duplicate row, right join reset to mark starts at beginning - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L} - ), - 1 - ); } @Test @@ -754,7 +743,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup( FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -762,17 +751,6 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"} ) ); - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, - // duplicate row, right join reset to mark starts at beginning - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"} - ), - 1 - ); } @Test @@ -799,7 +777,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -807,17 +785,6 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull() new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L} ) ); - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, - // duplicate row, full join reset to mark starts at beginning - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca", 16L} - ), - 1 - ); } @Test @@ -843,7 +810,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -851,17 +818,6 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup() new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"} ) ); - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, - // duplicate row, full join reset to mark starts at beginning - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, - new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "USCA", "Usca"} - ), - 1 - ); } @Test @@ -1298,7 +1254,7 @@ public void test_makeCursor_factToCountryAlwaysTrue() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -1322,36 +1278,6 @@ public void test_makeCursor_factToCountryAlwaysTrue() new Object[]{"Diskussion:Sebastian Schulz", "Fourems"} ) ); - - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, - new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, - new Object[]{"Diskussion:Sebastian Schulz", "Chile"}, - // duplicate rows start here, reset to mark resets right - new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, - new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, - new Object[]{"Diskussion:Sebastian Schulz", "Chile"}, - new Object[]{"Diskussion:Sebastian Schulz", "Germany"}, - new Object[]{"Diskussion:Sebastian Schulz", "Ecuador"}, - new Object[]{"Diskussion:Sebastian Schulz", "France"}, - new Object[]{"Diskussion:Sebastian Schulz", "United Kingdom"}, - new Object[]{"Diskussion:Sebastian Schulz", "Italy"}, - new Object[]{"Diskussion:Sebastian Schulz", "Japan"}, - new Object[]{"Diskussion:Sebastian Schulz", "Republic of Korea"}, - new Object[]{"Diskussion:Sebastian Schulz", "Mexico"}, - new Object[]{"Diskussion:Sebastian Schulz", "Norway"}, - new Object[]{"Diskussion:Sebastian Schulz", "El Salvador"}, - new Object[]{"Diskussion:Sebastian Schulz", "United States"}, - new Object[]{"Diskussion:Sebastian Schulz", "Atlantis"}, - new Object[]{"Diskussion:Sebastian Schulz", "States United"}, - new Object[]{"Diskussion:Sebastian Schulz", "Usca"}, - new Object[]{"Diskussion:Sebastian Schulz", "Fourems"} - ), - 3 - ); } @Test @@ -1429,7 +1355,7 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -1453,36 +1379,6 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup() new Object[]{"Diskussion:Sebastian Schulz", "Fourems"} ) ); - - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, - new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, - new Object[]{"Diskussion:Sebastian Schulz", "Chile"}, - new Object[]{"Diskussion:Sebastian Schulz", "Germany"}, - new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, - new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, - new Object[]{"Diskussion:Sebastian Schulz", "Chile"}, - new Object[]{"Diskussion:Sebastian Schulz", "Germany"}, - new Object[]{"Diskussion:Sebastian Schulz", "Ecuador"}, - new Object[]{"Diskussion:Sebastian Schulz", "France"}, - new Object[]{"Diskussion:Sebastian Schulz", "United Kingdom"}, - new Object[]{"Diskussion:Sebastian Schulz", "Italy"}, - new Object[]{"Diskussion:Sebastian Schulz", "Japan"}, - new Object[]{"Diskussion:Sebastian Schulz", "Republic of Korea"}, - new Object[]{"Diskussion:Sebastian Schulz", "Mexico"}, - new Object[]{"Diskussion:Sebastian Schulz", "Norway"}, - new Object[]{"Diskussion:Sebastian Schulz", "El Salvador"}, - new Object[]{"Diskussion:Sebastian Schulz", "United States"}, - new Object[]{"Diskussion:Sebastian Schulz", "Atlantis"}, - new Object[]{"Diskussion:Sebastian Schulz", "States United"}, - new Object[]{"Diskussion:Sebastian Schulz", "Usca"}, - new Object[]{"Diskussion:Sebastian Schulz", "Fourems"} - ), - 4 - ); } @Test @@ -1758,7 +1654,7 @@ public void test_makeCursor_factToRegionTheWrongWay() FACT_TO_REGION_PREFIX + "countryIsoCode" ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(buildSpec), columns, ImmutableList.of( @@ -1770,22 +1666,6 @@ public void test_makeCursor_factToRegionTheWrongWay() new Object[]{"Old Anatolian Turkish", "VA", "US", "Virginia", "US"} ) ); - - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(buildSpec), - columns, - ImmutableList.of( - new Object[]{"Giusy Ferreri discography", "VA", "IT", "Provincia di Varese", "IT"}, - new Object[]{"Giusy Ferreri discography", "VA", "IT", "Virginia", "US"}, - new Object[]{"Roma-Bangkok", "VA", "IT", "Provincia di Varese", "IT"}, - new Object[]{"Roma-Bangkok", "VA", "IT", "Virginia", "US"}, - new Object[]{"Old Anatolian Turkish", "VA", "US", "Provincia di Varese", "IT"}, - // duplicate row from mark/reset to mark - new Object[]{"Old Anatolian Turkish", "VA", "US", "Provincia di Varese", "IT"}, - new Object[]{"Old Anatolian Turkish", "VA", "US", "Virginia", "US"} - ), - 5 - ); } @Test @@ -2107,7 +1987,7 @@ public void test_makeCursor_factToCountryRightWithBaseFilter() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), columns, ImmutableList.of( @@ -2133,66 +2013,6 @@ public void test_makeCursor_factToCountryRightWithBaseFilter() new Object[]{null, null, "MMMM", "Fourems", 205L} ) ); - - // no dupes because mark/reset happens while advancing left - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), - columns, - ImmutableList.of( - new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, - new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, - new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, - new Object[]{null, null, "AU", "Australia", 0L}, - new Object[]{null, null, "CL", "Chile", 2L}, - new Object[]{null, null, "DE", "Germany", 3L}, - new Object[]{null, null, "EC", "Ecuador", 4L}, - new Object[]{null, null, "FR", "France", 5L}, - new Object[]{null, null, "GB", "United Kingdom", 6L}, - new Object[]{null, null, "IT", "Italy", 7L}, - new Object[]{null, null, "JP", "Japan", 8L}, - new Object[]{null, null, "KR", "Republic of Korea", 9L}, - new Object[]{null, null, "MX", "Mexico", 10L}, - new Object[]{null, null, "NO", "Norway", 11L}, - new Object[]{null, null, "SV", "El Salvador", 12L}, - new Object[]{null, null, "US", "United States", 13L}, - new Object[]{null, null, "AX", "Atlantis", 14L}, - new Object[]{null, null, "SU", "States United", 15L}, - new Object[]{null, null, "USCA", "Usca", 16L}, - new Object[]{null, null, "MMMM", "Fourems", 205L} - ), - 2 - ); - - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), - columns, - ImmutableList.of( - new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, - new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, - new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, - new Object[]{null, null, "AU", "Australia", 0L}, - new Object[]{null, null, "CL", "Chile", 2L}, - // dupes start here from mark/reset - new Object[]{null, null, "AU", "Australia", 0L}, - new Object[]{null, null, "CL", "Chile", 2L}, - new Object[]{null, null, "DE", "Germany", 3L}, - new Object[]{null, null, "EC", "Ecuador", 4L}, - new Object[]{null, null, "FR", "France", 5L}, - new Object[]{null, null, "GB", "United Kingdom", 6L}, - new Object[]{null, null, "IT", "Italy", 7L}, - new Object[]{null, null, "JP", "Japan", 8L}, - new Object[]{null, null, "KR", "Republic of Korea", 9L}, - new Object[]{null, null, "MX", "Mexico", 10L}, - new Object[]{null, null, "NO", "Norway", 11L}, - new Object[]{null, null, "SV", "El Salvador", 12L}, - new Object[]{null, null, "US", "United States", 13L}, - new Object[]{null, null, "AX", "Atlantis", 14L}, - new Object[]{null, null, "SU", "States United", 15L}, - new Object[]{null, null, "USCA", "Usca", 16L}, - new Object[]{null, null, "MMMM", "Fourems", 205L} - ), - 5 - ); } @Test @@ -2222,7 +2042,7 @@ public void test_makeCursor_factToCountryFullWithBaseFilter() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); - JoinTestHelper.verifyCursorNoMarkReset( + JoinTestHelper.verifyCursor( adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), columns, ImmutableList.of( @@ -2249,67 +2069,6 @@ public void test_makeCursor_factToCountryFullWithBaseFilter() new Object[]{null, null, "MMMM", "Fourems", 205L} ) ); - - // no dupes because mark/reset happens while advancing left - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), - columns, - ImmutableList.of( - new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, - new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, - new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, - new Object[]{"Orange Soda", "MatchNothing", null, null, NullHandling.sqlCompatible() ? null : 0L}, - new Object[]{null, null, "AU", "Australia", 0L}, - new Object[]{null, null, "CL", "Chile", 2L}, - new Object[]{null, null, "DE", "Germany", 3L}, - new Object[]{null, null, "EC", "Ecuador", 4L}, - new Object[]{null, null, "FR", "France", 5L}, - new Object[]{null, null, "GB", "United Kingdom", 6L}, - new Object[]{null, null, "IT", "Italy", 7L}, - new Object[]{null, null, "JP", "Japan", 8L}, - new Object[]{null, null, "KR", "Republic of Korea", 9L}, - new Object[]{null, null, "MX", "Mexico", 10L}, - new Object[]{null, null, "NO", "Norway", 11L}, - new Object[]{null, null, "SV", "El Salvador", 12L}, - new Object[]{null, null, "US", "United States", 13L}, - new Object[]{null, null, "AX", "Atlantis", 14L}, - new Object[]{null, null, "SU", "States United", 15L}, - new Object[]{null, null, "USCA", "Usca", 16L}, - new Object[]{null, null, "MMMM", "Fourems", 205L} - ), - 1 - ); - - JoinTestHelper.verifyCursorMarkReset( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), - columns, - ImmutableList.of( - new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, - new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, - new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, - new Object[]{"Orange Soda", "MatchNothing", null, null, NullHandling.sqlCompatible() ? null : 0L}, - new Object[]{null, null, "AU", "Australia", 0L}, - // dupes start here - new Object[]{null, null, "AU", "Australia", 0L}, - new Object[]{null, null, "CL", "Chile", 2L}, - new Object[]{null, null, "DE", "Germany", 3L}, - new Object[]{null, null, "EC", "Ecuador", 4L}, - new Object[]{null, null, "FR", "France", 5L}, - new Object[]{null, null, "GB", "United Kingdom", 6L}, - new Object[]{null, null, "IT", "Italy", 7L}, - new Object[]{null, null, "JP", "Japan", 8L}, - new Object[]{null, null, "KR", "Republic of Korea", 9L}, - new Object[]{null, null, "MX", "Mexico", 10L}, - new Object[]{null, null, "NO", "Norway", 11L}, - new Object[]{null, null, "SV", "El Salvador", 12L}, - new Object[]{null, null, "US", "United States", 13L}, - new Object[]{null, null, "AX", "Atlantis", 14L}, - new Object[]{null, null, "SU", "States United", 15L}, - new Object[]{null, null, "USCA", "Usca", 16L}, - new Object[]{null, null, "MMMM", "Fourems", 205L} - ), - 5 - ); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 74c9e56fa8e7..33ad0dfb6a8a 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -68,7 +68,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; import java.util.function.Function; import java.util.function.Supplier; import java.util.function.ToLongFunction; @@ -381,126 +380,12 @@ public static List readCursor(final CursorHolder cursorHolder, final L } } - public static List readCursorMarkReset(final CursorHolder cursorHolder, final List columns, int mark) - { - try { - final Cursor cursor = cursorHolder.asCursor(); - final List> readers = columns - .stream() - .map( - column -> - ColumnProcessors.makeProcessor( - column, - SIMPLE_READER, - cursor.getColumnSelectorFactory() - ) - ) - .collect(Collectors.toList()); - - final List rows = new ArrayList<>(); - boolean interruptible = false; // test both advance() and advanceUninterruptibly() - - // test cursor reset - while (!cursor.isDone()) { - if (interruptible) { - cursor.advance(); - } else { - cursor.advanceUninterruptibly(); - } - - interruptible = !interruptible; - } - - cursor.reset(); - - // test cursor mark/resetToMark - int ctr = 0; - while (!cursor.isDone()) { - if (ctr == mark) { - cursor.mark(); - } - final Object[] row = new Object[columns.size()]; - - for (int i = 0; i < row.length; i++) { - row[i] = readers.get(i).get(); - } - - rows.add(row); - if (interruptible) { - cursor.advance(); - } else { - cursor.advanceUninterruptibly(); - } - - interruptible = !interruptible; - ctr++; - } - - if (rows.size() > mark) { - cursor.resetToMark(); - rows.removeAll(rows.subList(mark, rows.size())); - while (!cursor.isDone()) { - final Object[] row = new Object[columns.size()]; - - for (int i = 0; i < row.length; i++) { - row[i] = readers.get(i).get(); - } - - rows.add(row); - if (interruptible) { - cursor.advance(); - } else { - cursor.advanceUninterruptibly(); - } - - interruptible = !interruptible; - } - } - - return rows; - } - finally { - cursorHolder.close(); - } - } public static void verifyCursor( final CursorHolder cursorHolder, final List columns, final List expectedRows ) - { - final int max = expectedRows.size(); - final List rows; - int mark = -1; - if (max > 1) { - mark = ThreadLocalRandom.current().nextInt(1, max); - rows = readCursorMarkReset(cursorHolder, columns, mark); - } else { - rows = readCursor(cursorHolder, columns); - } - - for (int i = 0; i < rows.size(); i++) { - try { - log.info("Row #%-2d: %s", i, TestHelper.JSON_MAPPER.writeValueAsString(rows.get(i))); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - Assert.assertEquals("number of rows" + (mark > 0 ? " (mark: " + mark + ")" : ""), expectedRows.size(), rows.size()); - - for (int i = 0; i < rows.size(); i++) { - Assert.assertArrayEquals("row #" + i, expectedRows.get(i), rows.get(i)); - } - } - - public static void verifyCursorNoMarkReset( - final CursorHolder cursorHolder, - final List columns, - final List expectedRows - ) { final List rows = readCursor(cursorHolder, columns); @@ -520,31 +405,6 @@ public static void verifyCursorNoMarkReset( } } - public static void verifyCursorMarkReset( - final CursorHolder cursorHolder, - final List columns, - final List expectedRows, - int mark - ) - { - final List rows = readCursorMarkReset(cursorHolder, columns, mark); - - for (int i = 0; i < rows.size(); i++) { - try { - log.info("Row #%-2d: %s", i, TestHelper.JSON_MAPPER.writeValueAsString(rows.get(i))); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - Assert.assertEquals("number of rows", expectedRows.size(), rows.size()); - - for (int i = 0; i < rows.size(); i++) { - Assert.assertArrayEquals("row #" + i, expectedRows.get(i), rows.get(i)); - } - } - public static T withRowsFromResource( final String resource, final Function>, T> f diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index 1435a7f27378..1a5e6dc63de4 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -141,18 +141,6 @@ public boolean isDoneOrInterrupted() return cursor.isDoneOrInterrupted(); } - @Override - public void mark() - { - cursor.mark(); - } - - @Override - public void resetToMark() - { - cursor.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index 01f6f8cba39b..a3e4d028c614 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -114,22 +114,6 @@ public boolean isDoneOrInterrupted() return isDone(); } - @Override - public void mark() - { - markCounter = rowCounter; - } - - @Override - public void resetToMark() - { - rowCounter = markCounter; - iterator = iteratorSupplier.get(); - for (int i = 0; i < markCounter; i++) { - iterator.next(); - } - } - @Override public void reset() { From ec3e7a174cec2e17ba82a5896c75fd5bd3a0b9e5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 01:35:22 -0700 Subject: [PATCH 65/74] remove Offset.mark Offset.resetToMark --- .../frame/segment/FrameFilteredOffset.java | 12 ---- .../apache/druid/segment/BitmapOffset.java | 15 ----- .../apache/druid/segment/FilteredOffset.java | 12 ---- .../segment/QueryableIndexCursorHolder.java | 12 ---- .../druid/segment/SimpleAscendingOffset.java | 12 ---- .../druid/segment/SimpleDescendingOffset.java | 12 ---- .../org/apache/druid/segment/data/Offset.java | 10 ---- .../druid/segment/BitmapOffsetTest.java | 57 ------------------- 8 files changed, 142 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java index 6aa5b81aeb02..3db294801c03 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java @@ -81,18 +81,6 @@ public void setCurrentOffset(int currentOffset) } } - @Override - public void mark() - { - baseOffset.mark(); - } - - @Override - public void resetToMark() - { - baseOffset.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index a6f424ea5fbf..94324fe22060 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; @@ -206,20 +205,6 @@ public boolean withinBounds() return value > INVALID_VALUE; } - @Override - public void mark() - { - iteratorForMark = iterator.clone(); - valueForMark = value; - } - - @Override - public void resetToMark() - { - iterator = Preconditions.checkNotNull(iteratorForMark).clone(); - value = valueForMark; - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index abbb26c417bb..755a399f8e0d 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -56,18 +56,6 @@ public boolean withinBounds() return baseOffset.withinBounds(); } - @Override - public void mark() - { - baseOffset.mark(); - } - - @Override - public void resetToMark() - { - baseOffset.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index bf8f3f8c3405..0f0646839ac7 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -515,18 +515,6 @@ public boolean withinBounds() return timeInRange(timestamps.getLongSingleValueRow(baseOffset.getOffset())); } - @Override - public void mark() - { - baseOffset.mark(); - } - - @Override - public void resetToMark() - { - baseOffset.resetToMark(); - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java index b8b16ead19c8..8fee96b6b719 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java @@ -61,18 +61,6 @@ public void setCurrentOffset(int currentOffset) this.currentOffset = currentOffset; } - @Override - public void mark() - { - markOffset = currentOffset; - } - - @Override - public void resetToMark() - { - currentOffset = markOffset; - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java index 3ba6f06f8a3b..7b0826bc4fa4 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java @@ -55,18 +55,6 @@ public boolean withinBounds() return currentOffset >= 0; } - @Override - public void mark() - { - markOffset = currentOffset; - } - - @Override - public void resetToMark() - { - currentOffset = markOffset; - } - @Override public void reset() { diff --git a/processing/src/main/java/org/apache/druid/segment/data/Offset.java b/processing/src/main/java/org/apache/druid/segment/data/Offset.java index 1f521e9f4a59..1fef6769936e 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/Offset.java +++ b/processing/src/main/java/org/apache/druid/segment/data/Offset.java @@ -53,16 +53,6 @@ public abstract class Offset implements ReadableOffset, Cloneable @CalledFromHotLoop public abstract boolean withinBounds(); - /** - * Mark the current offset to recall later with {@link #resetToMark()} - */ - public abstract void mark(); - - /** - * Reset to the offset set by {@link #mark()} - */ - public abstract void resetToMark(); - /** * Resets the Offset to the position it was created or cloned with. */ diff --git a/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java b/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java index 7cb55e62ac0f..ef892496bf1a 100644 --- a/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java +++ b/processing/src/test/java/org/apache/druid/segment/BitmapOffsetTest.java @@ -23,8 +23,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.collections.bitmap.BitSetBitmapFactory; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ConciseBitmapFactory; @@ -38,7 +36,6 @@ import org.junit.runners.Parameterized; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; /** */ @@ -106,58 +103,4 @@ public void testSanity() } Assert.assertEquals(count, expected.length); } - - @Test - public void testMarkResetToMark() - { - MutableBitmap mutable = factory.makeEmptyMutableBitmap(); - IntList rows = new IntArrayList(); - for (int i = 0; i < 1_000_000; i++) { - if (ThreadLocalRandom.current().nextBoolean()) { - rows.add(i); - mutable.add(i); - } - } - - ImmutableBitmap bitmap = factory.makeImmutableBitmap(mutable); - final BitmapOffset offset = BitmapOffset.of(bitmap, descending, bitmap.size()); - final IntList expected; - if (descending) { - expected = new IntArrayList(rows.size()); - for (int i = rows.size() - 1; i >= 0; i--) { - expected.add(rows.getInt(i)); - } - } else { - expected = rows; - } - - int count = 0; - int mark = rows.size() / 2; - while (offset.withinBounds()) { - Assert.assertEquals(expected.getInt(count), offset.getOffset()); - - if (count == mark) { - offset.mark(); - } - - ++count; - offset.increment(); - } - - offset.resetToMark(); - count = mark; - while (offset.withinBounds()) { - Assert.assertEquals(expected.getInt(count), offset.getOffset()); - - if (count == mark) { - offset.mark(); - } - - ++count; - offset.increment(); - } - - Assert.assertEquals(count, expected.size()); - } - } From 287f7e256f7d221f83914d29dc12cdefdc14025b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 01:53:15 -0700 Subject: [PATCH 66/74] remove stale unused --- .../java/org/apache/druid/frame/segment/FrameCursor.java | 2 -- .../src/main/java/org/apache/druid/segment/Cursor.java | 6 +++--- .../org/apache/druid/segment/SimpleAscendingOffset.java | 2 -- .../org/apache/druid/segment/SimpleDescendingOffset.java | 2 -- 4 files changed, 3 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 83bd1e673d98..82cb092779da 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -37,7 +37,6 @@ public class FrameCursor implements Cursor { private final SimpleSettableOffset offset; private final ColumnSelectorFactory columnSelectorFactory; - private int markOffset = 0; public FrameCursor( SimpleSettableOffset offset, @@ -82,7 +81,6 @@ public boolean isDoneOrInterrupted() @Override public void reset() { - markOffset = 0; offset.reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index 0a7ae22bb972..a1fc14c10540 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -87,9 +87,9 @@ public interface Cursor boolean isDoneOrInterrupted(); /** - * Reset to start of cursor and discard mark. Most cursor implementations are backed by immutable data, but there is - * generically no guarantee that advancing through a cursor again will read exactly the same data or even number of - * rows, since the underlying data might be mutable in some cases. + * Reset to start of cursor. Most cursor implementations are backed by immutable data, but there is generically no + * guarantee that advancing through a cursor again will read exactly the same data or even number of rows, since the + * underlying data might be mutable in some cases. */ void reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java index 8fee96b6b719..b4c6111804eb 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleAscendingOffset.java @@ -28,7 +28,6 @@ public class SimpleAscendingOffset extends SimpleSettableOffset private final int rowCount; private final int initialOffset; private int currentOffset; - private int markOffset; public SimpleAscendingOffset(int rowCount) { @@ -39,7 +38,6 @@ private SimpleAscendingOffset(int initialOffset, int rowCount) { this.initialOffset = initialOffset; this.currentOffset = initialOffset; - this.markOffset = initialOffset; this.rowCount = rowCount; } diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java index 7b0826bc4fa4..fe8aba36fb99 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleDescendingOffset.java @@ -28,7 +28,6 @@ public class SimpleDescendingOffset extends SimpleSettableOffset private final int rowCount; private final int initialOffset; private int currentOffset; - private int markOffset; public SimpleDescendingOffset(int rowCount) { @@ -40,7 +39,6 @@ private SimpleDescendingOffset(int initialOffset, int rowCount) this.rowCount = rowCount; this.initialOffset = initialOffset; this.currentOffset = initialOffset; - this.markOffset = initialOffset; } @Override From 19acd700718dbfd32f50ec2f8a2710b688bb9234 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 03:38:26 -0700 Subject: [PATCH 67/74] unused --- .../main/java/org/apache/druid/segment/RowBasedCursor.java | 3 --- .../apache/druid/segment/UnnestColumnValueSelectorCursor.java | 2 -- .../java/org/apache/druid/segment/UnnestDimensionCursor.java | 2 -- .../segment/incremental/IncrementalIndexCursorHolder.java | 4 ---- 4 files changed, 11 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index f8e3a4c5abeb..6aa42e8625ae 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -46,7 +46,6 @@ public class RowBasedCursor implements Cursor private final ValueMatcher valueMatcher; private long rowId = 0; - private DateTime markTime; public RowBasedCursor( final RowWalker rowWalker, @@ -63,7 +62,6 @@ public RowBasedCursor( this.interval = interval; this.descending = descending; this.startTime = descending ? interval.getEnd().minus(1) : interval.getStart(); - this.markTime = startTime; this.columnSelectorFactory = virtualColumns.wrap( new RowBasedColumnSelectorFactory<>( rowWalker::currentRow, @@ -121,7 +119,6 @@ public boolean isDoneOrInterrupted() public void reset() { rowId = 0; - markTime = startTime; rowWalker.reset(); rowWalker.skipToDateTime(startTime, descending); advanceToMatchingRow(); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index e894298cb10b..a2a920cba1d5 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -65,7 +65,6 @@ public class UnnestColumnValueSelectorCursor implements Cursor @MonotonicNonNull private List unnestListForCurrentRow; private boolean needInitialization; - private int markIndex = 0; public UnnestColumnValueSelectorCursor( @@ -247,7 +246,6 @@ public boolean isDoneOrInterrupted() public void reset() { index = 0; - markIndex = 0; needInitialization = true; baseCursor.reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 975566fab4f6..398c0a21deff 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -77,7 +77,6 @@ public class UnnestDimensionCursor implements Cursor private SingleIndexInts indexIntsForRow; private final int nullId; private final int idOffset; - private int markIndex = 0; public UnnestDimensionCursor( Cursor cursor, @@ -295,7 +294,6 @@ public boolean isDoneOrInterrupted() public void reset() { index = 0; - markIndex = 0; needInitialization = true; baseCursor.reset(); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 62682745bbdd..71f482ed10ad 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -106,8 +106,6 @@ static class IncrementalIndexCursor implements Cursor private boolean emptyRange; private int numAdvanced; private boolean done; - private int markRowId = -1; - private long markMillis = -1; IncrementalIndexCursor( IncrementalIndexStorageAdapter storageAdapter, @@ -217,8 +215,6 @@ public boolean isDoneOrInterrupted() @Override public void reset() { - markRowId = -1; - markMillis = isDescending ? Long.MAX_VALUE : Long.MIN_VALUE; baseIter = cursorIterable.iterator(); if (numAdvanced == -1) { From d0667609efc6406ebd3bda3f6ded56e60efed09f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 04:12:07 -0700 Subject: [PATCH 68/74] more unused cleanup --- .../segment/incremental/IncrementalIndexCursorHolder.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 71f482ed10ad..f0966a828eb9 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -99,8 +99,6 @@ static class IncrementalIndexCursor implements Cursor private final ValueMatcher filterMatcher; private final int maxRowIndex; private final IncrementalIndex.FactsHolder facts; - private final Interval interval; - private final boolean isDescending; private Iterator baseIter; private Iterable cursorIterable; private boolean emptyRange; @@ -128,8 +126,6 @@ static class IncrementalIndexCursor implements Cursor filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; facts = index.getFacts(); - interval = actualInterval; - isDescending = descending; cursorIterable = facts.timeRangeIterable( descending, actualInterval.getStartMillis(), From 176c3a5ae5112d551d242378f6d7e2e85a75c3e3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 16:02:03 -0700 Subject: [PATCH 69/74] misc cleanup --- .../CompressedBigDecimalFactoryTestBase.java | 7 +- .../apache/druid/segment/RowBasedCursor.java | 2 +- .../join/HashJoinSegmentStorageAdapter.java | 13 +-- .../join/table/FrameBasedIndexedTable.java | 74 +++++++++-------- .../processor/OutputChannelFactoryTest.java | 46 +++++------ .../testutil/RowNumberUpdatingCursor.java | 2 - .../org/apache/druid/segment/ListCursor.java | 2 - .../PredicateValueMatcherFactoryTest.java | 82 ++++++------------- .../selector/TestColumnValueSelector.java | 18 ++-- 9 files changed, 102 insertions(+), 144 deletions(-) diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java index 3562e6a35d65..ca98eb3796f2 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.aggregation.AggregateCombiner; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -151,8 +150,7 @@ protected void testCombinerResetHelper(AggregateCombiner combiner) { TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( CompressedBigDecimal.class, - ImmutableList.of(ArrayCompressedBigDecimal.wrap(new int[]{67, 0}, 0)), - DateTimes.of("2020-01-01") + ImmutableList.of(ArrayCompressedBigDecimal.wrap(new int[]{67, 0}, 0)) ); columnValueSelector.advance(); @@ -167,8 +165,7 @@ protected void testCombinerFoldHelper(AggregateCombiner combiner, String ImmutableList.of( ArrayCompressedBigDecimal.wrap(new int[]{1, 0}, 0), ArrayCompressedBigDecimal.wrap(new int[]{10, 0}, 0) - ), - DateTimes.of("2020-01-01") + ) ); columnValueSelector.advance(); diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index 6aa42e8625ae..d1a70783f6b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -41,7 +41,7 @@ public class RowBasedCursor implements Cursor private final ToLongFunction timestampFunction; private final Interval interval; private final boolean descending; - private DateTime startTime; + private final DateTime startTime; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher valueMatcher; diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 186352b14b1c..1276a60523f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -250,10 +250,11 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec) @Override public Cursor asCursor() { - // Filter pre-analysis key implied by the call to "makeCursor". We need to sanity-check that it matches - // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursor" call (it - // requires access to the query context) so we'll need to skip sanity-checking it, by re-using the one present - // in the cached key.) + // Filter pre-analysis key implied by the call to "makeCursorHolder". We need to sanity-check that it matches + // the actual pre-analysis that was done. Note: we could now infer a rewrite config from the "makeCursorHolder" + // call (it requires access to the query context which we now have access to since the move away from + // CursorFactory) but this code hasn't been updated to sanity-check it, so currently we are still skipping it + // by re-using the one present in the cached key. final JoinFilterPreAnalysisKey keyIn = new JoinFilterPreAnalysisKey( joinFilterPreAnalysis.getKey().getRewriteConfig(), @@ -265,7 +266,7 @@ public Cursor asCursor() final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey(); final JoinFilterPreAnalysis preAnalysis; if (keyIn.equals(keyCached)) { - // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursor call + // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursorHolder call // (keyIn). preAnalysis = joinFilterPreAnalysis; } else { @@ -353,7 +354,7 @@ public boolean isBaseColumn(final String column) */ private Optional getClauseForColumn(final String column) { - // Check clauses in reverse, since "makeCursor" creates the cursor in such a way that the last clause + // Check clauses in reverse, since "makeCursorHolder" creates the cursor in such a way that the last clause // gets first dibs to claim a column. return Lists.reverse(clauses) .stream() diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index 970ba5fe7279..080a3c920aee 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.FrameSignaturePair; @@ -40,6 +39,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SimpleAscendingOffset; @@ -114,56 +114,58 @@ public FrameBasedIndexedTable( indexBuilders.add(m); } - final Closer closer = Closer.create(); - final Sequence cursors = Sequences.simple( + final Sequence storageAdapters = Sequences.simple( frameBasedInlineDataSource .getFrames() .stream() .map(frameSignaturePair -> { Frame frame = frameSignaturePair.getFrame(); RowSignature rowSignature = frameSignaturePair.getRowSignature(); - FrameStorageAdapter frameStorageAdapter = - new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); - return closer.register(frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asCursor(); + return new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); }) .collect(Collectors.toList()) - ).withBaggage(closer); + ); final Sequence sequence = Sequences.map( - cursors, - cursor -> { - if (cursor == null) { - return 0; - } - int rowNumber = 0; - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap - // indexes, but, an optimization for another day - final List selectors = keyColumnNames - .stream() - .map(columnSelectorFactory::makeColumnValueSelector) - .collect(Collectors.toList()); - - while (!cursor.isDone()) { - for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { - final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); - final int columnPosition = rowSignature.indexOf(keyColumnName); - final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); - keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); + storageAdapters, + storageAdapter -> { + try (final CursorHolder holder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = holder.asCursor(); + if (cursor == null) { + return 0; } - if (rowNumber % 100_000 == 0) { - if (rowNumber == 0) { - LOG.debug("Indexed first row for frame based datasource"); - } else { - LOG.debug("Indexed row %s for frame based datasource", rowNumber); + int rowNumber = 0; + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap + // indexes, but, an optimization for another day + final List selectors = keyColumnNames + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + while (!cursor.isDone()) { + for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex + < selectors.size(); keyColumnSelectorIndex++) { + final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); + final int columnPosition = rowSignature.indexOf(keyColumnName); + final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); + keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); + } + + if (rowNumber % 100_000 == 0) { + if (rowNumber == 0) { + LOG.debug("Indexed first row for frame based datasource"); + } else { + LOG.debug("Indexed row %s for frame based datasource", rowNumber); + } } + rowNumber++; + cursor.advance(); } - rowNumber++; - cursor.advance(); + return rowNumber; } - return rowNumber; } ); diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java index 846d07ebc11b..1e65ed7142a8 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java @@ -128,32 +128,30 @@ protected void verifySingleFrameReadableChannel( Assert.assertTrue(readableFrameChannel.isFinished()); readableFrameChannel.close(); + FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( + readbackFrame, + FrameReader.create(adapter.getRowSignature()), + Intervals.ETERNITY + ); // build list of rows from written and read data to verify - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN);) { - List> writtenData = - FrameTestUtil.readRowsFromCursor( - cursorHolder.asCursor(), - adapter.getRowSignature() - ).toList(); - - FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( - readbackFrame, - FrameReader.create(adapter.getRowSignature()), - Intervals.ETERNITY + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN); + final CursorHolder frameMaker = frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN) + ) { + List> writtenData = FrameTestUtil.readRowsFromCursor( + cursorHolder.asCursor(), + adapter.getRowSignature() + ).toList(); + List> readData = FrameTestUtil.readRowsFromCursor( + frameMaker.asCursor(), + frameStorageAdapter.getRowSignature() + ).toList(); + + Assert.assertEquals( + "Read rows count is different from written rows count", + writtenData.size(), + readData.size() ); - try (final CursorHolder frameMaker = frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { - List> readData = FrameTestUtil.readRowsFromCursor( - frameMaker.asCursor(), - frameStorageAdapter.getRowSignature() - ).toList(); - - Assert.assertEquals( - "Read rows count is different from written rows count", - writtenData.size(), - readData.size() - ); - Assert.assertEquals("Read data is different from written data", writtenData, readData); - } + Assert.assertEquals("Read data is different from written data", writtenData, readData); } } diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index e97f420867e9..34cdab25a48b 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -30,7 +30,6 @@ public class RowNumberUpdatingCursor implements Cursor { private final Cursor baseCursor; private final SettableLongVirtualColumn rowNumberVirtualColumn; - private long markRowNumber = 0; RowNumberUpdatingCursor(Cursor baseCursor, SettableLongVirtualColumn rowNumberVirtualColumn) { @@ -73,7 +72,6 @@ public boolean isDoneOrInterrupted() @Override public void reset() { - markRowNumber = 0; rowNumberVirtualColumn.setValue(0); baseCursor.reset(); } diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index 265bffa371b1..2ed9e8431501 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -42,7 +42,6 @@ public class ListCursor implements Cursor { private final List baseList; private int index; - private int markIndex = 0; public ListCursor(List inputList) { @@ -218,6 +217,5 @@ public boolean isDoneOrInterrupted() public void reset() { index = 0; - markIndex = 0; } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java index c2e7a4912575..ff4dc11f220f 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.filter.SelectorPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; @@ -110,8 +109,7 @@ public void testFloatProcessorMatchingValue() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Float.class, - ImmutableList.of(2.f), - DateTimes.nowUtc() + ImmutableList.of(2.f) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2.f").makeFloatProcessor(columnValueSelector); @@ -123,8 +121,7 @@ public void testFloatProcessorNotMatchingValue() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Float.class, - ImmutableList.of(2.f), - DateTimes.nowUtc() + ImmutableList.of(2.f) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5.f").makeFloatProcessor(columnValueSelector); @@ -136,8 +133,7 @@ public void testDoubleProcessorMatchingValue() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Double.class, - ImmutableList.of(2.), - DateTimes.nowUtc() + ImmutableList.of(2.) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2.").makeDoubleProcessor(columnValueSelector); @@ -149,8 +145,7 @@ public void testDoubleProcessorNotMatchingValue() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Double.class, - ImmutableList.of(2.), - DateTimes.nowUtc() + ImmutableList.of(2.) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5.").makeDoubleProcessor(columnValueSelector); @@ -187,8 +182,7 @@ public double doubleValue() { return num; } - }), - DateTimes.nowUtc() + }) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2").makeComplexProcessor(columnValueSelector); @@ -225,8 +219,7 @@ public double doubleValue() { return num; } - }), - DateTimes.nowUtc() + }) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5").makeComplexProcessor(columnValueSelector); @@ -238,8 +231,7 @@ public void testLongProcessorMatchingValue() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Long.class, - ImmutableList.of(2L), - DateTimes.nowUtc() + ImmutableList.of(2L) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2").makeLongProcessor(columnValueSelector); @@ -251,8 +243,7 @@ public void testLongProcessorNotMatchingValue() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Long.class, - ImmutableList.of(2L), - DateTimes.nowUtc() + ImmutableList.of(2L) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5").makeLongProcessor(columnValueSelector); @@ -264,8 +255,7 @@ public void testComplexProcessorMatchingNull() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - Arrays.asList(null, "v"), - DateTimes.nowUtc() + Arrays.asList(null, "v") ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector(null).makeComplexProcessor(columnValueSelector); @@ -277,8 +267,7 @@ public void testComplexProcessorEmptyString() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - Arrays.asList("", "v"), - DateTimes.nowUtc() + Arrays.asList("", "v") ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector(null).makeComplexProcessor(columnValueSelector); @@ -294,8 +283,7 @@ public void testComplexProcessorMatchingInteger() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Integer.class, - ImmutableList.of(11), - DateTimes.nowUtc() + ImmutableList.of(11) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); @@ -307,8 +295,7 @@ public void testComplexProcessorNotMatchingInteger() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Integer.class, - ImmutableList.of(15), - DateTimes.nowUtc() + ImmutableList.of(15) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); @@ -320,8 +307,7 @@ public void testComplexProcessorMatchingLong() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Long.class, - ImmutableList.of(11L), - DateTimes.nowUtc() + ImmutableList.of(11L) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); @@ -333,8 +319,7 @@ public void testComplexProcessorNotMatchingLong() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Long.class, - ImmutableList.of(15L), - DateTimes.nowUtc() + ImmutableList.of(15L) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); @@ -346,8 +331,7 @@ public void testComplexProcessorMatchingFloat() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Float.class, - ImmutableList.of(11.f), - DateTimes.nowUtc() + ImmutableList.of(11.f) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.f").makeComplexProcessor(columnValueSelector); @@ -359,8 +343,7 @@ public void testComplexProcessorNotMatchingFloat() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Float.class, - ImmutableList.of(15.f), - DateTimes.nowUtc() + ImmutableList.of(15.f) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.f").makeComplexProcessor(columnValueSelector); @@ -372,8 +355,7 @@ public void testComplexProcessorMatchingDouble() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Double.class, - ImmutableList.of(11.d), - DateTimes.nowUtc() + ImmutableList.of(11.d) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.d").makeComplexProcessor(columnValueSelector); @@ -385,8 +367,7 @@ public void testComplexProcessorNotMatchingDouble() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( Double.class, - ImmutableList.of(15.d), - DateTimes.nowUtc() + ImmutableList.of(15.d) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.d").makeComplexProcessor(columnValueSelector); @@ -398,8 +379,7 @@ public void testComplexProcessorMatchingString() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of("val"), - DateTimes.nowUtc() + ImmutableList.of("val") ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); @@ -411,8 +391,7 @@ public void testComplexProcessorNotMatchingString() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of("bar"), - DateTimes.nowUtc() + ImmutableList.of("bar") ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); @@ -424,8 +403,7 @@ public void testComplexProcessorMatchingStringList() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(ImmutableList.of("val")), - DateTimes.nowUtc() + ImmutableList.of(ImmutableList.of("val")) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); @@ -437,8 +415,7 @@ public void testComplexProcessorNotMatchingStringList() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(ImmutableList.of("bar")), - DateTimes.nowUtc() + ImmutableList.of(ImmutableList.of("bar")) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); @@ -450,8 +427,7 @@ public void testComplexProcessorMatchingEmptyList() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(ImmutableList.of()), - DateTimes.nowUtc() + ImmutableList.of(ImmutableList.of()) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector(null).makeComplexProcessor(columnValueSelector); @@ -463,8 +439,7 @@ public void testComplexProcessorMatchingBoolean() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(false), - DateTimes.nowUtc() + ImmutableList.of(false) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("false").makeComplexProcessor(columnValueSelector); @@ -476,8 +451,7 @@ public void testComplexProcessorNotMatchingBoolean() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(true), - DateTimes.nowUtc() + ImmutableList.of(true) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("false").makeComplexProcessor(columnValueSelector); @@ -489,8 +463,7 @@ public void testComplexProcessorMatchingByteArray() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(StringUtils.toUtf8("var")), - DateTimes.nowUtc() + ImmutableList.of(StringUtils.toUtf8("var")) ); columnValueSelector.advance(); final String base64Encoded = StringUtils.encodeBase64String(StringUtils.toUtf8("var")); @@ -503,8 +476,7 @@ public void testComplexProcessorNotMatchingByteArray() { final TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( String.class, - ImmutableList.of(StringUtils.toUtf8("var")), - DateTimes.nowUtc() + ImmutableList.of(StringUtils.toUtf8("var")) ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index a3e4d028c614..09f51634beae 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -26,7 +26,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Collection; @@ -38,28 +37,24 @@ public class TestColumnValueSelector implements ColumnValueSelector, { private final Class clazz; private final Supplier> iteratorSupplier; - private final DateTime time; private Iterator iterator; private Object value; - private int rowCounter = 0; - private int markCounter = 0; - public static TestColumnValueSelector of(Class clazz, Collection collection, DateTime time) + public static TestColumnValueSelector of(Class clazz, Collection collection) { - return new TestColumnValueSelector<>(clazz, collection::iterator, time); + return new TestColumnValueSelector<>(clazz, collection::iterator); } - public static TestColumnValueSelector of(Class clazz, Stream stream, DateTime time) + public static TestColumnValueSelector of(Class clazz, Stream stream) { - return new TestColumnValueSelector<>(clazz, stream::iterator, time); + return new TestColumnValueSelector<>(clazz, stream::iterator); } - protected TestColumnValueSelector(Class clazz, Supplier> iteratorSupplier, DateTime time) + protected TestColumnValueSelector(Class clazz, Supplier> iteratorSupplier) { this.clazz = clazz; this.iteratorSupplier = iteratorSupplier; - this.time = time; this.iterator = iteratorSupplier.get(); } @@ -93,7 +88,6 @@ public ColumnCapabilities getColumnCapabilities(String column) public void advance() { value = iterator.next(); - rowCounter++; } @Override @@ -117,8 +111,6 @@ public boolean isDoneOrInterrupted() @Override public void reset() { - rowCounter = 0; - markCounter = 0; iterator = iteratorSupplier.get(); } From 514f031ccb011f45efe45b28f67dd7963dd152e0 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 16:16:48 -0700 Subject: [PATCH 70/74] adjust --- .../druid/query/scan/ScanQueryEngine.java | 13 +---------- .../org/apache/druid/segment/Cursors.java | 23 +++++++++++-------- 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index ff70f722339e..893480f54532 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; @@ -121,17 +120,7 @@ public Sequence process( final long limit = calculateRemainingScanRowsLimit(query, responseContext); final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); if (Order.NONE != query.getTimeOrder()) { - final Order requiredOrder = query.getTimeOrder(); - if (!(Cursors.isTimeOrdered(cursorHolder) && cursorHolder.getOrdering().get(0).getOrder() == requiredOrder)) { - String failureReason = StringUtils.format( - "Cursor must be ordered by [%s] with direction [%s] but was [%s] instead.", - ColumnHolder.TIME_COLUMN_NAME, - requiredOrder, - cursorHolder.getOrdering() - ); - cursorHolder.close(); - throw DruidException.defensive(failureReason); - } + Cursors.requireTimeOrdering(cursorHolder, query.getTimeOrder()); } return new BaseSequence<>( new BaseSequence.IteratorMaker>() diff --git a/processing/src/main/java/org/apache/druid/segment/Cursors.java b/processing/src/main/java/org/apache/druid/segment/Cursors.java index eb65d0b61d72..814681ab8b47 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursors.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursors.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnHolder; @@ -54,15 +55,16 @@ public static boolean preferDescendingTimeOrdering(CursorBuildSpec buildSpec) /** * Check if the first {@link OrderBy} of {@link CursorHolder#getOrdering()} is {@link ColumnHolder#TIME_COLUMN_NAME} + * in the specified {@link Order} */ - public static boolean isTimeOrdered(CursorHolder holder) + public static boolean isTimeOrdered(CursorHolder holder, Order order) { final List ordering = holder.getOrdering(); if (ordering.isEmpty()) { return false; } final OrderBy orderBy = ordering.get(0); - return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()); + return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()) && orderBy.getOrder() == order; } /** @@ -71,13 +73,16 @@ public static boolean isTimeOrdered(CursorHolder holder) */ public static void requireTimeOrdering(CursorHolder holder, Order order) { - DruidException.conditionalDefensive( - isTimeOrdered(holder) && holder.getOrdering().get(0).getOrder() == order, - "Cursor must be ordered by [%s] with direction [%s] but was [%s] instead.", - ColumnHolder.TIME_COLUMN_NAME, - order, - holder.getOrdering() - ); + if (!isTimeOrdered(holder, order)) { + final String failureReason = StringUtils.format( + "Cursor must be ordered by [%s] with direction [%s] but was [%s] instead.", + ColumnHolder.TIME_COLUMN_NAME, + order, + holder.getOrdering() + ); + holder.close(); + throw DruidException.defensive(failureReason); + } } /** From 1594f73e5437187c28c168aa406e9893ad77bf83 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 20:10:21 -0700 Subject: [PATCH 71/74] safer timeseries, minor adjustments --- .../druid/math/expr/ExprMacroTable.java | 5 ++++ .../org/apache/druid/query/QueryContexts.java | 5 ---- .../druid/query/groupby/GroupingEngine.java | 4 +-- .../timeseries/TimeseriesQueryEngine.java | 28 +++++++++++-------- .../AggregateTopNMetricFirstAlgorithm.java | 2 +- 5 files changed, 25 insertions(+), 19 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java b/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java index 34bf16538d81..699eb8967eb8 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java @@ -73,6 +73,11 @@ public static ExprMacroTable nil() return NIL; } + /** + * Specialized {@link ExprMacroTable} that only knows about {@link TimestampFloorExprMacro}, intended for use + * parsing generated expressions which translate {@link org.apache.druid.java.util.common.granularity.Granularity} + * into {@link Expr} + */ public static ExprMacroTable granularity() { return TIME_FLOOR_MACRO_TABLE; diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 73288566eb8c..8e4418c2c937 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -180,11 +180,6 @@ public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) public abstract boolean shouldVectorize(boolean canVectorize); - public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) - { - return shouldVectorize(canVectorize); - } - @JsonCreator public static Vectorize fromString(String str) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 6a6b935dd86c..aefecf8c6837 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -503,12 +503,12 @@ public Sequence process( ? null : DateTimes.utc(Long.parseLong(fudgeTimestampString)); - // group by specific vectorization check: - final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, groupByQueryMetrics); final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); + + // group by specific vectorization check final boolean canVectorize = cursorHolder.canVectorize() && VectorGroupByEngine.canVectorizeDimensions(inspector, query.getDimensions()); final boolean shouldVectorize = query.context().getVectorize().shouldVectorize(canVectorize); diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index 04cf153048cb..8588128b877d 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -109,19 +109,25 @@ public Sequence> process( final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); Cursors.requireTimeOrdering(cursorHolder, query.isDescending() ? Order.DESCENDING : Order.ASCENDING); - final Sequence> result; + try { + final Sequence> result; - if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize(), cursorHolder::close)) { - result = processVectorized(query, adapter, cursorHolder, interval, gran); - } else { - result = processNonVectorized(query, adapter, cursorHolder, interval, gran); - } + if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize())) { + result = processVectorized(query, adapter, cursorHolder, interval, gran); + } else { + result = processNonVectorized(query, adapter, cursorHolder, interval, gran); + } - final int limit = query.getLimit(); - if (limit < Integer.MAX_VALUE) { - return result.limit(limit).withBaggage(cursorHolder); - } else { - return result.withBaggage(cursorHolder); + final int limit = query.getLimit(); + if (limit < Integer.MAX_VALUE) { + return result.limit(limit).withBaggage(cursorHolder); + } else { + return result.withBaggage(cursorHolder); + } + } + catch (Throwable t) { + cursorHolder.close(); + throw t; } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 9c71a3842292..99759f515dd7 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -112,7 +112,7 @@ public void run( PooledTopNAlgorithm allMetricAlgo = new PooledTopNAlgorithm(storageAdapter, query, bufferPool); PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null; try { - // reset cursor to starting position (mark was set by the call to run) + // reset cursor since we call run again params.getCursor().reset(); // Run topN for all metrics for top N dimension values allMetricsParam = allMetricAlgo.makeInitParams(params.getSelectorPlus(), params.getCursor(), params.getGranularizer()); From 9b37926b90439a40a3964327b4545fb151aeaeb0 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 21:27:14 -0700 Subject: [PATCH 72/74] oops --- .../org/apache/druid/query/QueryContexts.java | 23 ------------------- 1 file changed, 23 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 8e4418c2c937..d3545a8494ae 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -135,12 +135,6 @@ public boolean shouldVectorize(final boolean canVectorize) { return false; } - - @Override - public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) - { - return false; - } }, TRUE { @Override @@ -148,12 +142,6 @@ public boolean shouldVectorize(final boolean canVectorize) { return canVectorize; } - - @Override - public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) - { - return canVectorize; - } }, FORCE { @Override @@ -165,17 +153,6 @@ public boolean shouldVectorize(final boolean canVectorize) return true; } - - @Override - public boolean shouldVectorize(boolean canVectorize, Runnable cleanup) - { - if (!canVectorize) { - cleanup.run(); - throw new ISE("Cannot vectorize!"); - } - - return true; - } }; public abstract boolean shouldVectorize(boolean canVectorize); From 24406dfb7be8d82d86561b743dddde76f41cab90 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 23:48:45 -0700 Subject: [PATCH 73/74] nicer cursor build spec granularity decoration --- .../common/granularity/Granularities.java | 56 +++++++++++++++++-- .../druid/query/groupby/GroupingEngine.java | 44 ++++----------- .../timeseries/TimeseriesQueryEngine.java | 55 +++++------------- .../druid/query/topn/TopNQueryEngine.java | 49 ++++------------ .../druid/query/groupby/GroupByQueryTest.java | 2 +- .../query/timeseries/TimeseriesQueryTest.java | 2 +- .../druid/query/topn/TopNQueryTest.java | 2 +- 7 files changed, 93 insertions(+), 117 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java index 82819c85d5f8..3c8c89b2c265 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java @@ -19,16 +19,24 @@ package org.apache.druid.java.util.common.granularity; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Query; import org.apache.druid.query.expression.TimestampFloorExprMacro; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.joda.time.chrono.ISOChronology; import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Collections; import java.util.Set; /** @@ -61,10 +69,27 @@ public static Granularity nullToAll(Granularity granularity) return granularity == null ? Granularities.ALL : granularity; } - @Nullable - public static ExpressionVirtualColumn toVirtualColumn(Query query) + + /** + * Decorates {@link CursorBuildSpec} with a grouping column and virtual column equivalent to the {@link Granularity} + * for a {@link Query}, if that query has granularity other than {@link Granularities#ALL}. If the query has 'ALL' + * granularity, the {@link CursorBuildSpec} will be returned as is. + * The reason for this is that a query with a granularity that is not 'ALL' is effectively an additional grouping on + * TIME_FLOOR(__time, granularity), so by adding this to the {@link CursorBuildSpec} allows a + * {@link org.apache.druid.segment.CursorHolder} to potentially specialize any {@link org.apache.druid.segment.Cursor} + * or {@link org.apache.druid.segment.vector.VectorCursor} it provides by using pre-aggregated data at the closest + * matching granularity if available. + * + * @see #toVirtualColumn(Granularity, String) for how {@link Granularity} is translated into a {@link VirtualColumn} + */ + public static CursorBuildSpec decorateCursorBuildSpec(Query query, CursorBuildSpec buildSpec) { - ImmutableSet.Builder builder = ImmutableSet.builder(); + // ALL granularity + if (ALL.equals(query.getGranularity())) { + return buildSpec; + } + // For any other granularity, we add a grouping on TIME_FLOOR(__time, granularity) or __time itself + final ImmutableSet.Builder builder = ImmutableSet.builder(); final Set requiredColumns = query.getRequiredColumns(); if (requiredColumns != null) { builder.addAll(requiredColumns); @@ -76,7 +101,30 @@ public static ExpressionVirtualColumn toVirtualColumn(Query query) while (columnNamesForConflictResolution.contains(virtualColumnName)) { virtualColumnName = virtualColumnName + ctr++; } - return toVirtualColumn(query.getGranularity(), virtualColumnName); + + final VirtualColumn granularityVirtual = toVirtualColumn(query.getGranularity(), virtualColumnName); + // granularityVirtual will not be null because we have already filtered out ALL granularity, but check just in case + Preconditions.checkNotNull( + granularityVirtual, + "Granularity virtual column is null for granularity[%s]", + query.getGranularity() + ); + + final VirtualColumns virtualColumns = VirtualColumns.fromIterable( + Iterables.concat( + Collections.singletonList(granularityVirtual), + () -> Arrays.stream(buildSpec.getVirtualColumns().getVirtualColumns()).iterator() + ) + ); + final ImmutableList.Builder groupingColumnsBuilder = ImmutableList.builder(); + groupingColumnsBuilder.add(granularityVirtual.getOutputName()); + if (buildSpec.getGroupingColumns() != null) { + groupingColumnsBuilder.addAll(buildSpec.getGroupingColumns()); + } + return CursorBuildSpec.builder(buildSpec) + .setVirtualColumns(virtualColumns) + .setGroupingColumns(groupingColumnsBuilder.build()) + .build(); } /** diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index aefecf8c6837..75d7ad50d8af 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -80,7 +80,6 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.StorageAdapter; -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.ColumnType; @@ -95,7 +94,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; @@ -849,36 +847,18 @@ private Set getAggregatorAndPostAggregatorNames(GroupByQuery query) public static CursorBuildSpec makeCursorBuildSpec(GroupByQuery query, @Nullable QueryMetrics queryMetrics) { - // virtual column is currently only used as a decorator to pass to the cursor holder to allow specializing cursor - // and vector cursors if any pre-aggregated data at the matching granularity is available - // eventually this could probably be reworked to be used by the granularizer instead of the existing method - // of creating a selector on the time column - final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query); - VirtualColumns virtualColumns; - List groupingColumns; - if (granularityVirtual == null) { - virtualColumns = query.getVirtualColumns(); - groupingColumns = query.getGroupingColumns(); - } else { - virtualColumns = VirtualColumns.fromIterable( - Iterables.concat( - Collections.singletonList(granularityVirtual), - () -> Arrays.stream(query.getVirtualColumns().getVirtualColumns()).iterator() - ) - ); - groupingColumns = ImmutableList.builder() - .add(granularityVirtual.getOutputName()) - .addAll(query.getGroupingColumns()).build(); - } - return CursorBuildSpec.builder() - .setInterval(query.getSingleInterval()) - .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) - .setVirtualColumns(virtualColumns) - .setGroupingColumns(groupingColumns) - .setAggregators(query.getAggregatorSpecs()) - .setQueryContext(query.context()) - .setQueryMetrics(queryMetrics) - .build(); + return Granularities.decorateCursorBuildSpec( + query, + CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setVirtualColumns(query.getVirtualColumns()) + .setGroupingColumns(query.getGroupingColumns()) + .setAggregators(query.getAggregatorSpecs()) + .setQueryContext(query.context()) + .setQueryMetrics(queryMetrics) + .build() + ); } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index 8588128b877d..e23f8e19e6df 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.Order; -import org.apache.druid.query.OrderBy; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; @@ -48,9 +47,6 @@ import org.apache.druid.segment.Cursors; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; @@ -58,8 +54,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -329,40 +323,19 @@ private Sequence> processNonVectorized( public static CursorBuildSpec makeCursorBuildSpec(TimeseriesQuery query, @Nullable QueryMetrics queryMetrics) { - // virtual column is currently only used as a decorator to pass to the cursor holder to allow specializing cursor - // and vector cursors if any pre-aggregated data at the matching granularity is available - // eventually this could probably be reworked to be used by the granularizer instead of the existing method - // of creating a selector on the time column - final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query); - VirtualColumns virtualColumns; - List groupingColumns; - if (granularityVirtual == null) { - virtualColumns = query.getVirtualColumns(); - groupingColumns = null; - } else { - virtualColumns = VirtualColumns.fromIterable( - Iterables.concat( - Collections.singletonList(granularityVirtual), - () -> Arrays.stream(query.getVirtualColumns().getVirtualColumns()).iterator() - ) - ); - groupingColumns = Collections.singletonList(granularityVirtual.getOutputName()); - } - return CursorBuildSpec.builder() - .setInterval(query.getSingleInterval()) - .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) - .setGroupingColumns(groupingColumns) - .setVirtualColumns(virtualColumns) - .setAggregators(query.getAggregatorSpecs()) - .setQueryContext(query.context()) - .setPreferredOrdering( - Collections.singletonList( - query.isDescending() ? - OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME) : - OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME) - ) - ) - .setQueryMetrics(queryMetrics) - .build(); + return Granularities.decorateCursorBuildSpec( + query, + CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setVirtualColumns(query.getVirtualColumns()) + .setAggregators(query.getAggregatorSpecs()) + .setQueryContext(query.context()) + .setPreferredOrdering( + query.isDescending() ? Cursors.descendingTimeOrder() : Cursors.ascendingTimeOrder() + ) + .setQueryMetrics(queryMetrics) + .build() + ); } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index af98381fe733..e1f1ad0e8dda 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -20,8 +20,6 @@ package org.apache.druid.query.topn; import com.google.common.base.Predicates; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.granularity.Granularities; @@ -37,8 +35,6 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; -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.Types; @@ -47,9 +43,7 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.Collections; -import java.util.List; /** */ @@ -242,37 +236,18 @@ private static boolean shouldUseAggregateMetricFirstAlgorithm(TopNQuery query, T public static CursorBuildSpec makeCursorBuildSpec(TopNQuery query, @Nullable QueryMetrics queryMetrics) { - // virtual column is currently only used as a decorator to pass to the cursor holder to allow specializing cursor - // and vector cursors if any pre-aggregated data at the matching granularity is available - // eventually this could probably be reworked to be used by the granularizer instead of the existing method - // of creating a selector on the time column - final VirtualColumn granularityVirtual = Granularities.toVirtualColumn(query); - VirtualColumns virtualColumns; - List groupingColumns; - if (granularityVirtual == null) { - virtualColumns = query.getVirtualColumns(); - groupingColumns = Collections.singletonList(query.getDimensionSpec().getDimension()); - } else { - virtualColumns = VirtualColumns.fromIterable( - Iterables.concat( - Collections.singletonList(granularityVirtual), - () -> Arrays.stream(query.getVirtualColumns().getVirtualColumns()).iterator() - ) - ); - groupingColumns = ImmutableList.of( - granularityVirtual.getOutputName(), - query.getDimensionSpec().getDimension() - ); - } - return CursorBuildSpec.builder() - .setInterval(query.getSingleInterval()) - .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) - .setGroupingColumns(groupingColumns) - .setVirtualColumns(virtualColumns) - .setAggregators(query.getAggregatorSpecs()) - .setQueryContext(query.context()) - .setQueryMetrics(queryMetrics) - .build(); + return Granularities.decorateCursorBuildSpec( + query, + CursorBuildSpec.builder() + .setInterval(query.getSingleInterval()) + .setFilter(Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()))) + .setGroupingColumns(Collections.singletonList(query.getDimensionSpec().getDimension())) + .setVirtualColumns(query.getVirtualColumns()) + .setAggregators(query.getAggregatorSpecs()) + .setQueryContext(query.context()) + .setQueryMetrics(queryMetrics) + .build() + ); } /** diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java index 07205872089d..e83eac25e30e 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java @@ -252,7 +252,7 @@ public void testAsCursorBuildSpecDayGranularity() Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); Assert.assertEquals( VirtualColumns.create( - Granularities.toVirtualColumn(query), + Granularities.toVirtualColumn(query.getGranularity(), Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME), virtualColumns.getVirtualColumns()[0] ), buildSpec.getVirtualColumns() diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java index 40e2f6ec9499..c1f8a144276c 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java @@ -196,7 +196,7 @@ public void testAsCursorBuildSpecDayGranularity() ); Assert.assertEquals( VirtualColumns.create( - Granularities.toVirtualColumn(query), + Granularities.toVirtualColumn(query.getGranularity(), Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME), virtualColumns.getVirtualColumns()[0] ), buildSpec.getVirtualColumns() diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java index 39371a8747a2..1fd483eabe0c 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java @@ -325,7 +325,7 @@ public void testAsCursorBuildSpecDayGranularity() Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators()); Assert.assertEquals( VirtualColumns.create( - Granularities.toVirtualColumn(query), + Granularities.toVirtualColumn(query.getGranularity(), Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME), virtualColumns.getVirtualColumns()[0] ), buildSpec.getVirtualColumns() From 3200af97f2c3a0621ff1140b3785635d5dcfabd3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 14 Aug 2024 23:53:54 -0700 Subject: [PATCH 74/74] missed a spot --- .../main/java/org/apache/druid/query/topn/TopNQueryEngine.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index e1f1ad0e8dda..ac130d3fa860 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -216,9 +216,8 @@ private static boolean canUsePooledAlgorithm( // need to make multiple passes (e.g. reset the cursor) try (final ResourceHolder resultsBufHolder = bufferPool.take()) { final ByteBuffer resultsBuf = resultsBufHolder.get(); - resultsBuf.clear(); - final int numBytesToWorkWith = resultsBuf.remaining(); + final int numBytesToWorkWith = resultsBuf.capacity(); final int numValuesPerPass = numBytesPerRecord > 0 ? numBytesToWorkWith / numBytesPerRecord : cardinality; return numValuesPerPass <= cardinality;