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..d65d13a324d8 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; @@ -35,9 +33,10 @@ 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.CursorHolder; 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; @@ -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; @@ -165,32 +163,22 @@ 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 - ); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); - 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 e7d8ad00bc5e..06f0952b51f0 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; @@ -33,9 +32,10 @@ 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.CursorHolder; 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; @@ -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) @@ -146,52 +145,34 @@ 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 List results = cursors - .map(cursor -> { - final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); - consumeString(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(expressionFilter.toFilter()) + .build(); + 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()); + cursor.advance(); + } + } } @Benchmark public void nativeFilter(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - nativeFilter.toFilter(), - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - 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(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(nativeFilter.toFilter()) + .build(); + 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()); + 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 dd33b6606440..5bc67bc9f729 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; @@ -35,9 +37,12 @@ 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.CursorHolder; 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; @@ -62,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) @@ -141,388 +146,281 @@ 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 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "timestamp_floor(__time, 'PT1H')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + 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); + } } @Benchmark public void timeFloorUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - 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); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + + 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).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.HOUR, - false, - null - ); + final StorageAdapter adapter = new QueryableIndexStorageAdapter(index); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + final CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + Granularities.HOUR, + adapter.getInterval(), + false + ); + 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; + } - 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; + blackhole.consume(count); } - - blackhole.consume(count); } @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 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "timestamp_format(__time, 'yyyy-MM-dd')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + 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).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - 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); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + 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).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "v", - "strlen(s)", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ) - ) - ), - Granularities.ALL, - false, - null - ); - - 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "strlen(s)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + 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); + } } @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 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "strlen(s)", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + 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)); + + consumeDimension(cursor, selector, blackhole); + } } @Benchmark public void strlenUsingExtractionFn(Blackhole blackhole) { - final Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - final List results = cursors - .map(cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new ExtractionDimensionSpec("x", "v", StrlenExtractionFn.instance())); + try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new ExtractionDimensionSpec("x", "v", StrlenExtractionFn.instance())); - consumeDimension(cursor, selector, blackhole); - return null; - }) - .toList(); - - blackhole.consume(results); + consumeDimension(cursor, selector, 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 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "n + 1", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + 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); + } } @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 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "concat(n, ' is my favorite number') == '3 is my favorite number'", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + 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); + } } @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 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() + .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 CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, 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 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() + .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 CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + 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( @@ -534,115 +432,90 @@ 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 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() + .setVirtualColumns( + VirtualColumns.create( + new ExpressionVirtualColumn( + "v", + "case_searched(s == 'asd' || isnull(s) || n == 1, 1, " + caseBranches + " 3)", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + ) + .build(); + + 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); + } } @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 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() + .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 CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + consumeLong(cursor, selector, 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 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() + .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 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 754eb7b6d381..5604f7fe8834 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; @@ -33,6 +32,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.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -150,15 +151,14 @@ public void scan(Blackhole blackhole) ) ) ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(virtualColumns) + .build(); + final CursorHolder cursorHolder = closer.register( + new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec) + ); if (vectorize) { - VectorCursor cursor = new QueryableIndexStorageAdapter(index).makeVectorCursor( - null, - index.getDataInterval(), - virtualColumns, - false, - 512, - null - ); + VectorCursor cursor = cursorHolder.asVectorCursor(); if (outputType.isNumeric()) { VectorValueSelector selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); if (outputType.is(ExprType.DOUBLE)) { @@ -174,29 +174,34 @@ public void scan(Blackhole blackhole) cursor.advance(); } } - closer.register(cursor); } } else { - Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - virtualColumns, - Granularities.ALL, - false, - null - ); - - 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 = cursorHolder.asCursor(); + 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 98c7ddf4f856..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,9 +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.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; @@ -48,6 +45,8 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -55,7 +54,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; @@ -234,8 +232,10 @@ private IncrementalIndex makeIncIndex() public void stringRead(Blackhole blackhole) { StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, null); - readCursors(cursors, blackhole); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -244,9 +244,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 CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -255,9 +256,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 (CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterNone)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -266,9 +268,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 CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterHalf)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -277,9 +280,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 CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterAll)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursorLong(cursor, blackhole); + } } @Benchmark @@ -290,8 +294,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 CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -302,8 +308,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 CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -314,8 +322,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 CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -326,8 +336,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 CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -343,8 +355,10 @@ public void readAndFilter(Blackhole blackhole) ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, andFilter); - readCursors(cursors, blackhole); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, andFilter)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -357,8 +371,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 CursorHolder cursorHolder = makeCursorHolder(sa, orFilter)) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -371,8 +387,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 CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(orFilter))) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -408,8 +426,10 @@ public void readComplexOrFilter(Blackhole blackhole) ); StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - Sequence cursors = makeCursors(sa, dimFilter3.toFilter()); - readCursors(cursors, blackhole); + try (final CursorHolder cursorHolder = makeCursorHolder(sa, dimFilter3.toFilter())) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } @Benchmark @@ -445,55 +465,46 @@ 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 CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(dimFilter3.toFilter()))) { + final Cursor cursor = cursorHolder.asCursor(); + readCursor(cursor, blackhole); + } } - private Sequence makeCursors(StorageAdapter sa, Filter filter) + private CursorHolder makeCursorHolder(StorageAdapter sa, Filter filter) { - return sa.makeCursors(filter, schemaInfo.getDataInterval(), VirtualColumns.EMPTY, Granularities.ALL, false, null); + return sa.makeCursorHolder( + CursorBuildSpec.builder() + .setFilter(filter) + .setInterval(schemaInfo.getDataInterval()) + .build() + ); } - 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 c2dd8ae9996a..952e9b188fe9 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; @@ -37,6 +36,8 @@ 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.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -216,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 CursorHolder cursorHolder = makeCursorHolder()) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = processRowsValueSelector(blackhole, cursor, projectionColumns); + blackhole.consume(rowCount); + } } @Benchmark @@ -226,21 +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 CursorHolder cursorHolder = makeCursorHolder()) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = processRowsDimensionSelectors(blackhole, cursor, projectionColumns); + blackhole.consume(rowCount); + } } - private Sequence makeCursors() + private CursorHolder makeCursorHolder() { - return hashJoinSegment.asStorageAdapter().makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + return hashJoinSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN); } @@ -313,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))); } - return rowCount; - }).accumulate(0, (acc, in) -> acc + in); + } 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(); + } + 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 ba07091fe5ae..c66ef6baf0a7 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,18 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -334,38 +334,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 @@ -373,16 +369,10 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, "countryIsoCode")); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "countryIsoCode")); + } } @Benchmark @@ -390,16 +380,14 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, "countryIsoCode")); + final Filter filter = new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .build(); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "countryIsoCode")); + } } @Benchmark @@ -407,16 +395,11 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, "c.v")); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -424,16 +407,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 - ); - - blackhole.consume(getLastValue(cursors, "c.v")); + final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .build(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -441,16 +423,11 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, "c.v")); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -458,16 +435,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 - ); - - blackhole.consume(getLastValue(cursors, "c.v")); + final Filter filter = new SelectorDimFilter("c.v", "Canada", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .build(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.v")); + } } @Benchmark @@ -475,16 +451,11 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -492,16 +463,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 - ); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .build(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -509,16 +479,11 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -526,16 +491,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 - ); - - blackhole.consume(getLastValue(cursors, "c.countryName")); + final Filter filter = new SelectorDimFilter("c.countryName", "Canada", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .build(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, "c.countryName")); + } } @Benchmark @@ -543,16 +507,14 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .build(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); + } } @Benchmark @@ -560,16 +522,16 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_CODE_TO_NAME)); + final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_CODE_TO_NAME, "Canada", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(lookupVirtualColumns) + .build(); + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + .makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); + } } @Benchmark @@ -577,16 +539,13 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .build(); + try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + } } @Benchmark @@ -594,15 +553,14 @@ 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 - ); - - blackhole.consume(getLastValue(cursors, LOOKUP_COUNTRY_NUMBER_TO_NAME)); + final Filter filter = new SelectorDimFilter(LOOKUP_COUNTRY_NUMBER_TO_NAME, "Canada", null).toFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(lookupVirtualColumns) + .setFilter(filter) + .build(); + 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 379885983241..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,8 +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.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; @@ -36,8 +34,9 @@ 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.CursorHolder; 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; @@ -149,22 +148,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 CursorHolder cursorHolder = makeCursor(sa, null)) { + Cursor cursor = cursorHolder.asCursor(); + + 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,35 +184,34 @@ 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 CursorHolder cursorHolder = makeCursor(sa, filter)) { + Cursor cursor = cursorHolder.asCursor(); + + 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 CursorHolder makeCursor(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() + .setInterval(schemaInfo.getDataInterval()); + if (filter != null) { + builder.setFilter(filter.toFilter()); + } + return sa.makeCursorHolder(builder.build()); } private static DimensionSelector makeDimensionSelector(Cursor cursor, String name) 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..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 @@ -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", "force"}) 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/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index a33dbc0dcfc3..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; @@ -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/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; 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/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/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..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 @@ -19,15 +19,18 @@ 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.CursorHolder; 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; @@ -54,7 +57,8 @@ public void setUp() @Test public void testRead() { - try (final VectorCursor cursor = makeCursor()) { + try (final CursorHolder cursorHolder = makeCursorHolder()) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); final Supplier qualitySupplier = ColumnProcessors.makeVectorProcessor( "quality", ToObjectVectorColumnProcessorFactory.INSTANCE, @@ -167,21 +171,22 @@ public void testComplexSketch() Assert.assertThat(sketch, CoreMatchers.instanceOf(HyperLogLogCollector.class)); } - private VectorCursor makeCursor() + private CursorHolder makeCursorHolder() { - 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.makeCursorHolder(buildSpec); } private List readColumn(final String column, final int limit) { - try (final VectorCursor cursor = makeCursor()) { + 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/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index efc2cbb2afbd..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,7 +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.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -377,9 +378,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/indexing/processor/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index 78b3e16f6702..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 @@ -38,17 +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.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; 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.VirtualColumns; +import org.apache.druid.segment.CursorBuildSpec; +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; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -182,42 +181,41 @@ 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) - .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); - - 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 CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return; + } + + 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 ccedf0402c6e..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 @@ -33,15 +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.granularity.Granularities; -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.VirtualColumns; +import org.apache.druid.segment.CursorBuildSpec; +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; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -152,57 +151,58 @@ 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); - - 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 CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + exportWriter.writeRowEnd(); + return; + } + 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 2104d1d40f41..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 @@ -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,10 +40,8 @@ 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.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,13 +59,13 @@ 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.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.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.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; @@ -77,9 +74,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; @@ -180,7 +175,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) @@ -252,21 +247,24 @@ 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." + ); + } + + final CursorHolder cursorHolder = closer.register( + adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); + final Cursor nextCursor = cursorHolder.asCursor(); - if (cursorYielder.isDone()) { + if (nextCursor == null) { // No cursors! - cursorYielder.close(); return ReturnOrAwait.returnObject(Unit.instance()); } else { - final long rowsFlushed = setNextCursor(cursorYielder.get(), segmentHolder.get()); + final long rowsFlushed = setNextCursor(nextCursor, segmentHolder.get()); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment - closer.register(cursorYielder); } } @@ -294,15 +292,23 @@ 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 CursorHolder cursorHolder = closer.register( + adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); + final Cursor nextCursor = cursorHolder.asCursor(); + + if (nextCursor == null) { + // no cursor + return ReturnOrAwait.returnObject(Unit.instance()); + } + final long rowsFlushed = setNextCursor(nextCursor, frameSegment); if (rowsFlushed > 0) { return ReturnOrAwait.runAgain(); @@ -430,27 +436,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." - ); - } - - 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 - ); - } } 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 2a90616fe1db..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,6 +37,8 @@ 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.ScanQuery; import org.apache.druid.segment.column.ColumnType; @@ -119,11 +121,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..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.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -862,7 +862,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(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 ab06f851af06..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,6 +33,7 @@ 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; @@ -189,7 +190,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(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 c6d67f14428f..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; @@ -463,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 ScanQuery.OrderBy("dim1", ScanQuery.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/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 3fbd8c72049c..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; @@ -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) 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 d868ddf20e52..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,6 +48,7 @@ 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; @@ -303,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 ScanQuery.OrderBy( - col, - ScanQuery.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/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..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; @@ -44,10 +43,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.CursorHolder; 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 +214,98 @@ 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()) + .build(); + final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec); + final Cursor cursor = cursorHolder.asCursor(); + 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(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 42618556d003..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 @@ -33,7 +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.granularity.Granularities; +import org.apache.druid.java.util.common.CloseableIterators; 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; @@ -48,10 +48,11 @@ 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.CursorHolder; 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 +124,16 @@ protected CloseableIterator> intermediateRowIterator() throw ), intervalFilter ); - - final Sequence cursors = storageAdapter.getAdapter().makeCursors( - Filters.toFilter(dimFilter), - storageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() + .setFilter(Filters.toFilter(dimFilter)) + .setInterval(storageAdapter.getInterval()) + .build(); + + final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec); + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return CloseableIterators.wrap(Collections.emptyIterator(), cursorHolder); + } // Retain order of columns from the original segments. Useful for preserving dimension order if we're in // schemaless mode. @@ -142,12 +144,7 @@ protected CloseableIterator> intermediateRowIterator() throw ) ); - final Sequence> sequence = Sequences.concat( - Sequences.map( - cursors, - cursor -> cursorToSequence(cursor, columnsToRead) - ) - ); + 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 7ebde700bee2..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 @@ -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; @@ -72,12 +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.CursorHolder; 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.TestIndex; -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; @@ -1702,23 +1702,19 @@ 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 - ); - - cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertNotNull(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")); @@ -1734,9 +1730,7 @@ public void testRunWithSpatialDimensions() throws Exception cursor.advance(); } - - return accumulated; - }); + } } Assert.assertEquals(spatialrows, rowsFromSegment); } @@ -1836,16 +1830,9 @@ 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 - ); - - cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertNotNull(cursor); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); Assert.assertEquals(ColumnType.STRING, factory.getColumnCapabilities("ts").toColumnType()); @@ -1873,9 +1860,7 @@ public void testRunWithAutoCastDimensions() throws Exception cursor.advance(); } - - return accumulated; - }); + } } Assert.assertEquals(rows, rowsFromSegment); } @@ -2070,16 +2055,9 @@ 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 - ); - - cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { + try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertNotNull(cursor); cursor.reset(); while (!cursor.isDone()) { final DimensionSelector selector1 = cursor.getColumnSelectorFactory() @@ -2107,9 +2085,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 3ec5b8d0aa8a..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 @@ -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; @@ -68,6 +67,8 @@ 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.CursorHolder; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; @@ -75,7 +76,6 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestIndex; -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; @@ -540,71 +540,65 @@ 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 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 CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + 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 @@ -776,31 +770,26 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception segment.getInterval() ); - final Sequence cursorSequence = adapter.getAdapter().makeCursors( - null, - segment.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - 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 CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + final List hashes = new ArrayList<>(); + final DimensionSelector selector = cursor.getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); + 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(); + } - Assert.assertTrue(hashes.stream().allMatch(h -> h.intValue() == hashes.get(0))); + Assert.assertTrue(hashes.stream().allMatch(h -> h.intValue() == hashes.get(0))); + } } } 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..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#makeCursorFactory} + * 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#makeCursorFactory} + * 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 2e21bbc312bf..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 @@ -32,9 +32,8 @@ 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.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,13 +114,16 @@ public static FrameCursor makeCursor( final VirtualColumns virtualColumns ) { - // 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) - ).get(); + final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() + .setVirtualColumns(virtualColumns) + .build(); + // 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 + // 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) + .asCursor(); } /** 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 46a848fb6b15..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,10 +29,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.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.CursorFactory; +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 +137,15 @@ public ColumnCapabilities columnCapabilities(final Frame frame, final String col } /** - * Create a {@link CursorFactory} for the given frame. + * Create a {@link CursorHolderFactory} for the given frame. */ - public CursorFactory makeCursorFactory(final Frame frame) + public CursorHolderFactory makeCursorHolderFactory(final Frame frame) { switch (frame.type()) { case COLUMNAR: - return new org.apache.druid.frame.segment.columnar.FrameCursorFactory(frame, signature, columnReaders); + return new ColumnarFrameCursorHolderFactory(frame, signature, columnReaders); case ROW_BASED: - return new FrameCursorFactory(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 3d4160a7abb4..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 @@ -19,16 +19,16 @@ package org.apache.druid.frame.segment; -import org.apache.druid.java.util.common.DateTimes; +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; -import org.joda.time.DateTime; /** - * 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 RowFrameCursorHolderFactory} + * and {@link ColumnarFrameCursorHolderFactory}. * * Adds the methods {@link #getCurrentRow()} and {@link #setCurrentRow(int)} so the cursor can be moved to * particular rows. @@ -53,12 +53,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 9733c548d422..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 @@ -21,21 +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.CursorFactory; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.CursorHolderFactory; 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; @@ -52,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 CursorHolderFactory 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.makeCursorHolderFactory(frame); } @Override @@ -147,49 +142,8 @@ public Metadata getMetadata() } @Override - public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - 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 cursorFactory.makeCursors( - 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 cursorFactory.makeVectorCursor( - filter, - interval, - virtualColumns, - descending, - vectorSize, - queryMetrics - ); + return cursorFactory.makeCursorHolder(spec); } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java new file mode 100644 index 000000000000..7c97530e37f1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.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.frame.segment.columnar; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.read.columnar.FrameColumnReader; +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; +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.QueryableIndexColumnSelectorFactory; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.SimpleDescendingOffset; +import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.vector.FilteredVectorOffset; +import org.apache.druid.segment.vector.NoFilterVectorOffset; +import org.apache.druid.segment.vector.QueryableIndexVectorColumnSelectorFactory; +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 javax.annotation.Nullable; +import java.util.List; + +/** + * 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 RowFrameCursorHolderFactory the row-based version + */ +public class ColumnarFrameCursorHolderFactory implements CursorHolderFactory +{ + private final Frame frame; + private final RowSignature signature; + private final List columnReaders; + + public ColumnarFrameCursorHolderFactory( + final Frame frame, + final RowSignature signature, + final List columnReaders + ) + { + this.frame = FrameType.COLUMNAR.ensureType(frame); + this.signature = signature; + this.columnReaders = columnReaders; + } + + @Override + 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 + public boolean canVectorize() + { + return (spec.getFilter() == null || spec.getFilter().canVectorizeMatcher(signature)) + && spec.getVirtualColumns().canVectorize(signature) + && !descending; + } + + @Override + 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 = Cursors.preferDescendingTimeOrdering(spec); + final SimpleSettableOffset baseOffset = descendingTimeOrder + ? new SimpleDescendingOffset(frame.numRows()) + : new SimpleAscendingOffset(frame.numRows()); + + final QueryableIndexColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( + spec.getVirtualColumns(), + descendingTimeOrder, + baseOffset, + columnCache + ); + + final SimpleSettableOffset offset; + if (filterToUse == null) { + offset = baseOffset; + } else { + offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse); + } + + return new FrameCursor(offset, columnSelectorFactory); + } + + @Override + public List getOrdering() + { + return ordering; + } + + @Nullable + @Override + public VectorCursor asVectorCursor() + { + if (!canVectorize()) { + throw new ISE("Cannot vectorize. Check 'canVectorize' before calling 'asVectorCursor'."); + } + + 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); + + // baseSelectorFactory using baseOffset is the column selector for filtering. + final VectorColumnSelectorFactory baseSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( + index, + baseOffset, + columnCache, + spec.getVirtualColumns() + ); + + if (filterToUse == null) { + return new FrameVectorCursor(baseOffset, baseSelectorFactory); + } else { + final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseSelectorFactory); + final FilteredVectorOffset filteredOffset = FilteredVectorOffset.create( + baseOffset, + matcher + ); + + final VectorColumnSelectorFactory filteredSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( + index, + filteredOffset, + columnCache, + spec.getVirtualColumns() + ); + + return new FrameVectorCursor(filteredOffset, filteredSelectorFactory); + } + } + + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; + } +} 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 deleted file mode 100644 index 3497ed39898a..000000000000 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.frame.segment.columnar; - -import org.apache.druid.frame.Frame; -import org.apache.druid.frame.FrameType; -import org.apache.druid.frame.read.columnar.FrameColumnReader; -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.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; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorFactory; -import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; -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; -import org.apache.druid.segment.vector.QueryableIndexVectorColumnSelectorFactory; -import org.apache.druid.segment.vector.VectorColumnSelectorFactory; -import org.apache.druid.segment.vector.VectorCursor; -import org.apache.druid.segment.vector.VectorOffset; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; - -/** - * A {@link CursorFactory} 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 - */ -public class FrameCursorFactory implements CursorFactory -{ - private final Frame frame; - private final RowSignature signature; - private final List columnReaders; - - public FrameCursorFactory( - final Frame frame, - final RowSignature signature, - final List columnReaders - ) - { - this.frame = FrameType.COLUMNAR.ensureType(frame); - this.signature = signature; - this.columnReaders = columnReaders; - } - - @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 - ) - { - 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); - } - } - - @Nullable - @Override - public VectorCursor makeVectorCursor( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @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); - } - } - - 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/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index f42a33ce6f78..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 FrameCursorFactory} 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/columnar/FrameVectorCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameVectorCursor.java index da1cbf9b0cea..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,14 +20,11 @@ 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; import org.apache.druid.segment.vector.VectorOffset; -import java.io.IOException; - /** * A {@link VectorCursor} that is based on a {@link Frame}. * @@ -37,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 @@ -75,17 +69,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/frame/segment/row/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java deleted file mode 100644 index b3232e15d7f6..000000000000 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/FrameCursorFactory.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.frame.segment.row; - -import org.apache.druid.frame.Frame; -import org.apache.druid.frame.FrameType; -import org.apache.druid.frame.field.FieldReader; -import org.apache.druid.frame.read.FrameReader; -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.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.CursorFactory; -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; - -/** - * A {@link CursorFactory} 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 - */ -public class FrameCursorFactory implements CursorFactory -{ - private final Frame frame; - private final FrameReader frameReader; - private final List fieldReaders; - - public FrameCursorFactory( - final Frame frame, - final FrameReader frameReader, - final List fieldReaders - ) - { - this.frame = FrameType.ROW_BASED.ensureType(frame); - this.frameReader = frameReader; - this.fieldReaders = fieldReaders; - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - 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 { - // Not currently needed for the intended use cases of frame-based cursors. - throw new UOE("Granularity [%s] not supported", gran); - } - } - - 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 = descending - ? new SimpleDescendingOffset(frame.numRows()) - : new SimpleAscendingOffset(frame.numRows()); - - final SimpleSettableOffset offset; - - 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); - } - - return new FrameCursor(offset, columnSelectorFactory); - } -} diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java new file mode 100644 index 000000000000..2d0d6567d5f1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java @@ -0,0 +1,127 @@ +/* + * 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.frame.segment.row; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.field.FieldReader; +import org.apache.druid.frame.read.FrameReader; +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; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * A {@link CursorHolderFactory} implementation based on a single row-based {@link Frame}. + * + * This class is only used for row-based frames. + * + * @see ColumnarFrameCursorHolderFactory the columnar version + */ +public class RowFrameCursorHolderFactory implements CursorHolderFactory +{ + private final Frame frame; + private final FrameReader frameReader; + private final List fieldReaders; + + public RowFrameCursorHolderFactory( + final Frame frame, + final FrameReader frameReader, + final List fieldReaders + ) + { + this.frame = FrameType.ROW_BASED.ensureType(frame); + this.frameReader = frameReader; + this.fieldReaders = fieldReaders; + } + + @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 + @Override + public Cursor asCursor() + { + final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); + + final SimpleSettableOffset baseOffset = descending + ? new SimpleDescendingOffset(frame.numRows()) + : new SimpleAscendingOffset(frame.numRows()); + + + final ColumnSelectorFactory columnSelectorFactory = + spec.getVirtualColumns().wrap( + new FrameColumnSelectorFactory( + frame, + frameReader.signature(), + fieldReaders, + new CursorFrameRowPointer(frame, baseOffset) + ) + ); + + final SimpleSettableOffset offset; + if (filterToUse == null) { + offset = baseOffset; + } else { + offset = new FrameFilteredOffset(baseOffset, columnSelectorFactory, filterToUse); + } + + 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; + } + + @Nullable + @Override + public List getOrdering() + { + return ordering; + } + }; + } +} 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..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,6 +19,26 @@ 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; + /** * This class was created b/c sometimes static initializers of a class that use a subclass can deadlock. * See: #2979, #3979 @@ -42,8 +62,103 @@ 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; } + + + /** + * 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) + { + // 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); + } + 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++; + } + + 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(); + } + + /** + * 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, String virtualColumnName) + { + 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( + virtualColumnName, + 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..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 @@ -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,16 @@ 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; + } + public List getMacros() { return ImmutableList.copyOf(macroMap.values()); 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/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java new file mode 100644 index 000000000000..b9d3418779a3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java @@ -0,0 +1,175 @@ +/* + * 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; + +/** + * 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. + */ +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); + } + + private final Cursor cursor; + + // Iterable that iterates over time buckets. + private final Iterable bucketIterable; + + // 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 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/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 824444410c08..aad70c9ecb5d 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -824,8 +824,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 +956,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/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/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index a0b7bda150d0..38a8b1e91d5b 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,15 @@ 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.CursorHolder; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; import java.util.Collections; @@ -81,7 +81,6 @@ public RowSignature getRowSignature() public Sequence getRowsAsSequence() { - final Sequence cursorSequence = Sequences.simple(frames) .flatMap( @@ -89,8 +88,11 @@ 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); + final CursorHolder holder = + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder( + CursorBuildSpec.FULL_SCAN + ); + return Sequences.simple(Collections.singletonList(holder.asCursor())).withBaggage(holder); } ); 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/Order.java b/processing/src/main/java/org/apache/druid/query/Order.java new file mode 100644 index 000000000000..464aa9ed7d26 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/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; + +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/OrderBy.java b/processing/src/main/java/org/apache/druid/query/OrderBy.java new file mode 100644 index 000000000000..5cdda9bb46ec --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/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; + +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/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 90b43469dce5..fa1f42d4c13d 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; 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; @@ -41,6 +42,7 @@ import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Interval; @@ -271,4 +273,18 @@ default Set getRequiredColumns() { return null; } + + /** + * Returns an interval if {@link #getIntervals()} has only a single interval, else explodes + */ + default Interval getSingleInterval() + { + 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 + ) + ); + } } 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 402001efe7d0..d3545a8494ae 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; 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/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/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/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index 99ad72f0ee93..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 @@ -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#makeCursorHolder(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 994705f55e30..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 @@ -74,7 +74,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; @@ -109,6 +108,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 @@ -206,8 +207,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; @@ -795,12 +798,17 @@ public Set getRequiredColumns() return Queries.computeRequiredColumns( virtualColumns, dimFilter, - dimensions, - aggregatorSpecs, - Collections.emptyList() + groupingColumns, + aggregatorSpecs ); } + @JsonIgnore + public List getGroupingColumns() + { + return groupingColumns; + } + @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 ab1ee1052b4b..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 @@ -47,11 +47,13 @@ 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; 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; @@ -63,7 +65,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 +75,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.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -486,6 +490,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) @@ -495,45 +501,44 @@ 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()); + final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, groupByQueryMetrics); + final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); - final boolean doVectorize = query.context().getVectorize().shouldVectorize( - VectorGroupByEngine.canVectorize(query, storageAdapter, filter) - ); + 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); final Sequence result; - - if (doVectorize) { + if (shouldVectorize) { result = VectorGroupByEngine.process( query, storageAdapter, + cursorHolder, bufferHolder.get(), fudgeTimestamp, - filter, - interval, + buildSpec.getInterval(), querySpecificConfig, - processingConfig, - groupByQueryMetrics + processingConfig ); } else { result = GroupByQueryEngine.process( query, storageAdapter, + cursorHolder, + buildSpec, bufferHolder.get(), fudgeTimestamp, querySpecificConfig, - processingConfig, - filter, - interval, - groupByQueryMetrics + processingConfig ); } - return result.withBaggage(bufferHolder); + return result.withBaggage(closer); } catch (Throwable e) { - bufferHolder.close(); + CloseableUtils.closeAndWrapExceptions(closer); throw e; } } @@ -840,6 +845,22 @@ private Set getAggregatorAndPostAggregatorNames(GroupByQuery query) return aggsAndPostAggs; } + public static CursorBuildSpec makeCursorBuildSpec(GroupByQuery query, @Nullable QueryMetrics queryMetrics) + { + 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() + ); + } + /** * 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/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index 35f09c5446d1..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 @@ -26,15 +26,17 @@ 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; 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 +50,8 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; @@ -56,7 +60,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,89 +97,141 @@ private GroupByQueryEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, + final CursorHolder cursorHolder, + final CursorBuildSpec buildSpec, 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(), + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return Sequences.empty(); + } + final CursorGranularizer granularizer = CursorGranularizer.create( + storageAdapter, + cursor, query.getGranularity(), - false, - groupByQueryMetrics + buildSpec.getInterval(), + false ); + 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(); + } - return cursors.flatMap( - cursor -> new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @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( - querySpecificConfig, - query, - storageAdapter, - processingBuffer - ); + final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( + querySpecificConfig, + query, + storageAdapter, + processingBuffer + ); - 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()) - ); - } - } + if (query.getGranularity().equals(Granularities.ALL)) { + return makeEngineIteratorSequence( + query, + processingBuffer, + fudgeTimestamp, + querySpecificConfig, + processingConfig, + cardinalityForArrayAggregation, + cursor, + granularizer, + dims, + columnSelectorFactory + ); + } - @Override - public void cleanup(GroupByEngineIterator iterFromMake) - { - iterFromMake.close(); - } + return Sequences.simple(granularizer.getBucketIterable()) + .flatMap(bucketInterval -> { + if (!granularizer.advanceToBucket(bucketInterval)) { + return Sequences.empty(); + } + 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(); + } + } ); } @@ -237,6 +292,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; @@ -253,6 +309,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, @@ -263,12 +320,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, @@ -332,7 +390,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(); } @@ -417,13 +475,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]; @@ -527,7 +586,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.) @@ -615,7 +676,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 @@ -656,6 +719,7 @@ private ArrayAggregateIterator( GroupByQueryConfig querySpecificConfig, DruidProcessingConfig processingConfig, Cursor cursor, + CursorGranularizer granularizer, ByteBuffer buffer, @Nullable DateTime fudgeTimestamp, GroupByColumnSelectorPlus[] dims, @@ -663,7 +727,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]; @@ -713,7 +777,9 @@ private void aggregateSingleValueDims(IntGrouper grouper) if (!grouper.aggregate(key).isOk()) { return; } - cursor.advance(); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } } @@ -751,7 +817,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/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index 75f4539e8c36..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 @@ -22,17 +22,14 @@ 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; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.aggregation.AggregatorAdapters; -import org.apache.druid.query.aggregation.AggregatorFactory; 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.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryMetrics; @@ -47,10 +44,9 @@ import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnProcessors; +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; -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; @@ -84,33 +80,21 @@ private VectorGroupByEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, + final CursorHolder cursorHolder, 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 = cursorHolder.asVectorCursor(); if (cursor == null) { // Return empty iterator. @@ -136,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 @@ -194,21 +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) - && canVectorizeAggregators(inspector, query.getAggregatorSpecs()); - } - - private static boolean canVectorizeDimensions( + public static boolean canVectorizeDimensions( final ColumnInspector inspector, final List dimensions ) @@ -241,19 +200,6 @@ private static boolean canVectorizeDimensions( return true; } - public static boolean canVectorizeAggregators( - final ColumnInspector inspector, - final List aggregatorFactories - ) - { - for (AggregatorFactory aggregatorFactory : aggregatorFactories) { - if (!aggregatorFactory.canVectorize(inspector)) { - return false; - } - } - return true; - } - @VisibleForTesting static class VectorGroupByEngineIterator implements CloseableIterator { @@ -366,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/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index 81fcf34ed013..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 @@ -23,19 +23,17 @@ 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.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.CursorHolder; 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; @@ -52,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; @@ -274,48 +270,25 @@ 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 - ); - - 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 CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + + 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/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..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,6 +25,8 @@ 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; @@ -34,7 +36,6 @@ import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -97,7 +98,6 @@ public WindowOperatorQuery( super( validateMaybeRewriteDataSource(dataSource, leafOperators != null), intervals, - false, context ); this.rowSignature = rowSignature; @@ -114,11 +114,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/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index ce199a7803c5..6be9500b9fe3 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,11 +30,8 @@ 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; -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; @@ -48,6 +45,8 @@ 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.CursorHolder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -208,16 +207,6 @@ 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; if (viewableColumns != null) { cols = viewableColumns; @@ -231,19 +220,27 @@ private Pair materializeStorageAdapter(StorageAdapter as) .build(); } } - AtomicReference siggy = new AtomicReference<>(null); + final CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() + .setFilter(filter); + if (interval != null) { + builder.setInterval(interval); + } + if (virtualColumns != null) { + builder.setVirtualColumns(virtualColumns); + } + try (final CursorHolder cursorHolder = as.makeCursorHolder(builder.build())) { + final Cursor cursor = cursorHolder.asCursor(); - 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); + if (cursor == null) { + return null; } + 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) { @@ -282,26 +279,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 edc6518ca046..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 @@ -24,17 +24,14 @@ 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.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.CursorHolder; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nonnull; @@ -98,25 +95,16 @@ protected RowsAndColumns getRealRAC() @Nonnull private static RowsAndColumns materialize(StorageAdapter as) { - final Sequence cursors = as.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - 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); + try (final CursorHolder cursorHolder = as.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + + if (cursor == null) { + return new EmptyRowsAndColumns(); } + + final RowSignature rowSignature = as.getRowSignature(); - final ColumnSelectorFactory columnSelectorFactory = in.getColumnSelectorFactory(); + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); final FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( new ArenaMemoryAllocatorFactory(200 << 20), // 200 MB, because, why not? @@ -124,19 +112,19 @@ private static RowsAndColumns materialize(StorageAdapter as) Collections.emptyList() ); - final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory); - while (!in.isDoneOrInterrupted()) { - frameWriter.addSelection(); - in.advance(); + 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); + } } - 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/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 96693a7b3898..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 @@ -30,12 +30,13 @@ 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; 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.filter.DimFilter; import org.apache.druid.query.operator.OffsetLimit; @@ -96,83 +97,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). @@ -210,7 +134,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; @@ -484,9 +408,8 @@ public Set getRequiredColumns() return Queries.computeRequiredColumns( virtualColumns, dimFilter, - Collections.emptyList(), - Collections.emptyList(), - columns + columns, + Collections.emptyList() ); } } 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 567e07fbdc73..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 @@ -26,16 +26,19 @@ 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.Order; 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.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.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; @@ -67,7 +70,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(); @@ -112,127 +115,121 @@ 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 - ) - .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 CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); + if (Order.NONE != query.getTimeOrder()) { + Cursors.requireTimeOrdering(cursorHolder, query.getTimeOrder()); + } + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return Collections.emptyIterator(); + } + 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(cursorHolder); } /** @@ -241,9 +238,21 @@ 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(); } + + 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 785b71ed42d1..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; @@ -99,7 +100,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..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; @@ -96,7 +97,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 +114,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..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; @@ -45,7 +46,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/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index 77325c58233c..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 @@ -21,16 +21,16 @@ 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.filter.Filter; 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.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; -import org.joda.time.Interval; +import org.apache.druid.segment.filter.Filters; import java.util.List; @@ -56,84 +56,72 @@ 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 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 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 = 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(); + + 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/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/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/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 7a695987fb76..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 @@ -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.query.ChainedExecutionQueryRunner; +import org.apache.druid.query.Order; 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,19 @@ 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.CursorHolder; +import org.apache.druid.segment.Cursors; 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 javax.annotation.Nullable; import java.util.Iterator; -import java.util.List; /** * @@ -114,24 +115,25 @@ public Result apply(Cursor cursor) }; } + @Nullable 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 List> resultList = resultSequence.limit(1).toList(); - if (resultList.size() > 0) { - return resultList.get(0).getValue(); + final CursorBuildSpec.CursorBuildSpecBuilder bob = CursorBuildSpec.builder(makeCursorBuildSpec(legacyQuery)); + if (descending) { + bob.setPreferredOrdering(Cursors.descendingTimeOrder()); + } else { + bob.setPreferredOrdering(Cursors.ascendingTimeOrder()); } - return null; + 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; + } + final Result result = skipToFirstMatching.apply(cursor); + return result == null ? null : result.getValue(); + } } @Override @@ -201,7 +203,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#makeCursorHolder(CursorBuildSpec)}. */ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, final StorageAdapter adapter) { @@ -232,4 +234,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 d255a4e4cf49..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 @@ -191,8 +191,7 @@ public Set getRequiredColumns() virtualColumns, dimFilter, Collections.emptyList(), - aggregatorSpecs, - Collections.emptyList() + aggregatorSpecs ); } 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 d8369c8c6dac..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 @@ -27,22 +27,26 @@ 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.QueryRunnerHelper; +import org.apache.druid.query.CursorGranularizer; +import org.apache.druid.query.Order; +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; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine; import org.apache.druid.query.vector.VectorCursorGranularizer; -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.CursorHolder; +import org.apache.druid.segment.Cursors; 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; @@ -50,7 +54,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -94,64 +97,52 @@ 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) - && VectorGroupByEngine.canVectorizeAggregators(inspector, query.getAggregatorSpecs()) - ); + final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); + Cursors.requireTimeOrdering(cursorHolder, query.isDescending() ? Order.DESCENDING : Order.ASCENDING); + try { + final Sequence> result; - final Sequence> result; + if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize())) { + result = processVectorized(query, adapter, cursorHolder, interval, gran); + } else { + result = processNonVectorized(query, adapter, cursorHolder, interval, gran); + } - if (doVectorize) { - result = processVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics); - } else { - result = processNonVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics); + 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); - } else { - return result; + catch (Throwable t) { + cursorHolder.close(); + throw t; } } private Sequence> processVectorized( final TimeseriesQuery query, final StorageAdapter adapter, - @Nullable final Filter filter, + final CursorHolder cursorHolder, 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 = cursorHolder.asVectorCursor(); if (cursor == null) { return Sequences.empty(); } final Closer closer = Closer.create(); - closer.register(cursor); - try { final VectorCursorGranularizer granularizer = VectorCursorGranularizer.create( adapter, @@ -253,60 +244,98 @@ private Sequence> processVectorized( private Sequence> processNonVectorized( final TimeseriesQuery query, final StorageAdapter adapter, - @Nullable final Filter filter, + final CursorHolder cursorHolder, 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(); - - return QueryRunnerHelper.makeCursorBasedQuery( + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return Sequences.empty(); + } + final CursorGranularizer granularizer = CursorGranularizer.create( adapter, - Collections.singletonList(queryInterval), - filter, - query.getVirtualColumns(), - descending, + cursor, 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(); - } - 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 + 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); + } + + public static CursorBuildSpec makeCursorBuildSpec(TimeseriesQuery query, @Nullable QueryMetrics queryMetrics) + { + 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/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 2180b73253ce..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 @@ -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, @@ -111,8 +112,10 @@ public void run( PooledTopNAlgorithm allMetricAlgo = new PooledTopNAlgorithm(storageAdapter, query, bufferPool); PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null; try { + // 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()); + 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..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 @@ -123,7 +123,9 @@ private void runWithCardinalityKnown( resetAggregators(aggregatesStore); numProcessed += numToProcess; - params.getCursor().reset(); + if (numProcessed < cardinality) { + params.getCursor().reset(); + } } 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().reset(); if (queryMetrics != null) { queryMetrics.addProcessedRows(processedRows); queryMetrics.stopRecordingScanTime(); @@ -330,7 +331,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/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 21729022d616..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 @@ -172,9 +172,8 @@ public Set getRequiredColumns() return Queries.computeRequiredColumns( virtualColumns, dimFilter, - Collections.singletonList(dimensionSpec), - aggregatorSpecs, - Collections.emptyList() + Collections.singletonList(dimensionSpec.getDimension()), + aggregatorSpecs ); } 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..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 @@ -19,16 +19,20 @@ 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.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; +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; -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.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -36,11 +40,10 @@ 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; +import java.util.Collections; /** */ @@ -57,8 +60,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, @@ -72,36 +75,36 @@ 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 + final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); + final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return Sequences.withBaggage(Sequences.empty(), cursorHolder); + } + final CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + query.getGranularity(), + buildSpec.getInterval(), + false ); + if (granularizer == null) { + return Sequences.withBaggage(Sequences.empty(), cursorHolder); + } + if (queryMetrics != null) { + queryMetrics.cursor(cursor); + } return Sequences.filter( - Sequences.map( - adapter.makeCursors( - filter, - queryIntervals.get(0), - query.getVirtualColumns(), - granularity, - query.isDescending(), - queryMetrics - ), - input -> { - if (queryMetrics != null) { - queryMetrics.cursor(input); - } - return mapFn.apply(input, queryMetrics); - } - ), - Predicates.notNull() - ); + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + granularizer.advanceToBucket(bucketInterval); + return mapFn.apply(cursor, granularizer, queryMetrics); + }), + Predicates.notNull() + ).withBaggage(cursorHolder); } /** @@ -132,13 +135,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); @@ -173,7 +176,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. @@ -181,7 +184,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()) { @@ -193,13 +199,54 @@ 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(); + + final int numBytesToWorkWith = resultsBuf.capacity(); + 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) + { + 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/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..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 @@ -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 ) { @@ -120,10 +122,11 @@ 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) { - return scanAndAggregateWithCardinalityKnown(query, cursor, selector, rowSelector); + // 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, selector); + return scanAndAggregateWithCardinalityUnknown(query, cursor, granularizer, selector); } } @@ -141,6 +144,7 @@ public void initAggregateStore() private long scanAndAggregateWithCardinalityKnown( TopNQuery query, Cursor cursor, + CursorGranularizer granularizer, DimensionSelector selector, Aggregator[][] rowSelector ) @@ -164,8 +168,10 @@ private long scanAndAggregateWithCardinalityKnown( aggregator.aggregate(); } } - cursor.advance(); processedRows++; + if (!granularizer.advanceCursorWithinBucket()) { + break; + } } return processedRows; } @@ -181,6 +187,7 @@ private long scanAndAggregateWithCardinalityKnown( private long scanAndAggregateWithCardinalityUnknown( TopNQuery query, Cursor cursor, + CursorGranularizer granularizer, DimensionSelector selector ) { @@ -198,8 +205,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..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,43 +31,15 @@ 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. 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. */ 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 +74,36 @@ public static VectorCursorGranularizer create( return new VectorCursorGranularizer(cursor, bucketIterable, timeSelector); } + 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/ArrayListSegment.java b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java index e7423b5b89d9..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,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#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 * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's 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..94324fe22060 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -23,13 +23,13 @@ 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; 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 +126,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) { @@ -153,7 +156,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; } @@ -169,13 +172,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 = iteratorForMark; + this.valueForMark = markValue; } @Override @@ -197,8 +208,10 @@ public boolean withinBounds() @Override public void reset() { - iterator = safeClone(iteratorForReset); + iterator = iteratorForReset.clone(); value = valueForReset; + valueForMark = valueForReset; + iteratorForMark = null; } @Override @@ -211,7 +224,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 @@ -226,11 +245,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.hasNext() ? iterator.clone() : EmptyIntIterator.instance(); - } } 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..a1fc14c10540 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -19,14 +19,33 @@ package org.apache.druid.segment; -import org.joda.time.DateTime; +import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.segment.incremental.IncrementalIndexCursorHolder; /** - * 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 - * org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter.IncrementalIndexCursor} is an implementation for {@link - * org.apache.druid.segment.incremental.IncrementalIndex}. + * 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(...)) {
+ *     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 * rather than query execution (as Cursor). If those abstractions could be collapsed (and if it is worthwhile) is yet to * be determined. @@ -35,11 +54,42 @@ */ 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(); - DateTime getTime(); + + /** + * 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 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, 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(); + + /** + * Check if the current cursor position is valid, or if the thread has been interrupted. + * + * @see #isDone() + */ boolean isDoneOrInterrupted(); + + /** + * 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/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java new file mode 100644 index 000000000000..f6cbbdeffe8d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -0,0 +1,292 @@ +/* + * 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.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.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; + +public class CursorBuildSpec +{ + public static final CursorBuildSpec FULL_SCAN = builder().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; + @Nullable + private final List groupingColumns; + private final VirtualColumns virtualColumns; + @Nullable + private final List aggregators; + @Nullable + private final List orderByColumns; + + private final QueryContext queryContext; + + @Nullable + private final QueryMetrics queryMetrics; + + public CursorBuildSpec( + @Nullable Filter filter, + Interval interval, + @Nullable List groupingColumns, + VirtualColumns virtualColumns, + @Nullable List aggregators, + @Nullable List preferredOrdering, + QueryContext queryContext, + @Nullable QueryMetrics queryMetrics + ) + { + this.filter = filter; + this.interval = interval; + this.groupingColumns = groupingColumns; + this.virtualColumns = virtualColumns; + this.aggregators = aggregators; + this.orderByColumns = preferredOrdering; + this.queryContext = queryContext; + 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; + } + + public static class CursorBuildSpecBuilder + { + @Nullable + private Filter filter; + private Interval interval = Intervals.ETERNITY; + + @Nullable + private List groupingColumns; + private VirtualColumns virtualColumns = VirtualColumns.EMPTY; + @Nullable + private List aggregators; + @Nullable + private List preferredOrdering; + + private QueryContext queryContext = QueryContext.empty(); + @Nullable + private QueryMetrics queryMetrics; + + private CursorBuildSpecBuilder() + { + // initialize with defaults + } + + private CursorBuildSpecBuilder(CursorBuildSpec buildSpec) + { + this.filter = buildSpec.filter; + this.interval = buildSpec.interval; + this.groupingColumns = buildSpec.groupingColumns; + this.virtualColumns = buildSpec.virtualColumns; + this.aggregators = buildSpec.aggregators; + this.preferredOrdering = buildSpec.orderByColumns; + this.queryContext = buildSpec.queryContext; + 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; + } + + /** + * @see CursorBuildSpec#getGroupingColumns() + */ + public CursorBuildSpecBuilder setGroupingColumns( + @Nullable List groupingColumns + ) + { + 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; + return this; + } + + public CursorBuildSpec build() + { + return new CursorBuildSpec( + filter, + interval, + groupingColumns, + virtualColumns, + aggregators, + preferredOrdering, + queryContext, + 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..551430a00e33 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.QueryMetrics; @@ -32,7 +33,12 @@ * 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 CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. Implementors should implement + * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. */ +@Deprecated public interface CursorFactory { /** @@ -40,27 +46,46 @@ 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 CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call + * {@link CursorHolder#canVectorize()}. + * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is + * no longer implemented by any built-in factories. */ + @Deprecated default boolean canVectorize( @Nullable Filter filter, VirtualColumns virtualColumns, boolean descending ) { - return false; + throw DruidException.defensive( + "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 CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call + * {@link CursorHolder#asCursor()}. + * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. + * This method is no longer implemented by any built-in factories. */ - Sequence makeCursors( + @Deprecated + default Sequence makeCursors( @Nullable Filter filter, Interval interval, VirtualColumns virtualColumns, Granularity gran, boolean descending, @Nullable QueryMetrics queryMetrics - ); + ) + { + throw DruidException.defensive( + "CursorFactory.makeCursors is no longer supported, use CursorHolderFactory.makeCursorHolder instead" + ); + } /** * Creates a VectorCursor. Unlike the Cursor returned by "makeCursor", there is just one of these. Hence, this method @@ -69,7 +94,13 @@ 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 CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call + * {@link CursorHolder#asVectorCursor()}. Implementors should implement + * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is no longer implemented by any + * built-in factories. */ + @Deprecated @Nullable default VectorCursor makeVectorCursor( @Nullable Filter filter, @@ -80,6 +111,8 @@ default VectorCursor makeVectorCursor( @Nullable QueryMetrics queryMetrics ) { - throw new UnsupportedOperationException("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'."); + throw DruidException.defensive( + "CursorFactory.makeVectorCursor is no longer supported, use CursorHolderFactory.makeCursorHolder instead" + ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java new file mode 100644 index 000000000000..d0384e265b34 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java @@ -0,0 +1,73 @@ +/* + * 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.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 +{ + /** + * Create a {@link Cursor} for use with non-vectorized query engines. + */ + @Nullable + Cursor asCursor(); + + /** + * Create a {@link VectorCursor} for use with vectorized query engines. + */ + @Nullable + default VectorCursor asVectorCursor() + { + throw new UOE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor' on %s.", this.getClass().getName()); + } + + /** + * Returns true if this {@link CursorHolder} supports calling {@link #asVectorCursor()}. + */ + 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. + */ + @Override + default void close() + { + // nothing to close + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java new file mode 100644 index 000000000000..e191f033f593 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.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 CursorHolderFactory +{ + CursorHolder makeCursorHolder(CursorBuildSpec spec); +} 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..814681ab8b47 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/Cursors.java @@ -0,0 +1,105 @@ +/* + * 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.java.util.common.StringUtils; +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} + * in the specified {@link Order} + */ + 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()) && orderBy.getOrder() == order; + } + + /** + * 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) + { + 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); + } + } + + /** + * 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/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index 1168e34b28af..b4b0ddfbb60b 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; @@ -35,36 +32,36 @@ 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; } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - final Filter andFilter; - if (filter == null) { + final CursorBuildSpec.CursorBuildSpecBuilder buildSpecBuilder = CursorBuildSpec.builder(spec); + final Filter newFilter; + if (spec.getFilter() == null) { if (filterOnDataSource != null) { - andFilter = filterOnDataSource.toFilter(); + newFilter = filterOnDataSource.toFilter(); } else { - andFilter = null; + newFilter = null; } } else { - andFilter = new AndFilter(ImmutableList.of(filter, filterOnDataSource.toFilter())); + if (filterOnDataSource != null) { + newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter())); + } else { + newFilter = spec.getFilter(); + } } - return baseStorageAdapter.makeCursors(andFilter, interval, virtualColumns, gran, descending, queryMetrics); + buildSpecBuilder.setFilter(newFilter); + return baseStorageAdapter.makeCursorHolder(buildSpecBuilder.build()); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java similarity index 52% rename from processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java rename to processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 86069068eaf9..0f0646839ac7 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -20,21 +20,20 @@ package org.apache.druid.segment; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import org.apache.druid.collections.bitmap.BitmapFactory; -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.DateTimes; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; 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; +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; @@ -53,65 +52,102 @@ 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; +import java.io.Closeable; import java.io.IOException; +import java.util.Collections; +import java.util.List; import java.util.concurrent.TimeUnit; -public class QueryableIndexCursorSequenceBuilder +public class QueryableIndexCursorHolder implements CursorHolder { - private static final Logger log = new Logger(QueryableIndexCursorSequenceBuilder.class); + private static final Logger log = new Logger(QueryableIndexCursorHolder.class); private final QueryableIndex index; private final Interval interval; 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 List ordering; private final boolean descending; + private final QueryContext queryContext; + private final int vectorSize; + private final Supplier resourcesSupplier; - public QueryableIndexCursorSequenceBuilder( + public QueryableIndexCursorHolder( 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.virtualColumns = cursorBuildSpec.getVirtualColumns(); + this.aggregatorFactories = cursorBuildSpec.getAggregators(); + this.filter = cursorBuildSpec.getFilter(); + // 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(); + this.resourcesSupplier = Suppliers.memoize(() -> new CursorResources(index, virtualColumns, filter, metrics)); } - 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(); + 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; + } - final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector( - index.getBitmapFactoryForDimensions(), - virtualColumns, - columnCache - ); + @Override + public Cursor asCursor() + { + 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,211 +157,147 @@ 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); + final long timeStart = Math.max(interval.getStartMillis(), minDataTimestamp); + final long timeEnd = interval.getEndMillis(); - 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; - } - } - } 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); - } - } - } - ), - closer + 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); + } else { + return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory); + } } - public VectorCursor buildVectorized(final int vectorSize) + @Nullable + @Override + public VectorCursor asVectorCursor() { - // 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()) { + close(); + throw new IllegalStateException("canVectorize()"); + } + if (metrics != null) { + metrics.vectorized(true); + } - NumericColumn timestamps = null; - final int startOffset; - final int endOffset; + final int startOffset; + final int endOffset; - if (interval.getStartMillis() > minDataTimestamp) { - timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); + if (interval.getStartMillis() > minDataTimestamp) { + startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows()); + } else { + startOffset = 0; + } - 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) { - if (timestamps == null) { - timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); - } + // 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); - endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows()); - } else { - endOffset = index.getNumRows(); - } + // 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 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 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 + ); - // baseColumnSelectorFactory using baseOffset is the column selector for filtering. - final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( + // Now create the cursor and column selector that will be returned to the caller. + final VectorColumnSelectorFactory filteredColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( columnCache, - baseOffset + filteredOffset ); - - // 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, closer); - } else { - return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer); - } - } - catch (Throwable t) { - throw CloseableUtils.closeAndWrapInCatch(t, closer); + return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize); + } else { + return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize); } } - @Nullable - private FilterBundle makeFilterBundle( - ColumnSelectorColumnIndexSelector bitmapIndexSelector, - int numRows - ) + @Override + public List getOrdering() { - 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; + return ordering; + } + + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(resourcesSupplier.get()); } - VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( + + private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( ColumnCache columnCache, VectorOffset baseOffset ) @@ -391,7 +363,6 @@ static int timeSearch( private static class QueryableIndexVectorCursor implements VectorCursor { - private final Closer closer; private final int vectorSize; private final VectorOffset offset; private final VectorColumnSelectorFactory columnSelectorFactory; @@ -399,14 +370,12 @@ private static class QueryableIndexVectorCursor implements VectorCursor public QueryableIndexVectorCursor( final VectorColumnSelectorFactory vectorColumnSelectorFactory, final VectorOffset offset, - final int vectorSize, - final Closer closer + final int vectorSize ) { this.columnSelectorFactory = vectorColumnSelectorFactory; this.vectorSize = vectorSize; this.offset = offset; - this.closer = closer; } @Override @@ -445,30 +414,17 @@ 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 { 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 @@ -483,19 +439,13 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return bucketStart; - } - @Override 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 state. We should not let // aggregators, etc. access this illegal state and throw a QueryInterruptedException by calling // BaseQuery.checkInterrupted(). BaseQuery.checkInterrupted(); @@ -526,7 +476,6 @@ public void reset() } } - public abstract static class TimestampCheckingOffset extends Offset { final Offset baseOffset; @@ -668,4 +617,98 @@ public Offset clone() return new DescendingTimestampCheckingOffset(baseOffset.clone(), timestamps, timeLimit, allWithinThreshold); } } + + 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( + 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(), + 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(); + } + + @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 57c7da953605..7d860465cb6f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -20,13 +20,6 @@ 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; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -35,7 +28,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; @@ -43,7 +35,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.LinkedHashSet; -import java.util.Objects; import java.util.Set; /** @@ -179,97 +170,11 @@ public DateTime getMaxIngestedEventTime() } @Override - public boolean canVectorize( - @Nullable final Filter filter, - final VirtualColumns virtualColumns, - final boolean descending - ) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - 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) { - 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 - ) - { - 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( + return new QueryableIndexCursorHolder( index, - actualInterval, - virtualColumns, - filter, - queryMetrics, - getMinTime().getMillis(), - getMaxTime().getMillis(), - descending - ).buildVectorized(vectorSize > 0 ? vectorSize : DEFAULT_VECTOR_SIZE); - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @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 + CursorBuildSpec.builder(spec).build() ); } @@ -288,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/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index f843fe631311..d1a70783f6b7 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.granularity.Granularity; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -42,7 +41,7 @@ public class RowBasedCursor implements Cursor private final ToLongFunction timestampFunction; private final Interval interval; private final boolean descending; - private final DateTime cursorTime; + private final DateTime startTime; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher valueMatcher; @@ -54,7 +53,6 @@ public RowBasedCursor( @Nullable final Filter filter, final Interval interval, final VirtualColumns virtualColumns, - final Granularity gran, final boolean descending, final RowSignature rowSignature ) @@ -63,7 +61,7 @@ public RowBasedCursor( this.timestampFunction = rowAdapter.timestampFunction(); this.interval = interval; this.descending = descending; - this.cursorTime = gran.toDateTime(interval.getStartMillis()); + this.startTime = descending ? interval.getEnd().minus(1) : interval.getStart(); this.columnSelectorFactory = virtualColumns.wrap( new RowBasedColumnSelectorFactory<>( rowWalker::currentRow, @@ -80,8 +78,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(); } @@ -91,12 +88,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return cursorTime; - } - @Override public void advance() { @@ -129,7 +120,7 @@ public void reset() { rowId = 0; rowWalker.reset(); - rowWalker.skipToDateTime(descending ? interval.getEnd().minus(1) : interval.getStart(), descending); + rowWalker.skipToDateTime(startTime, descending); advanceToMatchingRow(); } 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..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,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#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 * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's 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..f4330709a44b 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -20,21 +20,20 @@ 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; 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.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.query.OrderBy; 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; @@ -168,52 +167,52 @@ public Metadata getMetadata() } @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval queryInterval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable final QueryMetrics queryMetrics - ) - { - final Interval actualInterval = queryInterval.overlap(new Interval(getMinTime(), gran.bucketEnd(getMaxTime()))); - - if (actualInterval == null) { - return Sequences.empty(); + 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(); + + @Override + public Cursor asCursor() + { + final RowWalker rowWalker = closer.register( + new RowWalker<>(descending ? reverse(rowSequence) : rowSequence, rowAdapter) + ); + return new RowBasedCursor<>( + rowWalker, + rowAdapter, + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + descending, + rowSignature + ); + } - if (!isQueryGranularityAllowed(actualInterval, gran)) { - throw new IAE( - "Cannot support interval [%s] with granularity [%s]", - Intervals.ETERNITY.equals(actualInterval) ? "ETERNITY" : actualInterval, - gran - ); - } + @Nullable + @Override + public List getOrdering() + { + return ordering; + } - 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); + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; } /** 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..4e39ca00f18f 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#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 2d3fc6a50bbb..fba947f73d2a 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -21,20 +21,99 @@ 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; +import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.List; import java.util.Optional; /** */ @PublicApi -public interface StorageAdapter extends CursorFactory, ColumnInspector +public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHolderFactory { + + /** + * 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 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 + public boolean canVectorize() + { + return StorageAdapter.this.canVectorize( + spec.getFilter(), + spec.getVirtualColumns(), + Cursors.preferDescendingTimeOrdering(spec) + ); + } + + @Override + public Cursor asCursor() + { + return Iterables.getOnlyElement( + StorageAdapter.this.makeCursors( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + Granularities.ALL, + descending, + spec.getQueryMetrics() + ).toList() + ); + } + + @Override + public VectorCursor asVectorCursor() + { + return StorageAdapter.this.makeVectorCursor( + spec.getFilter(), + spec.getInterval(), + spec.getVirtualColumns(), + descending, + spec.getQueryContext().getVectorSize(), + spec.getQueryMetrics() + ); + } + + @Nullable + @Override + public List getOrdering() + { + return ordering; + } + + @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 5404937f5196..a2a920cba1d5 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,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 org.checkerframework.checker.nullness.qual.MonotonicNonNull; import javax.annotation.Nullable; import java.util.Arrays; @@ -62,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; @@ -210,12 +211,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..398c0a21deff 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; @@ -74,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; @@ -259,12 +259,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 752c6f486786..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,13 +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.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.OrderBy; import org.apache.druid.query.filter.BooleanFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.EqualityFilter; @@ -49,6 +48,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; @@ -57,8 +57,8 @@ import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; -import java.util.Objects; 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 @@ -92,65 +92,77 @@ public UnnestStorageAdapter( } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - final String inputColumn = getUnnestInputIfDirectAccess(unnestColumn); + final String input = getUnnestInputIfDirectAccess(unnestColumn); final Pair filterPair = computeBaseAndPostUnnestFilters( - filter, + spec.getFilter(), 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 + 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 Sequences.map( - baseCursorSequence, - cursor -> { - Objects.requireNonNull(cursor); - final ColumnCapabilities capabilities = unnestColumn.capabilities( + return new CursorHolder() + { + final Closer closer = Closer.create(); + final Supplier cursorHolderSupplier = Suppliers.memoize( + () -> closer.register(baseAdapter.makeCursorHolder(unnestBuildSpec)) + ); + + @Override + public Cursor asCursor() + { + final Cursor cursor = cursorHolderSupplier.get().asCursor(); + if (cursor == null) { + return null; + } + final ColumnCapabilities capabilities = unnestColumn.capabilities( + cursor.getColumnSelectorFactory(), + unnestColumn.getOutputName() + ); + final Cursor unnestCursor; + + if (useDimensionCursor(capabilities)) { + unnestCursor = new UnnestDimensionCursor( + cursor, cursor.getColumnSelectorFactory(), - unnestColumn.getOutputName() + unnestColumn, + outputColumnName ); - 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 + } else { + unnestCursor = new UnnestColumnValueSelectorCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName ); } - ); + return PostJoinCursor.wrap( + unnestCursor, + spec.getVirtualColumns(), + filterPair.rhs + ); + } + + @Nullable + @Override + public List getOrdering() + { + return cursorHolderSupplier.get().getOrdering(); + } + + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; } @Override @@ -270,9 +282,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 @@ -468,7 +480,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/VirtualColumns.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java index a32a85d16c77..bc2609b4a00e 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; @@ -129,15 +128,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; - } - } - // For equals, hashCode, toString, and serialization: private final List virtualColumns; private final List virtualColumnNames; 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..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 @@ -41,9 +41,15 @@ @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(); 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/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/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java new file mode 100644 index 000000000000..f0966a828eb9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -0,0 +1,251 @@ +/* + * 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.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, + IncrementalIndex index, + CursorBuildSpec spec + ) + { + 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 + public Cursor asCursor() + { + if (index.isEmpty()) { + return null; + } + + if (spec.getQueryMetrics() != null) { + spec.getQueryMetrics().vectorized(false); + } + + + return new IncrementalIndexCursor( + storageAdapter, + index, + spec.getVirtualColumns(), + descending, + spec.getFilter(), + spec.getInterval() + ); + } + + @Override + public List getOrdering() + { + return ordering; + } + + static class IncrementalIndexCursor implements Cursor + { + private IncrementalIndexRowHolder currEntry; + private final ColumnSelectorFactory columnSelectorFactory; + private final ValueMatcher filterMatcher; + private final int maxRowIndex; + private final IncrementalIndex.FactsHolder facts; + private Iterator baseIter; + private Iterable cursorIterable; + private boolean emptyRange; + private int numAdvanced; + private boolean done; + + IncrementalIndexCursor( + IncrementalIndexStorageAdapter storageAdapter, + IncrementalIndex index, + VirtualColumns virtualColumns, + boolean descending, + @Nullable Filter filter, + Interval actualInterval + ) + { + 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; + facts = index.getFacts(); + cursorIterable = facts.timeRangeIterable( + descending, + actualInterval.getStartMillis(), + actualInterval.getEndMillis() + ); + emptyRange = !cursorIterable.iterator().hasNext(); + + reset(); + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return columnSelectorFactory; + } + + @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/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/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/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 8d72133a205c..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 @@ -19,36 +19,23 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionIndexer; 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; 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; /** */ @@ -256,37 +243,9 @@ public DateTime getMaxIngestedEventTime() } @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - 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 new IncrementalIndexCursorHolder(this, index, spec); } @Override @@ -294,168 +253,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/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index 9efbcaecdee7..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 @@ -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() { @@ -256,6 +248,7 @@ public void reset() leftCursor.reset(); joinMatcher.reset(); joinColumnSelectorFactory.resetRowId(); + initialize(); } } 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 9dade664f8ac..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 @@ -23,14 +23,13 @@ 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.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.query.QueryMetrics; +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; @@ -42,11 +41,10 @@ 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; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.Arrays; import java.util.LinkedHashSet; @@ -219,135 +217,123 @@ public boolean hasBuiltInFilters() } @Override - public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - // 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); - } + // make a copy of CursorBuildSpec with filters removed + final CursorBuildSpec.CursorBuildSpecBuilder cursorBuildSpecBuilder = CursorBuildSpec.builder(spec) + .setFilter(null); - @Override - public boolean isFromTombstone() - { - return baseAdapter.isFromTombstone(); - } - - @Nullable - @Override - public VectorCursor makeVectorCursor( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @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 - ); - } - - @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 - ) - { - final Filter combinedFilter = baseFilterAnd(filter); + final Filter combinedFilter = baseFilterAnd(spec.getFilter()); if (clauses.isEmpty()) { - return baseAdapter.makeCursors( - combinedFilter, - interval, - virtualColumns, - gran, - descending, - queryMetrics - ); + // 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) + .build(); + return baseAdapter.makeCursorHolder(newSpec); } - // 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 JoinFilterPreAnalysis preAnalysis; - if (keyIn.equals(keyCached)) { - // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursors call (keyIn). - preAnalysis = joinFilterPreAnalysis; + // 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 { - // 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. - preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn); + ordering = Cursors.ascendingTimeOrder(); + descending = false; } + return new CursorHolder() + { + final Closer joinablesCloser = Closer.create(); + + @Override + public Cursor asCursor() + { + // 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(), + clauses, + spec.getVirtualColumns(), + combinedFilter + ); + + 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 makeCursorHolder call + // (keyIn). + preAnalysis = joinFilterPreAnalysis; + } 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. + preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn); + } + + final JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter( + preAnalysis, + baseFilter + ); - final JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter( - preAnalysis, - baseFilter - ); - final Sequence baseCursorSequence = baseAdapter.makeCursors( - joinFilterSplit.getBaseTableFilter().isPresent() ? joinFilterSplit.getBaseTableFilter().get() : null, - interval, - VirtualColumns.fromIterable( + if (joinFilterSplit.getBaseTableFilter().isPresent()) { + cursorBuildSpecBuilder.setFilter(joinFilterSplit.getBaseTableFilter().get()); + } + final VirtualColumns preJoinVirtualColumns = VirtualColumns.fromIterable( Iterables.concat( Sets.difference( - ImmutableSet.copyOf(virtualColumns.getVirtualColumns()), + ImmutableSet.copyOf(spec.getVirtualColumns().getVirtualColumns()), joinFilterPreAnalysis.getPostJoinVirtualColumns() ), joinFilterSplit.getPushDownVirtualColumns() ) - ), - gran, - descending, - queryMetrics - ); + ); + cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns); + + final Cursor baseCursor = joinablesCloser.register(baseAdapter.makeCursorHolder(cursorBuildSpecBuilder.build())) + .asCursor(); + + if (baseCursor == null) { + return null; + } + + Cursor retVal = baseCursor; - final 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.fromIterable(preAnalysis.getPostJoinVirtualColumns()), - joinFilterSplit.getJoinTableFilter().orElse(null) - ); + for (JoinableClause clause : clauses) { + retVal = HashJoinEngine.makeJoinCursor(retVal, clause, descending, joinablesCloser); } - ).withBaggage(joinablesCloser); + + return PostJoinCursor.wrap( + retVal, + VirtualColumns.fromIterable(preAnalysis.getPostJoinVirtualColumns()), + joinFilterSplit.getJoinTableFilter().orElse(null) + ); + } + + @Nullable + @Override + public List getOrdering() + { + return ordering; + } + + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(joinablesCloser); + } + }; + } + + @Override + public boolean isFromTombstone() + { + return baseAdapter.isFromTombstone(); } /** @@ -368,7 +354,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 "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/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 26c119bd34f3..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 @@ -24,14 +24,14 @@ 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#makeCursorHolder(CursorBuildSpec)} to add post-join + * virtual columns and filters. */ public class PostJoinCursor implements Cursor { @@ -105,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 de306209f908..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 @@ -21,9 +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.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; @@ -31,6 +28,8 @@ 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.CursorHolder; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; @@ -44,12 +43,12 @@ 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; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.Set; @@ -113,67 +112,54 @@ 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 Sequence sequence = Sequences.map( - cursors, - cursor -> { - if (cursor == null) { - return 0; - } - int rowNumber = 0; - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + this.keyColumnsIndexes = Collections.emptyList(); + return; + } - // 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 3e2b46d5f681..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 @@ -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,11 @@ 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.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; @@ -114,62 +114,58 @@ public FrameBasedIndexedTable( indexBuilders.add(m); } - final Sequence cursors = Sequences.concat( + 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 frameStorageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + return new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); }) .collect(Collectors.toList()) ); 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/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index fde58855a53e..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 @@ -19,22 +19,22 @@ package org.apache.druid.segment.vector; -import java.io.Closeable; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolderFactory; /** - * 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}. - * - * 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. - * + * 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: * *

- *   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()) {
@@ -45,7 +45,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.
@@ -67,10 +67,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/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/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
index a8190e29f3af..173750a74c75 100644
--- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
+++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java
@@ -20,31 +20,28 @@
 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.OrderBy;
 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.CursorHolder;
 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;
 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 org.joda.time.Interval;
 
 import javax.annotation.Nullable;
+import java.util.List;
 
 /**
  * Storage adapter around {@link QueryableIndex} that transforms all multi-value strings columns into string arrays.
@@ -57,29 +54,34 @@ public TestArrayStorageAdapter(QueryableIndex index)
   }
 
   @Override
-  public boolean canVectorize(
-      @Nullable Filter filter,
-      VirtualColumns virtualColumns,
-      boolean descending
-  )
+  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
-    return false;
-  }
+    final CursorHolder delegate = super.makeCursorHolder(spec);
+    return new CursorHolder()
+    {
+      @Nullable
+      @Override
+      public Cursor asCursor()
+      {
+        return new DecoratedCursor(delegate.asCursor());
+      }
 
-  @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);
+      @Nullable
+      @Override
+      public List getOrdering()
+      {
+        return delegate.getOrdering();
+      }
+
+      @Override
+      public void close()
+      {
+        delegate.close();
+      }
+    };
   }
 
+
   @Override
   public RowSignature getRowSignature()
   {
@@ -191,12 +193,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 16d155f3df36..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
@@ -31,10 +31,10 @@
 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.CursorHolder;
 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;
@@ -128,32 +128,31 @@ protected void verifySingleFrameReadableChannel(
     Assert.assertTrue(readableFrameChannel.isFinished());
     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();
-
     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();
-
-    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);
+    // build list of rows from written and read data to verify
+    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()
+      );
+      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 481851a0f75a..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
@@ -20,20 +20,25 @@
 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;
 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;
+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.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;
@@ -51,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;
@@ -67,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
@@ -227,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
@@ -244,6 +248,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 +267,13 @@ public CursorTests(
       this.interval = interval;
       this.virtualColumns = virtualColumns;
       this.descending = descending;
+      this.buildSpec = CursorBuildSpec.builder()
+                                      .setFilter(this.filter)
+                                      .setInterval(this.interval)
+                                      .setVirtualColumns(this.virtualColumns)
+                                      .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : null)
+                                      .setQueryContext(queryContext)
+                                      .build();
     }
 
     @Parameterized.Parameters(name = "frameType = {0}, "
@@ -345,58 +361,46 @@ public void tearDown()
     }
 
     @Test
-    public void test_makeCursors()
+    public void test_makeCursor()
     {
-      assertCursorsMatch(
-          adapter ->
-              adapter.makeCursors(
-                  filter,
-                  interval,
-                  virtualColumns,
-                  Granularities.ALL, // Frames only support Granularities.ALL: no point testing the others.
-                  descending,
-                  null
-              )
-      );
+      assertCursorMatch(adapter -> adapter.makeCursorHolder(buildSpec));
     }
 
     @Test
     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.makeCursorHolder(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 CursorHolder queryableMaker = call.apply(queryableAdapter);
+           final CursorHolder frameMaker = call.apply(frameAdapter)) {
+        final Sequence> queryableRows =
+            FrameTestUtil.readRowsFromCursor(advanceAndReset(queryableMaker.asCursor()), signature);
+        final Sequence> frameRows =
+            FrameTestUtil.readRowsFromCursor(advanceAndReset(frameMaker.asCursor()), signature);
+        FrameTestUtil.assertRowsEqual(queryableRows, frameRows);
+      }
     }
 
-    private void assertVectorCursorsMatch(final Function call)
+    private void assertVectorCursorsMatch(final Function call)
     {
-      final RowSignature signature = frameAdapter.getRowSignature();
-      final Sequence> queryableRows =
-          FrameTestUtil.readRowsFromVectorCursor(call.apply(queryableAdapter), signature);
-      final Sequence> frameRows =
-          FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(call.apply(frameAdapter)), signature);
-      FrameTestUtil.assertRowsEqual(queryableRows, frameRows);
+      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(advanceAndReset(cursorHolder.asVectorCursor()), signature).withBaggage(cursorHolder);
+        final Sequence> frameRows =
+            FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameCursorHolder.asVectorCursor()), signature)
+                         .withBaggage(frameCursorHolder);
+        FrameTestUtil.assertRowsEqual(queryableRows, frameRows);
+      } else {
+        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 573e5c521233..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,6 +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.CursorHolder;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
@@ -138,67 +139,65 @@ public Sequence frames()
       throw DruidException.defensive("Unrecognized frame type");
     }
 
-    final Sequence cursors = FrameTestUtil.makeCursorsForAdapter(adapter, populateRowNumber);
-
-    return cursors.flatMap(
-        cursor -> new BaseSequence<>(
-            new BaseSequence.IteratorMaker>()
+    final CursorHolder cursorHolder = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber);
+    final Cursor cursor = cursorHolder.asCursor();
+    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(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 7402e6ef5920..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
@@ -33,14 +33,16 @@
 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.OrderBy;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.segment.ColumnProcessors;
 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.CursorHolder;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.VirtualColumns;
@@ -218,7 +220,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#makeCursorHolder(CursorBuildSpec)}.
    *
    * @param channel     the channel
    * @param frameReader reader for this channel
@@ -230,10 +232,15 @@ public static Sequence> readRowsFromFrameChannel(
   {
     return new FrameChannelSequence(channel)
         .flatMap(
-            frame ->
-                new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)
-                    .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null)
-                    .flatMap(cursor -> readRowsFromCursor(cursor, frameReader.signature()))
+            frame -> {
+              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(), cursorHolder);
+              }
+              return readRowsFromCursor(cursor, frameReader.signature()).withBaggage(cursorHolder);
+            }
         );
   }
 
@@ -254,20 +261,23 @@ public static Sequence> readRowsFromAdapter(
   )
   {
     final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature;
-    return makeCursorsForAdapter(adapter, populateRowNumber).flatMap(
-        cursor -> readRowsFromCursor(cursor, signatureToUse)
-    );
+    final CursorHolder cursorHolder = makeCursorForAdapter(adapter, populateRowNumber);
+    final Cursor cursor = cursorHolder.asCursor();
+    if (cursor == null) {
+      return Sequences.withBaggage(Sequences.empty(), cursorHolder);
+    }
+    return readRowsFromCursor(cursor, signatureToUse).withBaggage(cursorHolder);
   }
 
   /**
-   * 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 Sequence makeCursorsForAdapter(
+  public static CursorHolder makeCursorForAdapter(
       final StorageAdapter adapter,
       final boolean populateRowNumber
   )
@@ -283,14 +293,40 @@ public static Sequence makeCursorsForAdapter(
       virtualColumns = VirtualColumns.EMPTY;
     }
 
-    return adapter.makeCursors(null, Intervals.ETERNITY, virtualColumns, Granularities.ALL, false, null)
-                  .map(cursor -> {
-                    if (populateRowNumber) {
-                      return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn);
-                    } else {
-                      return cursor;
-                    }
-                  });
+    final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
+                                                     .setVirtualColumns(virtualColumns)
+                                                     .build();
+
+    final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec);
+    if (populateRowNumber) {
+      return new CursorHolder()
+      {
+        @Nullable
+        @Override
+        public Cursor asCursor()
+        {
+          final Cursor cursor = cursorHolder.asCursor();
+          if (cursor == null) {
+            return null;
+          }
+          return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn);
+        }
+
+        @Override
+        public List getOrdering()
+        {
+          return cursorHolder.getOrdering();
+        }
+
+        @Override
+        public void close()
+        {
+          cursorHolder.close();
+        }
+      };
+    } else {
+      return cursorHolder;
+    }
   }
 
   public static Sequence> readRowsFromCursor(final Cursor cursor, final RowSignature signature)
@@ -326,46 +362,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/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 c45ffa697350..9d7a9b725531 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,19 @@
 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.Cursor;
+import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorHolder;
 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;
@@ -576,48 +577,45 @@ private static Pair writeFrame(
       inputSegment = new FrameSegment(inputFrame, FrameReader.create(signature), SegmentId.dummy("xxx"));
     }
 
-    return inputSegment.asStorageAdapter()
-                       .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null)
-                       .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 CursorHolder cursorHolder = inputSegment.asStorageAdapter()
+                                                       .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      final Cursor cursor = cursorHolder.asCursor();
+
+      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/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
index dbb952fbf1cf..41834d448fba 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, 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, column.getExpression());
+    column = Granularities.toVirtualColumn(hourWithTz, 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, column.getExpression());
+    column = Granularities.toVirtualColumn(Granularities.NONE, 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')", 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);
+    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
new file mode 100644
index 000000000000..8aeaf6260705
--- /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.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.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 (CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      final Cursor cursor = cursorHolder.asCursor();
+      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()
+                                                      .setPreferredOrdering(Cursors.descendingTimeOrder())
+                                                      .build();
+    try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) {
+      final Cursor cursor = cursorHolder.asCursor();
+      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/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 2784d6b89181..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
@@ -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,154 @@ public void testSegmentLookUpForNestedQueries()
     Assert.assertEquals(innerQuerySegmentSpec, BaseQuery.getQuerySegmentSpecForLookUp(query));
   }
 
+  @Test
+  public void testAsCursorBuildSpecAllGranularity()
+  {
+    final VirtualColumns virtualColumns = VirtualColumns.create(
+        new ExpressionVirtualColumn("v0", "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.ALL)
+        .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("quality", "market", "v0"), buildSpec.getGroupingColumns());
+    Assert.assertEquals(ImmutableList.of(QueryRunnerTestHelper.ROWS_COUNT, longSum), buildSpec.getAggregators());
+    Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
+  }
+
+  @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");
+    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, "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),
+            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()
+    );
+  }
+
   @Test
   public void testEquals()
   {
@@ -176,7 +326,8 @@ public void testEquals()
                       "forceLimitPushDown",
                       "postProcessingFn",
                       "resultRowSignature",
-                      "universalTimestamp"
+                      "universalTimestamp",
+                      "groupingColumns"
                   )
                   .verify();
   }
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 6c0ff3b49511..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
@@ -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.CursorHolder;
 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 = GroupingEngine.makeCursorBuildSpec(groupQuery, null);
     boolean allCanVectorize = segments.stream()
                                       .allMatch(
-                                          s -> s.asStorageAdapter()
-                                                .canVectorize(
-                                                    filter,
-                                                    groupQuery.getVirtualColumns(),
-                                                    groupQuery.isDescending()
-                                                )
-                                      );
-
-    Assert.assertEquals(NestedDataTestUtils.expectSegmentGeneratorCanVectorize(segmentsName), allCanVectorize);
+                                          s -> {
+                                            final CursorHolder cursorHolder = s.asStorageAdapter()
+                                                                               .makeCursorHolder(spec);
+                                            final boolean canVectorize = cursorHolder.canVectorize();
+                                            cursorHolder.close();
+                                            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..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,13 +27,14 @@
 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;
+import org.apache.druid.segment.CursorHolder;
 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 +62,11 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException
         .setAggregatorSpecs(factory)
         .build();
     final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex());
-    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 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(
         dimensionSpec ->
             ColumnProcessors.makeVectorProcessor(
@@ -100,5 +97,6 @@ VectorGrouper makeGrouper()
     };
     iterator.close();
     Mockito.verify(grouperCaptor.getValue()).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 2d36ffa1e632..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,14 +24,13 @@
 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.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.CursorHolder;
 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;
@@ -190,47 +189,38 @@ public void test_asStorageAdapter_getDimensionCardinalityV()
   }
 
   @Test
-  public void test_asStorageAdapter_makeCursors()
+  public void test_asStorageAdapter_makeCursor()
   {
-    final Sequence cursors = LOOKUP_SEGMENT.asStorageAdapter().makeCursors(
-        null,
-        Intervals.of("1970/PT1H"),
-        VirtualColumns.EMPTY,
-        Granularities.ALL,
-        false,
-        null
-    );
-
-    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"))
+                                                     .build();
+    try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asStorageAdapter().makeCursorHolder(buildSpec)) {
+      final Cursor cursor = cursorHolder.asCursor();
+
+      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
   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/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/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java
index bb2826498c54..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
@@ -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/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
index 919f9fcff82d..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,8 +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.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;
@@ -37,7 +35,8 @@
 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.CursorHolder;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.column.TypeStrategy;
@@ -250,28 +249,21 @@ private void validateDecorated(
           },
           siggy
       );
-      final Sequence cursors = seggy
-          .asStorageAdapter()
-          .makeCursors(
-              filter,
-              interval == null ? Intervals.ETERNITY : interval,
-              VirtualColumns.EMPTY,
-              Granularities.ALL,
-              false,
-              null
-          );
-
-      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;
-          }
-      );
+      final CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder()
+                                                                            .setFilter(filter);
+      if (interval != null) {
+        builder.setInterval(interval);
+      }
+      try (final CursorHolder cursorHolder = seggy.asStorageAdapter().makeCursorHolder(builder.build())) {
+        final Cursor cursor = cursorHolder.asCursor();
+
+        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/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java
index 60b098b43bfb..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;
@@ -104,7 +105,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 +146,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 +185,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 5b155aa4a983..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;
@@ -42,6 +43,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 +67,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";
@@ -206,7 +208,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,
@@ -240,7 +242,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,
@@ -274,7 +276,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..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;
@@ -91,7 +92,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 +117,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 +148,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 +201,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 +222,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 +237,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 +269,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 +371,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..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;
@@ -535,7 +536,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 +608,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 +704,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 +777,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..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;
@@ -57,7 +59,7 @@ public void testSerialization() throws Exception
         0,
         0,
         3,
-        ScanQuery.Order.NONE,
+        Order.NONE,
         null,
         null,
         Arrays.asList("market", "quality", "index"),
@@ -85,7 +87,7 @@ public void testSerializationWithTimeOrder() throws Exception
         0,
         0,
         3,
-        ScanQuery.Order.ASCENDING,
+        Order.ASCENDING,
         null,
         null,
         Arrays.asList("market", "quality", "index", "__time"),
@@ -96,9 +98,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(OrderBy.ascending("__time")),
         expectedQuery.getOrderBys()
     );
   }
@@ -118,7 +120,7 @@ public void testSerializationWithOrderBy() throws Exception
         0,
         3,
         null,
-        Collections.singletonList(new ScanQuery.OrderBy("quality", ScanQuery.Order.ASCENDING)),
+        Collections.singletonList(OrderBy.ascending("quality")),
         null,
         Arrays.asList("market", "quality", "index", "__time"),
         null,
@@ -128,9 +130,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(OrderBy.ascending("quality")),
         expectedQuery.getOrderBys()
     );
   }
@@ -146,7 +148,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 b0d3432c0dbd..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
@@ -27,14 +27,22 @@
 import org.apache.druid.java.util.common.ISE;
 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.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;
 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.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;
@@ -47,7 +55,7 @@
 import java.util.List;
 import java.util.Set;
 
-public class ScanQueryTest
+public class ScanQueryTest extends InitializedNullHandlingTest
 {
   private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
   private static QuerySegmentSpec intervalSpec;
@@ -117,7 +125,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)
@@ -128,7 +136,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)
@@ -143,12 +151,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)
+                          OrderBy.ascending("__time"),
+                          OrderBy.descending("quality")
                       )
                   )
                   .columns(ImmutableList.of("__time", "quality"))
@@ -163,8 +171,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(OrderBy.ascending("__time")))
               .columns(ImmutableList.of("__time", "quality"))
               .dataSource("source")
               .intervals(intervalSpec)
@@ -176,9 +184,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"))
@@ -208,9 +216,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)
@@ -246,7 +254,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)
@@ -254,7 +262,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)
@@ -262,7 +270,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)
@@ -311,7 +319,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)
@@ -334,7 +342,7 @@ public void testGetResultOrderingWithTimeBasedOrderBy()
     final ScanQuery scanQuery =
         Druids.newScanQueryBuilder()
               .columns("__time")
-              .orderBy(Collections.singletonList(new ScanQuery.OrderBy("__time", ScanQuery.Order.DESCENDING)))
+              .orderBy(Collections.singletonList(OrderBy.descending("__time")))
               .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)
               .dataSource("some src")
               .intervals(intervalSpec)
@@ -350,7 +358,7 @@ public void testGetResultOrderingWithNonTimeOrderBy()
     final ScanQuery scanQuery =
         Druids.newScanQueryBuilder()
               .columns("quality")
-              .orderBy(Collections.singletonList(new ScanQuery.OrderBy("quality", ScanQuery.Order.ASCENDING)))
+              .orderBy(Collections.singletonList(OrderBy.ascending("quality")))
               .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)
               .dataSource("some src")
               .intervals(intervalSpec)
@@ -364,7 +372,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)
@@ -378,7 +386,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)
@@ -419,4 +427,26 @@ 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 = ScanQueryEngine.makeCursorBuildSpec(query, null);
+    Assert.assertEquals(QueryRunnerTestHelper.FIRST_TO_THIRD.getIntervals().get(0), buildSpec.getInterval());
+    Assert.assertNull(buildSpec.getGroupingColumns());
+    Assert.assertNull(buildSpec.getAggregators());
+    Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
+  }
 }
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..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;
@@ -52,7 +53,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 +90,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 +127,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 +162,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..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;
@@ -308,7 +309,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/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
index f555d81cefe5..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
@@ -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.CursorHolder;
 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 = TimeseriesQueryEngine.makeCursorBuildSpec(timeseriesQuery, null);
     boolean allCanVectorize = segments.stream()
                                       .allMatch(
-                                          s -> s.asStorageAdapter()
-                                                .canVectorize(
-                                                    filter,
-                                                    timeseriesQuery.getVirtualColumns(),
-                                                    timeseriesQuery.isDescending()
-                                                )
-                                      );
+                                          s -> {
+                                            final CursorHolder cursorHolder = s.asStorageAdapter()
+                                                                               .makeCursorHolder(spec);
+                                            final boolean canVectorize = cursorHolder.canVectorize();
+                                            cursorHolder.close();
+                                            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/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
index e8f10da8bf3d..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
@@ -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;
@@ -36,9 +41,10 @@
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 
 @RunWith(Parameterized.class)
-public class TimeseriesQueryTest
+public class TimeseriesQueryTest extends InitializedNullHandlingTest
 {
   private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
 
@@ -101,4 +107,99 @@ public void testGetRequiredColumns()
 
     Assert.assertEquals(ImmutableSet.of("__time", "fieldFromVirtualColumn", "aField"), query.getRequiredColumns());
   }
+
+  @Test
+  public void testAsCursorBuildSpecAllGranularity()
+  {
+    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.ALL)
+              .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 = TimeseriesQueryEngine.makeCursorBuildSpec(query, null);
+    Assert.assertEquals(QueryRunnerTestHelper.FULL_ON_INTERVAL, buildSpec.getInterval());
+    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());
+  }
+
+  @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 = TimeseriesQueryEngine.makeCursorBuildSpec(query, 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(query.getGranularity(), Granularities.GRANULARITY_VIRTUAL_COLUMN_NAME),
+            virtualColumns.getVirtualColumns()[0]
+        ),
+        buildSpec.getVirtualColumns()
+    );
+  }
 }
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..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
@@ -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.CursorHolder;
 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 CursorHolder makeCursorHolder(CursorBuildSpec spec)
       {
-        return null;
+        return new CursorHolder()
+        {
+          @Nullable
+          @Override
+          public Cursor asCursor()
+          {
+            return null;
+          }
+        };
       }
     };
 
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..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
@@ -6193,6 +6193,165 @@ 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);
+  }
+
+  @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/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
index 6d7f1a03be9a..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
@@ -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,65 @@ public void testGetRequiredColumns()
 
     Assert.assertEquals(ImmutableSet.of("__time", "other", "index"), query.getRequiredColumns());
   }
+
+  @Test
+  public void testAsCursorBuildSpecAllGranularity()
+  {
+    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.ALL)
+        .postAggregators(ImmutableList.of(new FieldAccessPostAggregator("x", "idx")))
+        .metric(new NumericTopNMetricSpec("idx"))
+        .threshold(100)
+        .build();
+
+    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());
+    Assert.assertEquals(virtualColumns, buildSpec.getVirtualColumns());
+  }
+
+  @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 = 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"),
+        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),
+            virtualColumns.getVirtualColumns()[0]
+        ),
+        buildSpec.getVirtualColumns()
+    );
+  }
 }
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 73a2b6db50a4..788181859ffa 100644
--- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java
@@ -26,9 +26,7 @@
 import org.apache.druid.data.input.impl.DimensionsSpec;
 import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.guice.BuiltInTypesModule;
-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;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.DimensionSpec;
@@ -46,7 +44,6 @@
 import org.junit.Test;
 
 import javax.annotation.Nonnull;
-import java.util.List;
 import java.util.Map;
 
 public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
@@ -151,58 +148,44 @@ 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
-    );
-    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 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();
+
+      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
@@ -218,88 +201,74 @@ 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
-    );
-    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 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();
+
+      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
@@ -315,87 +284,74 @@ 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
-    );
-    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 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();
+
+      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
@@ -411,57 +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.makeCursors(
-        null,
-        Intervals.ETERNITY,
-        VirtualColumns.EMPTY,
-        Granularities.NONE,
-        false,
-        null
-    );
-    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 CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      Cursor cursor = cursorHolder.asCursor();
+      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
@@ -477,50 +415,35 @@ 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
-    );
-    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 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();
+
+      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
@@ -536,57 +459,31 @@ 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
-    );
-    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 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();
+
+      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
@@ -621,49 +518,35 @@ 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
-    );
-    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 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();
+
+      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 359d2de58cb7..f13b6c5632a6 100644
--- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java
+++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java
@@ -26,9 +26,7 @@
 import org.apache.druid.data.input.impl.DimensionsSpec;
 import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.guice.BuiltInTypesModule;
-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;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.DimensionSpec;
@@ -45,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,56 +147,43 @@ 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
-    );
-    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 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();
+
+      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
@@ -215,85 +199,71 @@ 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
-    );
-    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 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();
+
+      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());
+      }
     }
   }
 
@@ -310,85 +280,71 @@ 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
-    );
-    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 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();
+
+      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());
+      }
     }
   }
 
@@ -405,56 +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.makeCursors(
-        null,
-        Intervals.ETERNITY,
-        VirtualColumns.EMPTY,
-        Granularities.NONE,
-        false,
-        null
-    );
-    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 CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+      Cursor cursor = cursorHolder.asCursor();
+      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
@@ -470,48 +404,34 @@ 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
-    );
-    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 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();
+
+      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
@@ -527,56 +447,30 @@ 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
-    );
-    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 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();
+
+      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/QueryableIndexCursorSequenceBuilderTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorHolderTest.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/QueryableIndexCursorHolderTest.java
index ea93e1b75c2e..349236a9dcf3 100644
--- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilderTest.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 QueryableIndexCursorSequenceBuilderTest
+public class QueryableIndexCursorHolderTest
 {
   @Test
   public void testTimeSearch()
@@ -66,52 +66,52 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset)
 
     Assert.assertEquals(
         0,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 0, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 0, 0, values.length)
     );
 
     Assert.assertEquals(
         2,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 0, 2, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 0, 2, values.length)
     );
 
     Assert.assertEquals(
         0,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 0, 0, values.length / 2)
+        QueryableIndexCursorHolder.timeSearch(column, 0, 0, values.length / 2)
     );
 
     Assert.assertEquals(
         1,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 0, values.length)
     );
 
     Assert.assertEquals(
         2,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 2, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 2, values.length)
     );
 
     Assert.assertEquals(
         1,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 0, values.length / 2)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 0, values.length / 2)
     );
 
     Assert.assertEquals(
         1,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 1, 1, 8)
+        QueryableIndexCursorHolder.timeSearch(column, 1, 1, 8)
     );
 
     Assert.assertEquals(
         8,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 2, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 2, 0, values.length)
     );
 
     Assert.assertEquals(
         10,
-        QueryableIndexCursorSequenceBuilder.timeSearch(column, 10, 0, values.length)
+        QueryableIndexCursorHolder.timeSearch(column, 10, 0, values.length)
     );
 
     Assert.assertEquals(
         11,
-        QueryableIndexCursorSequenceBuilder.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 ddb1cc419f33..4cde2a796585 100644
--- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
@@ -20,12 +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;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
@@ -74,18 +69,8 @@ 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
-            )
-        );
+        final VectorCursor cursor = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asVectorCursor();
 
         final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
@@ -96,17 +81,8 @@ 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 Yielder yielder = closer.register(Yielders.each(cursors));
-        final Cursor cursor = yielder.get();
+        final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN));
+        final Cursor cursor = cursorHolder.asCursor();
         final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
 
         qualitySelector =
@@ -250,18 +226,9 @@ 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 Yielder cursorYielder = Yielders.each(cursors);
-      cursor = cursorYielder.get();
+      final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN));
+      cursor = cursorHolder.asCursor();
       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 9822f6e5f332..dca89de18b12 100644
--- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java
@@ -19,18 +19,21 @@
 
 package org.apache.druid.segment;
 
+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;
-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;
+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;
@@ -38,16 +41,20 @@
 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;
 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;
@@ -78,10 +85,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;
@@ -98,11 +105,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()) {
@@ -173,7 +175,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 @@ -208,14 +210,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 ); } @@ -262,7 +302,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)); } @@ -331,7 +371,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)); } @@ -451,418 +491,447 @@ public void test_getMetadata() } @Test - public void test_makeCursors_filterOnLong() + public void test_makeCursor() { 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 - ); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + 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(1, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test - public void test_makeCursors_filterOnNonexistentColumnEqualsNull() + public void test_makeCursor_filterOnLong() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.makeCursors( - new SelectorDimFilter("nonexistent", null, null).toFilter(), - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("0"), - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test - public void test_makeCursors_filterOnVirtualColumn() + public void test_makeCursor_filterOnNonexistentColumnEqualsNull() { 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 - ); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("0"), + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test - public void test_makeCursors_descending() + public void test_makeCursor_filterOnVirtualColumn() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - final Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - true, - null - ); + final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("2"), - ImmutableList.of("1"), - ImmutableList.of("0") - ), - 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() + ) + ) + ) + ) + .build(); + + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } + @Test - public void test_makeCursors_intervalDoesNotMatch() + public void test_makeCursor_descending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.makeCursors( - null, - Intervals.of("2000/P1D"), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - Assert.assertEquals( - ImmutableList.of(), - walkCursors(cursors, READ_STRING) - ); + 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("2"), + ImmutableList.of("1"), + ImmutableList.of("0") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @Test - public void test_makeCursors_intervalPartiallyMatches() + public void test_makeCursor_intervalDoesNotMatch() { 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 - ); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("2000/P1D")) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertEquals( + ImmutableList.of(), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test - public void test_makeCursors_hourGranularity() + public void test_makeCursor_intervalPartiallyMatches() { - 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 RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - 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-01-01T01/PT1H")) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals(1, numCloses.get()); + Assert.assertEquals(2, numCloses.get()); } @Test - public void test_makeCursors_hourGranularityWithInterval() + public void test_makeCursor_hourGranularity() { 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 - ); - - 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/1971")) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + 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, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @Test - public void test_makeCursors_hourGranularityWithIntervalDescending() + public void test_makeCursor_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, - true, - null - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .build(); - 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) - ); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + + 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, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @Test - public void test_makeCursors_allProcessors() + public void test_makeCursor_hourGranularityWithIntervalDescending() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - - final Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); + final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - 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())) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setPreferredOrdering(Cursors.descendingTimeOrder()) + .build(); + + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + 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, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @Test - public void test_makeCursors_filterOnNonexistentColumnEqualsNonnull() + public void test_makeCursor_allProcessors() { 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 - ); - - Assert.assertEquals( - ImmutableList.of(), - walkCursors(cursors, new ArrayList<>(PROCESSORS.values())) - ); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = cursorHolder.asCursor(); + 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()); + Assert.assertEquals(2, numCloses.get()); } @Test - public void test_makeCursors_eternityIntervalWithMonthGranularity() + public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - Assert.assertThrows(IAE.class, () -> { - adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.MONTH, - false, - null + + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) + .build(); + + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + Assert.assertEquals( + ImmutableList.of(), + walkCursor(cursor, new ArrayList<>(PROCESSORS.values())) ); - }); + } + + Assert.assertEquals(2, numCloses.get()); } - 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<>(); + // test cursor reset + while (!cursor.isDone()) { + cursor.advanceUninterruptibly(); + } - for (Supplier supplier : suppliers) { - row.add(supplier.get()); - } + cursor.reset(); - retVal.add(row); - cursor.advanceUninterruptibly(); - } + while (!cursor.isDone()) { + final List row = new ArrayList<>(); - return Sequences.simple(retVal); - } - ).toList(); + 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, + final CursorBuildSpec buildSpec, + final Granularity granularity, + final List>> processors + ) + { + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + granularity, + buildSpec.getInterval(), + Cursors.preferDescendingTimeOrdering(buildSpec) + ); + + 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..8950693f14fa 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 CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return null; + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + 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 b467bc6c9386..8ea5791463fc 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.guava.Sequences; 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; @@ -38,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; @@ -57,7 +56,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,10 +63,12 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.IOException; +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; @@ -233,26 +233,24 @@ public void test_unnest_adapter_column_capabilities() @Test public void test_unnest_adapters_basic() { + try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER.makeCursors( - null, - UNNEST_STORAGE_ADAPTER.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); int count = 0; + + List rows = new ArrayList<>(); + // test cursor reset + while (!cursor.isDone()) { + cursor.advance(); + } + cursor.reset(); + while (!cursor.isDone()) { Object dimSelectorVal = dimSelector.getObject(); - if (dimSelectorVal == null) { - Assert.assertNull(dimSelectorVal); - } + rows.add(dimSelectorVal); cursor.advance(); count++; } @@ -261,25 +259,115 @@ public void test_unnest_adapters_basic() unnest 2 rows -> 16 rows after unnest */ Assert.assertEquals(count, 16); - return null; - }); + 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() + { + try (final CursorHolder cursorHolder = UNNEST_ARRAYS.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(); + + while (!cursor.isDone()) { + Object dimSelectorVal = dimSelector.getObject(); + rows.add(dimSelectorVal); + cursor.advance(); + count++; + } + Assert.assertEquals(count, 12); + Assert.assertEquals( + Arrays.asList(2L, 3L, 1L, null, 3L, 1L, null, 2L, 9L, 1L, 2L, 3L), + rows + ); + } } @Test - public void test_two_levels_of_unnest_adapters() + public void test_unnest_adapters_basic_row_based_array_column() { - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER1.makeCursors( - null, - UNNEST_STORAGE_ADAPTER1.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, + 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 ("a".equals(columnName)) { + 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(); + while (!cursor.isDone()) { + Object dimSelectorVal = dimSelector.getObject(); + rows.add(dimSelectorVal); + cursor.advance(); + count++; + } + Assert.assertEquals(count, 11); + Assert.assertEquals( + Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, null, 7L, 8L, 9L, 10L), + rows + ); + } + } - cursorSequence.accumulate(null, (accumulated, cursor) -> { + @Test + public void test_two_levels_of_unnest_adapters() + { + try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME1)); @@ -305,8 +393,7 @@ public void test_two_levels_of_unnest_adapters() */ Assert.assertEquals(count, 128); Assert.assertEquals(dimSelector.getValueCardinality(), 17); - return null; - }); + } } @Test @@ -332,26 +419,20 @@ 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 TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(baseFilter) + .build(); + try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + 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(); // OR-case so base filter should match the postJoinFilter Assert.assertEquals(baseFilter, postFilter); - return null; - }); + } } @Test @@ -383,27 +464,22 @@ 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 TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(baseFilter) + .build(); + try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + 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(); // OR-case so base filter should match the postJoinFilter Assert.assertEquals(baseFilter, postFilter); - return null; - }); + } } + @Test public void test_nested_filters_unnested_and_topLevel1And3filtersInOR() { @@ -421,6 +497,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() { @@ -444,6 +521,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() { @@ -470,6 +548,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() { @@ -487,6 +566,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() { @@ -652,7 +732,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(); @@ -662,21 +742,12 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() final Filter expectedPushDownFilter = selector(inputColumn, "1"); + try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); - final Sequence cursorSequence = unnestStorageAdapter.makeCursors( - null, - unnestStorageAdapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - - 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); @@ -687,8 +758,7 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() count++; } Assert.assertEquals(1, count); - return null; - }); + } } @@ -709,20 +779,16 @@ 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 CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(queryFilter) + .build(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + 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(queryFilter, postFilter); @@ -733,8 +799,7 @@ public void test_pushdown_filters_unnested_dimension_outside() count++; } Assert.assertEquals(1, count); - return null; - }); + } } @Test @@ -761,16 +826,9 @@ 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 - ); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); @@ -788,9 +846,7 @@ public void testUnnestValueMatcherValueDoesntExist() count++; } Assert.assertEquals(count, 618); - return null; - }); - + } } public void testComputeBaseAndPostUnnestFilters( @@ -866,16 +922,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 CursorHolder makeCursorHolder(CursorBuildSpec spec) { - this.pushDownFilter = filter; - return super.makeCursors(filter, interval, virtualColumns, gran, descending, queryMetrics); + this.pushDownFilter = spec.getFilter(); + 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 179e4e96d10c..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 @@ -44,16 +44,14 @@ import org.apache.druid.guice.BuiltInTypesModule; 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; -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; 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 +68,8 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; @@ -416,11 +416,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 +449,40 @@ public void setUp() throws Exception { BuiltInTypesModule.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); } @@ -516,7 +526,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -544,7 +554,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -573,7 +583,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -652,7 +662,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -675,7 +685,7 @@ public static Collection makeConstructors() new IncrementalIndexSchema( schema.getMinTimestamp(), schema.getTimestampSpec(), - schema.getGran(), + schema.getQueryGranularity(), schema.getVirtualColumns(), schema.getDimensionsSpec().withDimensions( schema.getDimensionsSpec() @@ -780,29 +790,32 @@ private DimFilter maybeOptimize(final DimFilter dimFilter) return optimize ? dimFilter.optimize(false) : dimFilter; } - private Sequence makeCursorSequence(final Filter filter) + private CursorBuildSpec makeCursorBuildSpec(@Nullable Filter filter) { - return adapter.makeCursors( - filter, - Intervals.ETERNITY, - VIRTUAL_COLUMNS, - Granularities.ALL, - false, - null - ); + return CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(VIRTUAL_COLUMNS) + .build(); + } - private VectorCursor makeVectorCursor(final Filter filter) + private CursorBuildSpec makeVectorCursorBuildSpec(@Nullable Filter filter) { + return CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(virtualColumns) + .setQueryContext( + QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) + ) + ) + .build(); + } - 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 - ); + private VectorCursor makeVectorCursor(final Filter filter) + { + final CursorBuildSpec buildSpec = makeVectorCursorBuildSpec(filter); + return adapter.makeCursorHolder(buildSpec).asVectorCursor(); } /** @@ -810,48 +823,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 CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(makeFilter(filter)))) { + final Cursor cursor = cursorHolder.asCursor(); + 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 CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(null))) { + final Cursor cursor = cursorHolder.asCursor(); + 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) @@ -862,7 +866,9 @@ private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFil dimFilter ); - try (final VectorCursor cursor = makeVectorCursor(null)) { + + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(null))) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), maybeOptimize(dimFilter) @@ -928,27 +934,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 CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(postFilteringFilter))) { + final Cursor cursor = cursorHolder.asCursor(); + 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( @@ -992,7 +994,8 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - try (final VectorCursor cursor = makeVectorCursor(postFilteringFilter)) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(postFilteringFilter))) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1016,7 +1019,8 @@ private List selectColumnValuesMatchingFilterUsingVectorCursor( final String selectColumn ) { - try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1042,7 +1046,8 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur ) { final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); - try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { + 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<>(); @@ -1240,4 +1245,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/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/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 09f78e066fb2..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 @@ -50,6 +50,7 @@ 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.topn.TopNQueryBuilder; @@ -58,9 +59,11 @@ 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.CursorHolder; +import org.apache.druid.segment.Cursors; 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; @@ -83,7 +86,6 @@ import java.util.Collections; import java.util.List; import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; /** */ @@ -131,34 +133,34 @@ 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 = GroupingEngine.makeCursorBuildSpec(query, 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 Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); final Sequence rows = GroupByQueryEngine.process( query, - new IncrementalIndexStorageAdapter(index), + adapter, + cursorHolder, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), - new DruidProcessingConfig(), - filter, - interval, - null + new DruidProcessingConfig() ); final List results = rows.toList(); @@ -195,46 +197,46 @@ 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 = GroupingEngine.makeCursorBuildSpec(query, 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 Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); final Sequence rows = GroupByQueryEngine.process( query, - new IncrementalIndexStorageAdapter(index), + adapter, + cursorHolder, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), - new DruidProcessingConfig(), - filter, - interval, - null + new DruidProcessingConfig() ); final List results = rows.toList(); @@ -278,38 +280,38 @@ 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 - ); - - 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") - ) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorFilter("sally", "bo")) + .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("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))); + } } } @@ -374,36 +376,36 @@ 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 = GroupingEngine.makeCursorBuildSpec(query, 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 Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); - final Sequence rows = GroupByQueryEngine.process( query, - new IncrementalIndexStorageAdapter(index), + adapter, + cursorHolder, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), - new DruidProcessingConfig(), - filter, - interval, - null + new DruidProcessingConfig() ); final List results = rows.toList(); @@ -433,48 +435,40 @@ 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 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()); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) + .build(); + try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + 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 @@ -497,37 +491,26 @@ 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 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()); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new DictionaryRaceTestFilter(index, timestamp)) + .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) + .build(); + try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + 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 @@ -548,93 +531,86 @@ 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 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()); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) + .build(); + try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + Cursor cursor = cursorHolder.asCursor(); + + 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/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java index e5ab7d1be8aa..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 @@ -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#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 5931bff83599..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 @@ -26,13 +26,13 @@ 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; 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.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; @@ -296,7 +296,7 @@ public void test_getMetadata_factToCountry() } @Test - public void test_makeCursors_factToCountryLeft() + public void test_makeCursor_factToCountryLeft() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); @@ -305,19 +305,12 @@ public void test_makeCursors_factToCountryLeft() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -360,7 +353,7 @@ public void test_makeCursors_factToCountryLeft() } @Test - public void test_makeCursors_factToCountryLeftUsingLookup() + public void test_makeCursor_factToCountryLeftUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); @@ -369,19 +362,12 @@ public void test_makeCursors_factToCountryLeftUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -423,7 +409,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( @@ -431,19 +417,12 @@ public void test_makeCursors_factToCountryInner() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -480,7 +459,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( @@ -488,19 +467,12 @@ public void test_makeCursors_factToCountryInnerUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -536,7 +508,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). @@ -547,18 +519,13 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -601,7 +568,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). @@ -612,18 +579,13 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -664,7 +626,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(); @@ -673,18 +635,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -700,7 +657,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(); @@ -709,18 +666,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -735,7 +687,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(); @@ -744,27 +696,23 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + 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} @@ -773,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(); @@ -782,26 +730,22 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + 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"} @@ -810,7 +754,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(); @@ -819,27 +763,23 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + 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} @@ -848,7 +788,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(); @@ -857,26 +797,22 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + 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"} @@ -885,7 +821,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( @@ -899,18 +835,13 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -927,7 +858,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( @@ -941,18 +872,13 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -968,7 +894,7 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() } @Test - public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() + public void test_makeCursor_factToCountryLeftWithFilterOnJoinable() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); @@ -983,18 +909,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1012,7 +933,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( @@ -1025,18 +946,13 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1052,7 +968,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. @@ -1080,18 +996,13 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1129,7 +1040,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. @@ -1156,18 +1067,13 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1204,7 +1110,7 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit } @Test - public void test_makeCursors_factToRegionToCountryLeft() + public void test_makeCursor_factToRegionToCountryLeft() { List joinableClauses = ImmutableList.of( factToRegion(JoinType.LEFT), @@ -1215,19 +1121,12 @@ public void test_makeCursors_factToRegionToCountryLeft() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1268,7 +1167,7 @@ public void test_makeCursors_factToRegionToCountryLeft() } @Test - public void test_makeCursors_factToRegionToCountryInnerIncludeNull() + public void test_makeCursor_factToRegionToCountryInnerIncludeNull() { List joinableClauses = ImmutableList.of( factToRegionIncludeNull(JoinType.INNER), @@ -1279,19 +1178,12 @@ public void test_makeCursors_factToRegionToCountryInnerIncludeNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1331,7 +1223,7 @@ public void test_makeCursors_factToRegionToCountryInnerIncludeNull() } @Test - public void test_makeCursors_factToCountryAlwaysTrue() + public void test_makeCursor_factToCountryAlwaysTrue() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1352,23 +1244,19 @@ public void test_makeCursors_factToCountryAlwaysTrue() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + adapter.makeCursorHolder(buildSpec), + columns, ImmutableList.of( new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, @@ -1393,7 +1281,7 @@ public void test_makeCursors_factToCountryAlwaysTrue() } @Test - public void test_makeCursors_factToCountryAlwaysFalse() + public void test_makeCursor_factToCountryAlwaysFalse() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1416,18 +1304,13 @@ public void test_makeCursors_factToCountryAlwaysFalse() VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1440,7 +1323,7 @@ public void test_makeCursors_factToCountryAlwaysFalse() } @Test - public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() + public void test_makeCursor_factToCountryAlwaysTrueUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1462,23 +1345,19 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + adapter.makeCursorHolder(buildSpec), + columns, ImmutableList.of( new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, @@ -1503,7 +1382,7 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() } @Test - public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() + public void test_makeCursor_factToCountryAlwaysFalseUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1526,18 +1405,13 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1550,7 +1424,7 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() } @Test - public void test_makeCursors_factToCountryUsingVirtualColumn() + public void test_makeCursor_factToCountryUsingVirtualColumn() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1576,18 +1450,13 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() joinableClauses, virtualColumns ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build() ), ImmutableList.of( "page", @@ -1606,7 +1475,7 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() } @Test - public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() + public void test_makeCursor_factToCountryUsingVirtualColumnUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1632,18 +1501,13 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() joinableClauses, virtualColumns ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build() ), ImmutableList.of( "page", @@ -1662,7 +1526,7 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() } @Test - public void test_makeCursors_factToCountryUsingExpression() + public void test_makeCursor_factToCountryUsingExpression() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1685,19 +1549,12 @@ public void test_makeCursors_factToCountryUsingExpression() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1714,7 +1571,7 @@ public void test_makeCursors_factToCountryUsingExpression() } @Test - public void test_makeCursors_factToCountryUsingExpressionUsingLookup() + public void test_makeCursor_factToCountryUsingExpressionUsingLookup() { List joinableClauses = ImmutableList.of( new JoinableClause( @@ -1737,19 +1594,12 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1766,7 +1616,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( @@ -1791,26 +1641,22 @@ public void test_makeCursors_factToRegionTheWrongWay() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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.verifyCursor( + 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"}, @@ -1823,7 +1669,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"); @@ -1847,25 +1693,18 @@ public void test_makeCursors_errorOnNonEquiJoin() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @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"); @@ -1889,25 +1728,18 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @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: " @@ -1931,25 +1763,18 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @Test - public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() + public void test_makeCursor_errorOnNonKeyBasedJoinUsingLookup() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage( @@ -1973,25 +1798,18 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @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)); @@ -2001,18 +1819,13 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2026,7 +1839,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)); @@ -2035,18 +1848,13 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2059,7 +1867,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"); @@ -2073,18 +1881,13 @@ public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowIS factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + ).makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ); } @Test - public void test_makeCursors_factToCountryLeftWithBaseFilter() + public void test_makeCursor_factToCountryLeftWithBaseFilter() { final Filter baseFilter = Filters.or(Arrays.asList( new SelectorDimFilter("countryIsoCode", "CA", null).toFilter(), @@ -2098,20 +1901,13 @@ public void test_makeCursors_factToCountryLeftWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -2129,7 +1925,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(), @@ -2142,20 +1938,13 @@ public void test_makeCursors_factToCountryInnerWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), + ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -2172,7 +1961,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(), @@ -2185,27 +1974,22 @@ public void test_makeCursors_factToCountryRightWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - baseFilter, - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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" - ), + 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.verifyCursor( + adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), + columns, ImmutableList.of( new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, @@ -2232,7 +2016,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(), @@ -2245,27 +2029,22 @@ public void test_makeCursors_factToCountryFullWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - baseFilter, - joinableClauses, - joinFilterPreAnalysis - ).makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - 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" - ), + 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.verifyCursor( + adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), + columns, ImmutableList.of( new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, 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 68a77f38884e..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 @@ -24,9 +24,7 @@ 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; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.expression.TestExprMacroTable; @@ -34,6 +32,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; @@ -82,14 +81,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -159,14 +153,9 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -215,14 +204,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -282,14 +266,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -353,14 +332,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -423,14 +397,12 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder() + .setFilter(originalFilter) + .setVirtualColumns(virtualColumns) + .build() ), ImmutableList.of( "page", @@ -497,14 +469,12 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - virtualColumns, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder() + .setFilter(originalFilter) + .setVirtualColumns(virtualColumns) + .build() ), ImmutableList.of( "page", @@ -601,14 +571,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -728,14 +693,9 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -822,14 +782,9 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan joinableClauses, joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -872,14 +827,9 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -974,14 +924,9 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1064,14 +1009,9 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1138,14 +1078,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1196,14 +1131,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1253,14 +1183,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1309,14 +1234,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1366,14 +1286,9 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1423,14 +1338,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 + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1487,14 +1397,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 + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1548,14 +1453,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1603,14 +1503,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1657,14 +1552,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1715,14 +1605,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( "page", @@ -1772,14 +1657,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1828,14 +1708,9 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1895,14 +1770,9 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -1966,14 +1836,9 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2047,14 +1912,9 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR + "Equality{leftExpr=user, rightColumn='regionName', includeNull=false}" ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2108,14 +1968,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2188,14 +2043,9 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2391,14 +2241,9 @@ public boolean supportsRequiredColumnRewrite() expectedVirtualColumns = ImmutableSet.of(); } - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2525,14 +2370,9 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", @@ -2605,14 +2445,9 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + JoinTestHelper.verifyCursor( + adapter.makeCursorHolder( + CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( "page", 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 86d614cf4aca..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 @@ -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.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; @@ -325,54 +324,70 @@ public static RowBasedIndexedTable> createRegionsIndexedTabl ); } - public static List readCursors(final Sequence cursors, final List columns) + public static List readCursor(final CursorHolder cursorHolder, 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<>(); - boolean interruptible = false; // test both advance() and advanceUninterruptibly() - - 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; - } + 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(); + + while (!cursor.isDone()) { + final Object[] row = new Object[columns.size()]; - return Sequences.simple(rows); + for (int i = 0; i < row.length; i++) { + row[i] = readers.get(i).get(); } - ).toList(); + + rows.add(row); + if (interruptible) { + cursor.advance(); + } else { + cursor.advanceUninterruptibly(); + } + + interruptible = !interruptible; + } + + return rows; + } + finally { + cursorHolder.close(); + } } - public static void verifyCursors( - final Sequence cursors, + + public static void verifyCursor( + final CursorHolder cursorHolder, final List columns, final List expectedRows ) { - final List rows = readCursors(cursors, 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 6813c04bb10f..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 @@ -20,18 +20,14 @@ package org.apache.druid.segment.join; 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.OrderBy; 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.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -39,8 +35,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.joda.time.Interval; import org.junit.Test; import javax.annotation.Nullable; @@ -76,17 +70,31 @@ public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return super.makeCursors(filter, interval, virtualColumns, gran, descending, queryMetrics) - .map(cursor -> new CursorNoAdvance(cursor, countDownLatch)); + final CursorHolder delegate = super.makeCursorHolder(spec); + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + return new CursorNoAdvance(delegate.asCursor(), countDownLatch); + } + + @Nullable + @Override + public List getOrdering() + { + return delegate.getOrdering(); + } + + @Override + public void close() + { + delegate.close(); + } + }; } private static class CursorNoAdvance implements Cursor @@ -106,12 +114,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return cursor.getColumnSelectorFactory(); } - @Override - public DateTime getTime() - { - return cursor.getTime(); - } - @Override public void advance() { @@ -142,7 +144,7 @@ public boolean isDoneOrInterrupted() @Override public void reset() { - + cursor.reset(); } } } @@ -235,30 +237,25 @@ public void makeCursorAndAdvance() joinFilterPreAnalysis ); - Cursor cursor = Iterables.getOnlyElement(hashJoinSegmentStorageAdapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ).toList()); - - ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() - { - @Override - public boolean matches(boolean includeUnknown) - { - return false; - } + try (final CursorHolder cursorHolder = hashJoinSegmentStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() { + @Override + public boolean matches(boolean includeUnknown) + { + return false; + } - } - }); + @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 e46b7893a0de..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 @@ -23,11 +23,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.error.DruidException; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.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; -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 +31,8 @@ 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.CursorHolder; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; @@ -347,17 +345,11 @@ 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 Yielder yielder = Yielders.each(cursorSequence); - closer.register(yielder); - final Cursor cursor = yielder.get(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(virtualColumns) + .build(); + final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); + final Cursor cursor = cursorHolder.asCursor(); return cursor.getColumnSelectorFactory(); } @@ -378,14 +370,8 @@ 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 = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor(); 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..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,26 +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; - 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(); } @@ -87,12 +84,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 b6b9713682a8..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 @@ -28,9 +28,7 @@ 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; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; @@ -45,6 +43,8 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -138,17 +138,9 @@ 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 - ); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); ExpressionPlan plan = ExpressionPlanner.plan( adapter, @@ -202,9 +194,7 @@ public void test_single_value_string_bindings() cursor.advance(); } - - return null; - }); + } } } @@ -213,16 +203,8 @@ 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 - ); - - cursorSequence.accumulate(null, (ignored, cursor) -> { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // identifier, uses dimension selector supplier supplier, no null coercion @@ -289,8 +271,7 @@ public void test_multi_value_string_bindings() cursor.advance(); } - return ignored; - }); + } } } @@ -299,16 +280,8 @@ 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 - ); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans ExpressionPlan plan = ExpressionPlanner.plan( @@ -343,9 +316,7 @@ public void test_long_bindings() } cursor.advance(); } - - return null; - }); + } } } @@ -354,17 +325,8 @@ 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 - ); - - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans ExpressionPlan plan = ExpressionPlanner.plan( @@ -399,9 +361,7 @@ public void test_double_bindings() } cursor.advance(); } - - return null; - }); + } } } @@ -684,16 +644,8 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept ); IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - - Sequence cursors = adapter.makeCursors( - null, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - int rowsProcessed = cursors.map(cursor -> { + 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()), @@ -722,10 +674,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/ExpressionVectorSelectorsCastTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java index 6f733ee0b91c..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 @@ -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,10 @@ 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) + .build(); + 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 6d27439f8a61..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 @@ -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; @@ -36,6 +35,8 @@ import org.apache.druid.query.groupby.epinephelinae.vector.GroupByVectorColumnSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; @@ -234,109 +235,101 @@ public static void sanityTestVectorizedExpressionSelectors( ) ); final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); - VectorCursor cursor = storageAdapter.makeVectorCursor( - null, - index.getDataInterval(), - virtualColumns, - false, - 512, - null - ); - - 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])); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setVirtualColumns(virtualColumns) + .build(); + try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) { + final VectorCursor cursor = cursorHolder.asVectorCursor(); + Assert.assertNotNull(cursor); + + 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: + Assert.assertNotNull(selector); 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: + Assert.assertNotNull(selector); + // 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; + default: + Assert.assertNotNull(objectSelector); + 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); - - Sequence cursors = new QueryableIndexStorageAdapter(index).makeCursors( - null, - index.getDataInterval(), - virtualColumns, - Granularities.ALL, - false, - null - ); - 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); + final Cursor nonVectorized = cursorHolder.asCursor(); + Assert.assertNotNull(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(); + } + + Assert.assertTrue(rows > 0); + Assert.assertEquals(rows, rowCount); + } } private static void verifyGroupBySelector( 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..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,6 +35,7 @@ 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; @@ -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/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index c58c686074d0..14eb3d44220d 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.CursorHolder; 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 CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return null; + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + return null; + } + }; } }; 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/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 3c663e22488f..3f8cfcb94a45 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; @@ -71,13 +70,14 @@ 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.CursorHolder; 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; @@ -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; @@ -197,7 +196,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 +225,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,77 +283,74 @@ 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 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)) + .build(); - withOutputStream( - new Function() - { - @Override - public Object apply(final OutputStream out) + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + if (cursor == null) { + return; + } + + 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; - } - } - ); + cursor.advance(); + } + + return null; + } + }, + outputFileName + ); + } } @VisibleForTesting @@ -687,9 +693,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()) { @@ -707,11 +714,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 75c8bb6b143f..0ca08523fb2a 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(BuiltInTypesModule.getJacksonModulesList()); + mapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), mapper) + .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, 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 { 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..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; @@ -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..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.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.CalciteCatalogIngestionDmlTest.CatalogIngestionDmlComponentSupplier; @@ -581,8 +581,8 @@ public void testInsertTableWithClusteringWithClusteringFromCatalog() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("b", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("d", ScanQuery.Order.ASCENDING) + OrderBy.ascending("b"), + OrderBy.ascending("d") ) ) // Scan query lists columns in alphabetical order independent of the @@ -643,7 +643,7 @@ public void testInsertTableWithClusteringWithClusteringFromQuery() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("b", ScanQuery.Order.ASCENDING) + OrderBy.ascending("b") ) ) // Scan query lists columns in alphabetical order independent of the @@ -708,7 +708,7 @@ public void testInsertTableWithClusteringWithClusteringOnNewColumnFromQuery() ) .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("e", ScanQuery.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 ba7b19311ce9..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,6 +33,7 @@ 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; @@ -727,9 +728,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) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context( @@ -930,9 +931,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) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -971,9 +972,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) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -1021,8 +1022,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) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1") ) ) .context(queryContextWithGranularity(Granularities.DAY)) @@ -1210,9 +1211,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) + OrderBy.ascending("v0"), + OrderBy.ascending("dim1"), + OrderBy.ascending("v1") ) ) .context( @@ -1325,8 +1326,8 @@ public void testInsertFromExternalProjectSort() .columns("v0", "z") .orderBy( ImmutableList.of( - new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("z", ScanQuery.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/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 599f09d3a034..546776575f28 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; @@ -5047,7 +5048,7 @@ public void testGroupByJoinAsNativeQueryWithUnoptimizedFilter(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()