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

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

- * Advancing this {@link VectorCursor} to the end or explicitly calling {@link VectorCursor#close()} will - * automatically close all resources associated with this {@link CursorMaker} so calling {@link #cleanup()} is not - * needed. */ @Nullable default VectorCursor makeVectorCursor() @@ -52,8 +43,7 @@ default VectorCursor makeVectorCursor() } /** - * Returns true if this {@link CursorMaker} supports creating vectorized selectors. This operation may acquire - * underlying resources, so calling {@link #cleanup()} is necessary if no cursors are created and consumed. + * Returns true if this {@link CursorMaker} supports calling {@link #makeVectorCursor()}. */ default boolean canVectorize() { @@ -61,12 +51,12 @@ default boolean canVectorize() } /** - * Release any resources acquired if cursors are not consumed. Typically consuming a cursor or vector cursor releases - * the resources upon completion, but if for some reason this will not happen, this method must be called. + * Release any resources acquired by cursors. */ - default void cleanup() + @Override + default void close() { - // nothing to cleanup + // nothing to close } CursorMaker EMPTY = new CursorMaker() @@ -78,9 +68,9 @@ public boolean canVectorize() } @Override - public Sequence makeCursors() + public Cursor makeCursor() { - return Sequences.empty(); + return null; } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index e2f8ecadb681..6275a8fea9fd 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -20,9 +20,6 @@ package org.apache.druid.segment; import com.google.common.collect.ImmutableList; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -62,19 +59,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) return baseStorageAdapter.asCursorMaker(buildSpecBuilder.build()); } - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Override public Interval getInterval() { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java index b0ce54d66d08..9bfda8a20cd3 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorMaker.java @@ -20,17 +20,12 @@ package org.apache.druid.segment; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.BaseQuery; @@ -65,7 +60,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.List; -import java.util.Objects; import java.util.concurrent.TimeUnit; public class QueryableIndexCursorMaker implements CursorMaker @@ -140,7 +134,7 @@ public boolean canVectorize() } @Override - public Sequence makeCursors() + public Cursor makeCursor() { if (metrics != null) { metrics.vectorized(false); @@ -163,84 +157,59 @@ public Sequence makeCursors() baseOffset = BitmapOffset.of(filterBundle.getIndex().getBitmap(), descending, index.getNumRows()); } - Iterable iterable = gran.getIterable(interval); + final long timeStart = Math.max(interval.getStartMillis(), minDataTimestamp); + final long timeEnd = interval.getEndMillis(); + if (descending) { - iterable = Lists.reverse(ImmutableList.copyOf(iterable)); - } - - return Sequences.filter( - Sequences.withBaggage( - Sequences.map( - Sequences.simple(iterable), - new Function() - { - @Override - public Cursor apply(final Interval inputInterval) - { - final long timeStart = Math.max(interval.getStartMillis(), inputInterval.getStartMillis()); - final long timeEnd = Math.min( - interval.getEndMillis(), - gran.increment(inputInterval.getStartMillis()) - ); - - if (descending) { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { - break; - } - } - } else { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { - break; - } - } - } - - final Offset offset = descending ? - new DescendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeStart, - minDataTimestamp >= timeStart - ) : - new AscendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeEnd, - maxDataTimestamp < timeEnd - ); - - - final Offset baseCursorOffset = offset.clone(); - final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( - virtualColumns, - descending, - baseCursorOffset.getBaseReadableOffset(), - columnCache - ); - final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis()); - // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter - // needs to use a value matcher - if (filterBundle != null && filterBundle.getMatcherBundle() != null) { - final ValueMatcher matcher = filterBundle.getMatcherBundle() - .valueMatcher( - columnSelectorFactory, - baseCursorOffset, - descending - ); - final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); - return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, myBucket); - } else { - return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket); - } - } - } - ), - resources - ), - Objects::nonNull + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { + break; + } + } + } else { + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { + break; + } + } + } + + final Offset offset = descending ? + new DescendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeStart, + minDataTimestamp >= timeStart + ) : + new AscendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeEnd, + maxDataTimestamp < timeEnd + ); + + + final Offset baseCursorOffset = offset.clone(); + final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( + virtualColumns, + descending, + baseCursorOffset.getBaseReadableOffset(), + columnCache ); + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter + // needs to use a value matcher + if (filterBundle != null && filterBundle.getMatcherBundle() != null) { + final ValueMatcher matcher = filterBundle.getMatcherBundle() + .valueMatcher( + columnSelectorFactory, + baseCursorOffset, + descending + ); + final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); + return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, DateTimes.utc(timeStart)); + } else { + return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, DateTimes.utc(timeStart)); + } } @Nullable @@ -260,7 +229,7 @@ public VectorCursor makeVectorCursor() // sanity check if (!canVectorize()) { - cleanup(); + close(); throw new IllegalStateException("canVectorize()"); } if (metrics != null) { @@ -332,7 +301,7 @@ public VectorCursor makeVectorCursor() } @Override - public void cleanup() + public void close() { CloseableUtils.closeAndWrapExceptions(resourcesSupplier.get()); } @@ -539,12 +508,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return bucketStart; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 810f59cb4dbe..7d13e57e6531 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -21,9 +21,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -32,7 +29,6 @@ import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -189,56 +185,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) ); } - @Override - public boolean canVectorize( - @Nullable final Filter filter, - final VirtualColumns virtualColumns, - final boolean descending - ) - { - // For safety, this uses the old-school canVectorize implementation instead of delegating to the CursorMaker, - // because QueryableIndexCursorMaker expects to make cursors one way or another and so opens stuff that must be - // cleaned if for some reason a cursor or vector cursor is not constructed - if (filter != null) { - - final boolean filterCanVectorize = filter.canVectorizeMatcher(this); - - if (!filterCanVectorize) { - return false; - } - } - - // vector cursors can't iterate backwards yet - return !descending; - } - - @Override - @Nullable - public VectorCursor makeVectorCursor( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final boolean descending, - final int vectorSize, - @Nullable final QueryMetrics queryMetrics - ) - { - return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Override public Metadata getMetadata() { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index f843fe631311..d591e0cd11d2 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -91,12 +91,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return cursorTime; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index d8b16cf900af..2092b17f428d 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -20,7 +20,6 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; @@ -29,12 +28,12 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.SimpleSequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -172,15 +171,17 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) { return new CursorMaker() { + final Closer closer = Closer.create(); + @Override - public Sequence makeCursors() + public Cursor makeCursor() { final Granularity gran = spec.getGranularity(); final Interval actualInterval = spec.getInterval() .overlap(new Interval(getMinTime(), gran.bucketEnd(getMaxTime()))); if (actualInterval == null) { - return Sequences.empty(); + return null; } if (!isQueryGranularityAllowed(actualInterval, gran)) { @@ -195,42 +196,27 @@ public Sequence makeCursors() spec.isDescending() ? reverse(rowSequence) : rowSequence, rowAdapter ); + closer.register(rowWalker); + return new RowBasedCursor<>( + rowWalker, + rowAdapter, + spec.getFilter(), + actualInterval, + spec.getVirtualColumns(), + gran, + spec.isDescending(), + rowSignature + ); + } - final Iterable bucketIntervals = gran.getIterable(actualInterval); - - return Sequences.simple( - Iterables.transform( - spec.isDescending() ? reverse(bucketIntervals) : bucketIntervals, - bucketInterval -> - (Cursor) new RowBasedCursor<>( - rowWalker, - rowAdapter, - spec.getFilter(), - bucketInterval, - spec.getVirtualColumns(), - gran, - spec.isDescending(), - rowSignature - ) - ) - ).withBaggage(rowWalker::close); + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); } }; } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval queryInterval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable final QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, queryInterval, virtualColumns, gran, descending, queryMetrics); - } - /** * Reverse a Sequence. * diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index 5404937f5196..8d3ab329e032 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -24,7 +24,6 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Arrays; @@ -210,12 +209,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 4d4aeaf70465..86874a5140a1 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -259,12 +258,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index 4400dcd52267..59923c81db54 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -23,10 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.QueryMetrics; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.filter.BooleanFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.EqualityFilter; @@ -49,6 +46,7 @@ import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.join.PostJoinCursor; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -102,67 +100,55 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) input, input == null ? null : spec.getVirtualColumns().getColumnCapabilitiesWithFallback(baseAdapter, input) ); - final CursorBuildSpec unnestBuildSpec = CursorBuildSpec.builder(spec) - .setFilter(filterPair.lhs) - .setVirtualColumns( - VirtualColumns.create( - Collections.singletonList(unnestColumn) - ) - ) - .build(); + final CursorBuildSpec unnestBuildSpec = + CursorBuildSpec.builder(spec) + .setFilter(filterPair.lhs) + .setVirtualColumns(VirtualColumns.create(Collections.singletonList(unnestColumn))) + .build(); + return new CursorMaker() { + final Closer closer = Closer.create(); @Override - public Sequence makeCursors() + public Cursor makeCursor() { - final Sequence baseCursorSequence = baseAdapter.asCursorMaker(unnestBuildSpec).makeCursors(); - return Sequences.map( - baseCursorSequence, - cursor -> { - Objects.requireNonNull(cursor); - final ColumnCapabilities capabilities = unnestColumn.capabilities( - cursor.getColumnSelectorFactory(), - unnestColumn.getOutputName() - ); - final Cursor unnestCursor; - - if (useDimensionCursor(capabilities)) { - unnestCursor = new UnnestDimensionCursor( - cursor, - cursor.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } else { - unnestCursor = new UnnestColumnValueSelectorCursor( - cursor, - cursor.getColumnSelectorFactory(), - unnestColumn, - outputColumnName - ); - } - return PostJoinCursor.wrap( - unnestCursor, - spec.getVirtualColumns(), - filterPair.rhs - ); - } + final CursorMaker maker = closer.register(baseAdapter.asCursorMaker(unnestBuildSpec)); + final Cursor cursor = maker.makeCursor(); + Objects.requireNonNull(cursor); + final ColumnCapabilities capabilities = unnestColumn.capabilities( + cursor.getColumnSelectorFactory(), + unnestColumn.getOutputName() + ); + final Cursor unnestCursor; + + if (useDimensionCursor(capabilities)) { + unnestCursor = new UnnestDimensionCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); + } else { + unnestCursor = new UnnestColumnValueSelectorCursor( + cursor, + cursor.getColumnSelectorFactory(), + unnestColumn, + outputColumnName + ); + } + return PostJoinCursor.wrap( + unnestCursor, + spec.getVirtualColumns(), + filterPair.rhs ); } - }; - } - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java index 22f58693b7c0..442666f084dd 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorMaker.java @@ -19,12 +19,8 @@ package org.apache.druid.segment.incremental; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -58,10 +54,10 @@ public IncrementalIndexCursorMaker( } @Override - public Sequence makeCursors() + public Cursor makeCursor() { if (index.isEmpty()) { - return Sequences.empty(); + return null; } if (builder.getQueryMetrics() != null) { @@ -74,26 +70,19 @@ public Sequence makeCursors() ); if (!builder.getInterval().overlaps(dataInterval)) { - return Sequences.empty(); + return null; } final Interval actualInterval = builder.getInterval().overlap(dataInterval); - Iterable intervals = builder.getGranularity().getIterable(actualInterval); - if (builder.isDescending()) { - intervals = Lists.reverse(ImmutableList.copyOf(intervals)); - } - return Sequences - .simple(intervals) - .map(i -> new IncrementalIndexCursor( - storageAdapter, - index, - builder.getVirtualColumns(), - builder.isDescending(), - builder.getFilter(), - i, - actualInterval, - builder.getGranularity() - )); + return new IncrementalIndexCursor( + storageAdapter, + index, + builder.getVirtualColumns(), + builder.isDescending(), + builder.getFilter(), + actualInterval, + builder.getGranularity() + ); } static class IncrementalIndexCursor implements Cursor @@ -115,7 +104,6 @@ static class IncrementalIndexCursor implements Cursor VirtualColumns virtualColumns, boolean descending, @Nullable Filter filter, - Interval interval, Interval actualInterval, Granularity gran ) @@ -131,14 +119,13 @@ static class IncrementalIndexCursor implements Cursor maxRowIndex = index.getLastRowIndex(); filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; - final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); cursorIterable = index.getFacts().timeRangeIterable( descending, - timeStart, - Math.min(actualInterval.getEndMillis(), gran.increment(interval.getStartMillis())) + actualInterval.getStartMillis(), + actualInterval.getEndMillis() ); emptyRange = !cursorIterable.iterator().hasNext(); - time = gran.toDateTime(interval.getStartMillis()); + time = gran.toDateTime(actualInterval.getStartMillis()); reset(); } @@ -149,12 +136,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return time; - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 5fb9adba1d57..b4a9270ee582 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -19,11 +19,6 @@ package org.apache.druid.segment.incremental; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; @@ -31,7 +26,6 @@ import org.apache.druid.segment.Metadata; import org.apache.druid.segment.NestedDataColumnIndexerV4; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; @@ -254,19 +248,6 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) return new IncrementalIndexCursorMaker(this, index, spec); } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - final Interval interval, - final VirtualColumns virtualColumns, - final Granularity gran, - final boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); - } - @Override public Metadata getMetadata() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index e6f67a1683e5..ee8a70aea84c 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -28,7 +28,6 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowIdSupplier; import org.apache.druid.segment.column.ColumnCapabilities; -import org.joda.time.DateTime; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -167,13 +166,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return joinColumnSelectorFactory; } - @Override - @Nonnull - public DateTime getTime() - { - return leftCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 1f5bec5cebce..547fb7be28f1 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -23,11 +23,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; @@ -44,10 +40,10 @@ import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey; import org.apache.druid.segment.join.filter.JoinFilterSplit; import org.apache.druid.segment.vector.VectorCursor; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.Arrays; import java.util.LinkedHashSet; @@ -240,11 +236,13 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) .build(); return baseAdapter.asCursorMaker(newSpec); } + return new CursorMaker() { + final Closer joinablesCloser = Closer.create(); @Override - public Sequence makeCursors() + public Cursor makeCursor() { // Filter pre-analysis key implied by the call to "makeCursors". We need to sanity-check that it matches // the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursors" call (it @@ -289,66 +287,43 @@ public Sequence makeCursors() ); cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns); - final Sequence baseCursorSequence = baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build()) - .makeCursors(); + final Cursor baseCursor = joinablesCloser.register(baseAdapter.asCursorMaker(cursorBuildSpecBuilder.build())) + .makeCursor(); - Closer joinablesCloser = Closer.create(); - return Sequences.map( - baseCursorSequence, - cursor -> { - assert cursor != null; - Cursor retVal = cursor; + assert baseCursor != null; + Cursor retVal = baseCursor; - for (JoinableClause clause : clauses) { - retVal = HashJoinEngine.makeJoinCursor(retVal, clause, spec.isDescending(), joinablesCloser); - } + for (JoinableClause clause : clauses) { + retVal = HashJoinEngine.makeJoinCursor(retVal, clause, spec.isDescending(), joinablesCloser); + } - return PostJoinCursor.wrap( - retVal, - VirtualColumns.fromIterable(preAnalysis.getPostJoinVirtualColumns()), - joinFilterSplit.getJoinTableFilter().orElse(null) - ); - } - ).withBaggage(joinablesCloser); + return PostJoinCursor.wrap( + retVal, + VirtualColumns.fromIterable(preAnalysis.getPostJoinVirtualColumns()), + joinFilterSplit.getJoinTableFilter().orElse(null) + ); } - }; - } - @Override - public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) - { - // HashJoinEngine isn't vectorized yet. - // However, we can still vectorize if there are no clauses, since that means all we need to do is apply - // a base filter. That's easy enough! - return clauses.isEmpty() && baseAdapter.canVectorize(baseFilterAnd(filter), virtualColumns, descending); - } + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(joinablesCloser); + } - @Nullable - @Override - public VectorCursor makeVectorCursor( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - boolean descending, - int vectorSize, - @Nullable QueryMetrics queryMetrics - ) - { - return delegateMakeVectorCursorToMaker(filter, interval, virtualColumns, descending, vectorSize, queryMetrics); - } + @Override + public boolean canVectorize() + { + return CursorMaker.super.canVectorize(); + } - @Override - public Sequence makeCursors( - @Nullable final Filter filter, - @Nonnull final Interval interval, - @Nonnull final VirtualColumns virtualColumns, - @Nonnull final Granularity gran, - final boolean descending, - @Nullable final QueryMetrics queryMetrics - ) - { - return delegateMakeCursorToMaker(filter, interval, virtualColumns, gran, descending, queryMetrics); + @Nullable + @Override + public VectorCursor makeVectorCursor() + { + return CursorMaker.super.makeVectorCursor(); + } + }; } /** diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 38413b57a4fc..81cd032bc3f9 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -26,7 +26,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -106,12 +105,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return columnSelectorFactory; } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Nullable public Filter getPostJoinFilter() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index ee8d70cab305..edc9bb056158 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -22,8 +22,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.cache.CacheKeyBuilder; @@ -32,6 +30,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; @@ -121,59 +120,50 @@ public BroadcastSegmentIndexedTable( ) .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - final Sequence sequence = Sequences.map( - cursors, - cursor -> { - if (cursor == null) { - return 0; - } - int rowNumber = 0; - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap - // indexes, but, an optimization for another day - final List selectors = keyColumnNames - .stream() - .map(columnName -> { - // multi-value dimensions are not currently supported - if (adapter.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) { - return NilColumnValueSelector.instance(); - } - return columnSelectorFactory.makeColumnValueSelector(columnName); - }) - .collect(Collectors.toList()); + int rowNumber = 0; + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - while (!cursor.isDone()) { - for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { - final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); - final int columnPosition = rowSignature.indexOf(keyColumnName); - final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); - keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); + // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap + // indexes, but, an optimization for another day + final List selectors = keyColumnNames + .stream() + .map(columnName -> { + // multi-value dimensions are not currently supported + if (adapter.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) { + return NilColumnValueSelector.instance(); } + return columnSelectorFactory.makeColumnValueSelector(columnName); + }) + .collect(Collectors.toList()); - if (rowNumber % 100_000 == 0) { - if (rowNumber == 0) { - LOG.debug("Indexed first row for table %s", theSegment.getId()); - } else { - LOG.debug("Indexed row %s for table %s", rowNumber, theSegment.getId()); - } - } - rowNumber++; - cursor.advance(); - } - return rowNumber; + while (!cursor.isDone()) { + for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { + final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); + final int columnPosition = rowSignature.indexOf(keyColumnName); + final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); + keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); } - ); - Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in); + if (rowNumber % 100_000 == 0) { + if (rowNumber == 0) { + LOG.debug("Indexed first row for table %s", theSegment.getId()); + } else { + LOG.debug("Indexed row %s for table %s", rowNumber, theSegment.getId()); + } + } + rowNumber++; + cursor.advance(); + } - this.keyColumnsIndexes = indexBuilders.stream() - .map(builder -> builder != null ? builder.build() : null) - .collect(Collectors.toList()); + this.keyColumnsIndexes = indexBuilders.stream() + .map(builder -> builder != null ? builder.build() : null) + .collect(Collectors.toList()); - LOG.info("Created BroadcastSegmentIndexedTable with %s rows.", totalRows); + LOG.info("Created BroadcastSegmentIndexedTable with %s rows.", rowNumber); + } } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index 7f63fbbe0f83..3eaf8a463f52 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -113,7 +113,7 @@ public FrameBasedIndexedTable( indexBuilders.add(m); } - final Sequence cursors = Sequences.concat( + final Sequence cursors = Sequences.simple( frameBasedInlineDataSource .getFrames() .stream() @@ -122,7 +122,7 @@ public FrameBasedIndexedTable( RowSignature rowSignature = frameSignaturePair.getRowSignature(); FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); - return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); + return frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursor(); }) .collect(Collectors.toList()) ); diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java index da9a7eeeb3c5..07e9714cb981 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java @@ -38,7 +38,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -56,7 +55,21 @@ public TestArrayStorageAdapter(QueryableIndex index) public CursorMaker asCursorMaker(CursorBuildSpec spec) { final CursorMaker delegate = super.asCursorMaker(spec); - return () -> delegate.makeCursors().map(DecoratedCursor::new); + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return new DecoratedCursor(delegate.makeCursor()); + } + + @Override + public void close() + { + delegate.close(); + } + }; } @@ -171,12 +184,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return cursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java index e84d46852605..582efbd9a27b 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; @@ -128,25 +129,32 @@ protected void verifySingleFrameReadableChannel( readableFrameChannel.close(); // build list of rows from written and read data to verify - List> writtenData = - adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())) - .toList(); - - FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( - readbackFrame, - FrameReader.create(adapter.getRowSignature()), - Intervals.ETERNITY - ); - List> readData = - adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())) - .toList(); - - Assert.assertEquals("Read rows count is different from written rows count", writtenData.size(), readData.size()); - Assert.assertEquals("Read data is different from written data", writtenData, readData); + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN);) { + List> writtenData = + FrameTestUtil.readRowsFromCursor( + maker.makeCursor(), + adapter.getRowSignature() + ).toList(); + + FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( + readbackFrame, + FrameReader.create(adapter.getRowSignature()), + Intervals.ETERNITY + ); + try (final CursorMaker frameMaker = frameStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + List> readData = FrameTestUtil.readRowsFromCursor( + frameMaker.makeCursor(), + frameStorageAdapter.getRowSignature() + ).toList(); + + Assert.assertEquals( + "Read rows count is different from written rows count", + writtenData.size(), + readData.size() + ); + Assert.assertEquals("Read data is different from written data", writtenData, readData); + } + } } @Test diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index 9b0d46f2fa80..3fa224a62b5f 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -365,7 +365,7 @@ public void tearDown() @Test public void test_makeCursors() { - assertCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec).makeCursors()); + assertCursorMatch(adapter -> adapter.asCursorMaker(buildSpec)); } @Test @@ -374,15 +374,17 @@ public void test_makeVectorCursor() assertVectorCursorsMatch(adapter -> adapter.asCursorMaker(buildSpec)); } - private void assertCursorsMatch(final Function> call) + private void assertCursorMatch(final Function call) { final RowSignature signature = frameAdapter.getRowSignature(); - final Sequence> queryableRows = - call.apply(queryableAdapter).flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, signature)); - final Sequence> frameRows = - call.apply(frameAdapter) - .flatMap(cursor -> FrameTestUtil.readRowsFromCursor(advanceAndReset(cursor), signature)); - FrameTestUtil.assertRowsEqual(queryableRows, frameRows); + try (final CursorMaker queryableMaker = call.apply(queryableAdapter); + final CursorMaker frameMaker = call.apply(frameAdapter)) { + final Sequence> queryableRows = + FrameTestUtil.readRowsFromCursor(queryableMaker.makeCursor(), signature); + final Sequence> frameRows = + FrameTestUtil.readRowsFromCursor(frameMaker.makeCursor(), signature); + FrameTestUtil.assertRowsEqual(queryableRows, frameRows); + } } private void assertVectorCursorsMatch(final Function call) @@ -397,8 +399,8 @@ private void assertVectorCursorsMatch(final Function frames() throw DruidException.defensive("Unrecognized frame type"); } - final Sequence cursors = FrameTestUtil.makeCursorsForAdapter(adapter, populateRowNumber); - - return cursors.flatMap( - cursor -> new BaseSequence<>( - new BaseSequence.IteratorMaker>() + final Pair cursorAndCloseable = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber); + final Cursor cursor = cursorAndCloseable.lhs; + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + return new Iterator() { @Override - public Iterator make() + public boolean hasNext() { - final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - return new Iterator() - { - @Override - public boolean hasNext() - { - return !cursor.isDone(); - } - - @Override - public Frame next() - { - if (cursor.isDone()) { - throw new NoSuchElementException(); - } - - try (final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { - while (!cursor.isDone()) { - if (!writer.addSelection()) { - if (writer.getNumRows() == 0) { - throw new FrameRowTooLargeException(allocator.capacity()); - } + return !cursor.isDone(); + } - return makeFrame(writer); - } + @Override + public Frame next() + { + if (cursor.isDone()) { + throw new NoSuchElementException(); + } + + try (final FrameWriter writer = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + while (!cursor.isDone()) { + if (!writer.addSelection()) { + if (writer.getNumRows() == 0) { + throw new FrameRowTooLargeException(allocator.capacity()); + } - cursor.advance(); + return makeFrame(writer); + } - if (writer.getNumRows() >= maxRowsPerFrame) { - return makeFrame(writer); - } - } + cursor.advance(); + if (writer.getNumRows() >= maxRowsPerFrame) { return makeFrame(writer); } } - private Frame makeFrame(final FrameWriter writer) - { - return Frame.wrap(writer.toByteArray()); - } - }; + return makeFrame(writer); + } } - @Override - public void cleanup(Iterator iterFromMake) + private Frame makeFrame(final FrameWriter writer) { - // Nothing to do. + return Frame.wrap(writer.toByteArray()); } - } - ) - ); + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // Nothing to do. + } + } + ).withBaggage(cursorAndCloseable.rhs); } } diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java index 7cd94b4f3972..1ae8561cd602 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java @@ -33,6 +33,7 @@ import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -42,6 +43,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -55,6 +57,7 @@ import org.junit.Assert; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -232,10 +235,14 @@ public static Sequence> readRowsFromFrameChannel( return new FrameChannelSequence(channel) .flatMap( frame -> - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .flatMap(cursor -> readRowsFromCursor(cursor, frameReader.signature())) + readRowsFromCursor( + // if FrameStorageAdapter.asCursorMaker ever needs closing.. this needs to change to add to a + // closer that is tied to baggage of this sequence... + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) + .asCursorMaker(CursorBuildSpec.FULL_SCAN) + .makeCursor(), + frameReader.signature() + ) ); } @@ -256,9 +263,8 @@ public static Sequence> readRowsFromAdapter( ) { final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature; - return makeCursorsForAdapter(adapter, populateRowNumber).flatMap( - cursor -> readRowsFromCursor(cursor, signatureToUse) - ); + final Pair cursorAndCloseable = makeCursorForAdapter(adapter, populateRowNumber); + return readRowsFromCursor(cursorAndCloseable.lhs, signatureToUse).withBaggage(cursorAndCloseable.rhs); } /** @@ -269,7 +275,7 @@ public static Sequence> readRowsFromAdapter( * @param adapter the adapter * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static Sequence makeCursorsForAdapter( + public static Pair makeCursorForAdapter( final StorageAdapter adapter, final boolean populateRowNumber ) @@ -289,15 +295,14 @@ public static Sequence makeCursorsForAdapter( .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - return adapter.asCursorMaker(buildSpec) - .makeCursors() - .map(cursor -> { - if (populateRowNumber) { - return new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn); - } else { - return cursor; - } - }); + + final CursorMaker maker = adapter.asCursorMaker(buildSpec); + final Cursor cursor = maker.makeCursor(); + if (populateRowNumber) { + return new Pair<>(new RowNumberUpdatingCursor(cursor, rowNumberVirtualColumn), maker); + } else { + return new Pair<>(cursor, maker); + } } public static Sequence> readRowsFromCursor(final Cursor cursor, final RowSignature signature) diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 3a47434891cc..34cdab25a48b 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -22,7 +22,6 @@ import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.joda.time.DateTime; /** * Used by {@link FrameTestUtil#readRowsFromAdapter} and {@link FrameTestUtil#readRowsFromCursor}. @@ -44,12 +43,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return baseCursor.getColumnSelectorFactory(); } - @Override - public DateTime getTime() - { - return baseCursor.getTime(); - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index a9424a0f5272..9ea17a258027 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -48,7 +48,9 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.RowIdSupplier; @@ -575,49 +577,44 @@ private static Pair writeFrame( inputSegment = new FrameSegment(inputFrame, FrameReader.create(signature), SegmentId.dummy("xxx")); } - return inputSegment.asStorageAdapter() - .asCursorMaker(CursorBuildSpec.FULL_SCAN) - .makeCursors() - .accumulate( - null, - (retVal, cursor) -> { - int numRows = 0; - final FrameWriterFactory frameWriterFactory; - if (FrameType.ROW_BASED.equals(outputFrameType)) { - frameWriterFactory = FrameWriters.makeRowBasedFrameWriterFactory( - new SingleMemoryAllocatorFactory(allocator), - signature, - keyColumns, - false - ); - } else { - frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( - new SingleMemoryAllocatorFactory(allocator), - signature, - keyColumns - ); - } - - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - - if (capabilitiesAdjustFn != null) { - columnSelectorFactory = new OverrideCapabilitiesColumnSelectorFactory( - columnSelectorFactory, - capabilitiesAdjustFn - ); - } - - try (final FrameWriter frameWriter = - frameWriterFactory.newFrameWriter(columnSelectorFactory)) { - while (!cursor.isDone() && frameWriter.addSelection()) { - numRows++; - cursor.advance(); - } - - return Pair.of(Frame.wrap(frameWriter.toByteArray()), numRows); - } - } - ); + try (final CursorMaker maker = inputSegment.asStorageAdapter().asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + + int numRows = 0; + final FrameWriterFactory frameWriterFactory; + if (FrameType.ROW_BASED.equals(outputFrameType)) { + frameWriterFactory = FrameWriters.makeRowBasedFrameWriterFactory( + new SingleMemoryAllocatorFactory(allocator), + signature, + keyColumns, + false + ); + } else { + frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( + new SingleMemoryAllocatorFactory(allocator), + signature, + keyColumns + ); + } + + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + if (capabilitiesAdjustFn != null) { + columnSelectorFactory = new OverrideCapabilitiesColumnSelectorFactory( + columnSelectorFactory, + capabilitiesAdjustFn + ); + } + + try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + while (!cursor.isDone() && frameWriter.addSelection()) { + numRows++; + cursor.advance(); + } + + return Pair.of(Frame.wrap(frameWriter.toByteArray()), numRows); + } + } } /** diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java new file mode 100644 index 000000000000..f708714cbcef --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.ListBasedInputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class CursorGranularizerTest extends InitializedNullHandlingTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private StorageAdapter adapter; + + @Before + public void setup() throws IOException + { + final RowSignature signature = RowSignature.builder().add("x", ColumnType.STRING).build(); + final List dims = ImmutableList.of("x"); + final IncrementalIndexSchema schema = + IncrementalIndexSchema.builder() + .withDimensionsSpec(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .withQueryGranularity(Granularities.MINUTE) + .build(); + IndexBuilder bob = + IndexBuilder.create() + .schema(schema) + .rows( + ImmutableList.of( + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:00Z"), + dims, + ImmutableList.of("a") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:01Z"), + dims, + ImmutableList.of("b") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:02Z"), + dims, + ImmutableList.of("c") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T00:03Z"), + dims, + ImmutableList.of("d") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T01:00Z"), + dims, + ImmutableList.of("e") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T01:01Z"), + dims, + ImmutableList.of("f") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T03:04Z"), + dims, + ImmutableList.of("g") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T03:05Z"), + dims, + ImmutableList.of("h") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T03:15Z"), + dims, + ImmutableList.of("i") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T05:03Z"), + dims, + ImmutableList.of("j") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T06:00Z"), + dims, + ImmutableList.of("k") + ), + new ListBasedInputRow( + signature, + DateTimes.of("2024-01-01T09:01Z"), + dims, + ImmutableList.of("l") + ) + ) + ) + .tmpDir(temporaryFolder.newFolder()); + + adapter = new QueryableIndexStorageAdapter(bob.buildMMappedIndex()); + } + + @Test + public void testGranularizeFullScan() + { + try (CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + Granularities.HOUR, + adapter.getInterval(), + false + ); + + final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); + final ColumnValueSelector xSelector = selectorFactory.makeColumnValueSelector("x"); + final Sequence> theSequence = + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + List bucket = new ArrayList<>(); + if (!granularizer.advanceToBucket(bucketInterval)) { + return bucket; + } + while (!cursor.isDone()) { + bucket.add((String) xSelector.getObject()); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } + } + return bucket; + }); + + List> granularized = theSequence.toList(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("a", "b", "c", "d"), + ImmutableList.of("e", "f"), + ImmutableList.of(), + ImmutableList.of("g", "h", "i"), + ImmutableList.of(), + ImmutableList.of("j"), + ImmutableList.of("k"), + ImmutableList.of(), + ImmutableList.of(), + ImmutableList.of("l") + ), + granularized + ); + } + } + + @Test + public void testGranularizeFullScanDescending() + { + final CursorBuildSpec descending = CursorBuildSpec.builder() + .setGranularity(Granularities.ALL) + .isDescending(true) + .build(); + try (CursorMaker maker = adapter.asCursorMaker(descending)) { + final Cursor cursor = maker.makeCursor(); + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + Granularities.HOUR, + adapter.getInterval(), + true + ); + + final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); + final ColumnValueSelector xSelector = selectorFactory.makeColumnValueSelector("x"); + final Sequence> theSequence = + Sequences.simple(granularizer.getBucketIterable()) + .map(bucketInterval -> { + List bucket = new ArrayList<>(); + if (!granularizer.advanceToBucket(bucketInterval)) { + return bucket; + } + while (!cursor.isDone()) { + bucket.add((String) xSelector.getObject()); + if (!granularizer.advanceCursorWithinBucket()) { + break; + } + } + return bucket; + }); + + List> granularized = theSequence.toList(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("l"), + ImmutableList.of(), + ImmutableList.of(), + ImmutableList.of("k"), + ImmutableList.of("j"), + ImmutableList.of(), + ImmutableList.of("i", "h", "g"), + ImmutableList.of(), + ImmutableList.of("f", "e"), + ImmutableList.of("d", "c", "b", "a") + ), + granularized + ); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index ea4af2ffeced..79a0ae80875b 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -662,7 +662,7 @@ private void runResults( final CursorMaker maker = s.asStorageAdapter() .asCursorMaker(spec); final boolean canVectorize = maker.canVectorize(); - maker.cleanup(); + maker.close(); return canVectorize; }); diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java index aa369dc63777..486ef7e20293 100644 --- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java @@ -25,11 +25,11 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.RowBasedStorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -192,37 +192,31 @@ public void test_asStorageAdapter_getDimensionCardinalityV() @Test public void test_asStorageAdapter_makeCursors() { - final Sequence cursors = LOOKUP_SEGMENT.asStorageAdapter().asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970/PT1H")) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final List> kvs = new ArrayList<>(); - - cursors.accumulate( - null, - (ignored, cursor) -> { - final ColumnValueSelector keySelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("k"); - final ColumnValueSelector valueSelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); - - while (!cursor.isDone()) { - kvs.add(Pair.of(String.valueOf(keySelector.getObject()), String.valueOf(valueSelector.getObject()))); - cursor.advanceUninterruptibly(); - } - - return null; - } - ); - - Assert.assertEquals( - ImmutableList.of( - Pair.of("a", "b"), - Pair.of("x", "y") - ), - kvs - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970/PT1H")) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = LOOKUP_SEGMENT.asStorageAdapter().asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + final List> kvs = new ArrayList<>(); + + final ColumnValueSelector keySelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("k"); + final ColumnValueSelector valueSelector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); + + while (!cursor.isDone()) { + kvs.add(Pair.of(String.valueOf(keySelector.getObject()), String.valueOf(valueSelector.getObject()))); + cursor.advanceUninterruptibly(); + } + + Assert.assertEquals( + ImmutableList.of( + Pair.of("a", "b"), + Pair.of("x", "y") + ), + kvs + ); + } } @Test diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java index 367c29653473..b9fc63db8808 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.groupby.ResultRow; @@ -38,6 +37,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.TypeStrategy; @@ -256,22 +256,16 @@ private void validateDecorated( if (interval != null) { builder.setInterval(interval); } - final Sequence cursors = seggy - .asStorageAdapter() - .asCursorMaker(builder.build()) - .makeCursors(); - - vals = cursors.accumulate( - new ArrayList<>(), - (accumulated, in) -> { - final ColumnValueSelector idSupplier = in.getColumnSelectorFactory().makeColumnValueSelector("arrayIndex"); - while (!in.isDone()) { - accumulated.add(originalVals[(int) idSupplier.getLong()]); - in.advance(); - } - return accumulated; - } - ); + try (final CursorMaker maker = seggy.asStorageAdapter().asCursorMaker(builder.build())) { + final Cursor cursor = maker.makeCursor(); + + vals = new ArrayList<>(); + final ColumnValueSelector idSupplier = cursor.getColumnSelectorFactory().makeColumnValueSelector("arrayIndex"); + while (!cursor.isDone()) { + vals.add(originalVals[(int) idSupplier.getLong()]); + cursor.advance(); + } + } } if (ordering != null) { diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index b1b16103a77b..fbac92a4adac 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -669,7 +669,7 @@ private void runResults( final CursorMaker maker = s.asStorageAdapter() .asCursorMaker(spec); final boolean canVectorize = maker.canVectorize(); - maker.cleanup(); + maker.close(); return canVectorize; }); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index f68104ff4815..16a7b7066af0 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -24,23 +24,20 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; import org.apache.druid.query.filter.DruidPredicateFactory; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; @@ -321,16 +318,17 @@ public Metadata getMetadata() } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return null; + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return null; + } + }; } }; diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index ff0f4e22133b..2f5af7240e86 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; @@ -45,7 +44,6 @@ import org.junit.Test; import javax.annotation.Nonnull; -import java.util.List; import java.util.Map; public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest @@ -150,51 +148,44 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("b", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("b", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("c", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("c", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(STRING_COL).toColumnType()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("b", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("b", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("c", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("c", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(STRING_COL).toColumnType()); + } } @Test @@ -210,81 +201,74 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1L, valueSelector.getObject()); - Assert.assertEquals(1L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertEquals(2L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3L, valueSelector.getObject()); - Assert.assertEquals(3L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1L, valueSelector.getObject()); + Assert.assertEquals(1L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); - } - + Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertEquals(2L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3L, valueSelector.getObject()); + Assert.assertEquals(3L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } + + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } + Assert.assertEquals(ColumnType.LONG, storageAdapter.getColumnCapabilities(LONG_COL).toColumnType()); } - Assert.assertEquals(ColumnType.LONG, storageAdapter.getColumnCapabilities(LONG_COL).toColumnType()); } @Test @@ -300,80 +284,74 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1.1, valueSelector.getObject()); - Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1.1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2.2, valueSelector.getObject()); - Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2.2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1.1, valueSelector.getObject()); + Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); - } + Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1.1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + + cursor.advance(); + Assert.assertEquals(2.2, valueSelector.getObject()); + Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2.2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } + Assert.assertEquals(ColumnType.DOUBLE, storageAdapter.getColumnCapabilities(DOUBLE_COL).toColumnType()); } - Assert.assertEquals(ColumnType.DOUBLE, storageAdapter.getColumnCapabilities(DOUBLE_COL).toColumnType()); } @Test @@ -389,50 +367,39 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec( + STRING_ARRAY_COL, + STRING_ARRAY_COL, + ColumnType.STRING + ); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(ColumnType.STRING_ARRAY, storageAdapter.getColumnCapabilities(STRING_ARRAY_COL).toColumnType()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals( + ColumnType.STRING_ARRAY, + storageAdapter.getColumnCapabilities(STRING_ARRAY_COL).toColumnType() + ); + } } @Test @@ -448,43 +415,35 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(VARIANT_COL).toColumnType()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + DimensionSelector dimensionSelector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(VARIANT_COL).toColumnType()); + } } @Test @@ -500,50 +459,31 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(ColumnType.NESTED_DATA, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(ColumnType.NESTED_DATA, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + } } @Test @@ -578,42 +518,35 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("2", valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + DimensionSelector dimensionSelector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("2", valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + } } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index fc1eba743d4f..2ed9e8431501 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -26,7 +26,6 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.List; @@ -189,12 +188,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - throw new UnsupportedOperationException(); - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index bedf83c20dad..ce18715d1c2c 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; @@ -44,7 +43,6 @@ import org.junit.Test; import javax.annotation.Nonnull; -import java.util.List; import java.util.Map; public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest @@ -150,49 +148,42 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("b", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("b", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals("c", valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("c", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("a", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("b", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("b", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("b", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals("c", valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals("c", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("c", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } } @Test @@ -208,78 +199,71 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1L, valueSelector.getObject()); - Assert.assertEquals(1L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertEquals(2L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3L, valueSelector.getObject()); - Assert.assertEquals(3L, valueSelector.getLong()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1L, valueSelector.getObject()); + Assert.assertEquals(1L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); - } + Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertEquals(2L, valueSelector.getLong()); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertEquals("2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3L, valueSelector.getObject()); + Assert.assertEquals(3L, valueSelector.getLong()); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultLongValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + Assert.assertEquals("3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultLongValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); + } } } @@ -296,78 +280,71 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(1.1, valueSelector.getObject()); - Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("1.1", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2.2, valueSelector.getObject()); - Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("2.2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); - Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); + DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); + Assert.assertEquals(1.1, valueSelector.getObject()); + Assert.assertEquals(1.1, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); - } + Assert.assertEquals("1.1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("1.1", dimensionSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL); - dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec); - if (NullHandling.sqlCompatible()) { - Assert.assertNull(valueSelector.getObject()); - Assert.assertTrue(valueSelector.isNull()); + cursor.advance(); + Assert.assertEquals(2.2, valueSelector.getObject()); + Assert.assertEquals(2.2, valueSelector.getDouble(), 0.0); + Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); - Assert.assertNull(dimensionSelector.getObject()); - } else { - Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertEquals("2.2", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("2.2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertEquals(3.3, valueSelector.getDouble(), 0.0); Assert.assertFalse(valueSelector.isNull()); Assert.assertEquals(1, dimensionSelector.getRow().size()); - Assert.assertEquals( - String.valueOf(NullHandling.defaultDoubleValue()), - dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) - ); - Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + Assert.assertEquals("3.3", dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } + + cursor.advance(); + if (NullHandling.sqlCompatible()) { + Assert.assertNull(valueSelector.getObject()); + Assert.assertTrue(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); + Assert.assertNull(dimensionSelector.getObject()); + } else { + Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals(1, dimensionSelector.getRow().size()); + Assert.assertEquals( + String.valueOf(NullHandling.defaultDoubleValue()), + dimensionSelector.lookupName(dimensionSelector.getRow().get(0)) + ); + Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); + } } } @@ -384,49 +361,34 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_ARRAY_COL, STRING_ARRAY_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec( + STRING_ARRAY_COL, + STRING_ARRAY_COL, + ColumnType.STRING + ); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + } } @Test @@ -442,41 +404,34 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals("a", valueSelector.getObject()); - Assert.assertEquals("a", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(2L, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("2", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertEquals(3.3, valueSelector.getObject()); - Assert.assertFalse(valueSelector.isNull()); - Assert.assertEquals("3.3", dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); - - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); - Assert.assertNull(valueSelector.getObject()); - Assert.assertNull(dimensionSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); + DimensionSelector dimensionSelector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(2L, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertEquals(3.3, valueSelector.getObject()); + Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); + } } @Test @@ -492,49 +447,30 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - Sequence cursorSequence = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build()).makeCursors(); - final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); - List cursorList = cursorSequence.toList(); - ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); - - ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + try (final CursorMaker maker = storageAdapter.asCursorMaker(CursorBuildSpec.builder().build())) { + Cursor cursor = maker.makeCursor(); + final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> cursor.getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) + ); + Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); + cursor.advance(); + Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1.1, "y", 2L)), valueSelector.getObject()); - columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); - columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); - valueSelector = columnSelectorFactory.makeColumnValueSelector(NESTED_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertNull(valueSelector.getObject()); + cursor.advance(); + Assert.assertNull(valueSelector.getObject()); + } } @Nonnull diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java index 1f5c18fdd056..f4b4d41145e7 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java @@ -21,9 +21,6 @@ import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -86,9 +83,8 @@ public void setUp() partialNullSelector = columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column")); } else { - final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - final Yielder yielder = closer.register(Yielders.each(cursors)); - final Cursor cursor = yielder.get(); + final CursorMaker maker = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)); + final Cursor cursor = maker.makeCursor(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); qualitySelector = @@ -232,11 +228,9 @@ public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - final Yielder cursorYielder = Yielders.each(cursors); - cursor = cursorYielder.get(); + final CursorMaker maker = closer.register(adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)); + cursor = maker.makeCursor(); columnSelectorFactory = cursor.getColumnSelectorFactory(); - closer.register(cursorYielder); } @After diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 2382b6d0e071..d12ca72d638f 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; @@ -31,6 +32,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnCapabilities; @@ -49,6 +51,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import java.util.function.ToLongFunction; @@ -79,10 +82,10 @@ public class RowBasedStorageAdapterTest } ); - private static final List>> READ_TIME_AND_STRING = + private static final List>> READ_TIME_AND_STRING_GRAN = ImmutableList.of( - cursor -> cursor::getTime, - cursor -> { + (cursor, granularizer) -> granularizer::getBucketStart, + (cursor, granularizer) -> { final BaseObjectColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector(ValueType.STRING.name()); return selector::getObject; @@ -99,11 +102,6 @@ public static void setUpClass() PROCESSORS.clear(); - PROCESSORS.put( - "cursor-time", - cursor -> cursor::getTime - ); - // Read all the types as all the other types. for (final String valueTypeName : ROW_SIGNATURE.getColumnNames()) { @@ -456,19 +454,21 @@ public void test_makeCursors_filterOnLong() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); Assert.assertEquals(1, numCloses.get()); } @@ -478,20 +478,21 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("0"), - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("0"), + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -501,31 +502,32 @@ public void test_makeCursors_filterOnVirtualColumn() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) - .setVirtualColumns( - VirtualColumns.create( - ImmutableList.of( - new ExpressionVirtualColumn( - "vc", - "\"LONG\" + 1", - ColumnType.LONG, - ExprMacroTable.nil() - ) - ) - ) - ) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) + .setVirtualColumns( + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "vc", + "\"LONG\" + 1", + ColumnType.LONG, + ExprMacroTable.nil() + ) + ) + ) + ) + .setGranularity(Granularities.ALL) + .build(); + + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -535,18 +537,18 @@ public void test_makeCursors_descending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("2"), - ImmutableList.of("1"), - ImmutableList.of("0") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setGranularity(Granularities.ALL).isDescending(true).build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("2"), + ImmutableList.of("1"), + ImmutableList.of("0") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -556,17 +558,17 @@ public void test_makeCursors_intervalDoesNotMatch() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("2000/P1D")) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of(), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("2000/P1D")) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of(), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -576,19 +578,19 @@ public void test_makeCursors_intervalPartiallyMatches() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970-01-01T01/PT1H")) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of("1") - ), - walkCursors(cursors, READ_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT1H")) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("1") + ), + walkCursor(cursor, READ_STRING) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -598,24 +600,23 @@ public void test_makeCursors_hourGranularity() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970/1971")) - .setGranularity(Granularities.HOUR) - .build() - ).makeCursors(); - - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(DateTimes.of("1970-01-01T00"), "0"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), - ImmutableList.of(DateTimes.of("1970-01-01T03"), "3") - ), - walkCursors(cursors, READ_TIME_AND_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970/1971")) + .setGranularity(Granularities.HOUR) + .build(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(DateTimes.of("1970-01-01T00"), "0"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), + ImmutableList.of(DateTimes.of("1970-01-01T03"), "3") + ), + walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -625,21 +626,23 @@ public void test_makeCursors_hourGranularityWithInterval() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970-01-01T01/PT2H")) - .setGranularity(Granularities.HOUR) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T02"), "2") - ), - walkCursors(cursors, READ_TIME_AND_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setGranularity(Granularities.HOUR) + .build(); + + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T02"), "2") + ), + walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -649,22 +652,23 @@ public void test_makeCursors_hourGranularityWithIntervalDescending() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(Intervals.of("1970-01-01T01/PT2H")) - .setGranularity(Granularities.HOUR) - .isDescending(true) - .build() - ).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), - ImmutableList.of(DateTimes.of("1970-01-01T01"), "1") - ), - walkCursors(cursors, READ_TIME_AND_STRING) - ); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(Intervals.of("1970-01-01T01/PT2H")) + .setGranularity(Granularities.HOUR) + .isDescending(true) + .build(); + + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), + ImmutableList.of(DateTimes.of("1970-01-01T01"), "1") + ), + walkCursorGranularized(adapter, cursor, buildSpec, READ_TIME_AND_STRING_GRAN) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -674,103 +678,102 @@ public void test_makeCursors_allProcessors() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - - Assert.assertEquals( - ImmutableList.of( - Lists.newArrayList( - Intervals.ETERNITY.getStart(), - - // FLOAT - 0f, - 0d, - 0L, - "0.0", - 0f, - - // DOUBLE - 0f, - 0d, - 0L, - "0.0", - 0d, - - // LONG - 0f, - 0d, - 0L, - "0", - 0L, - - // STRING - 0f, - 0d, - 0L, - "0", - "0", - - // COMPLEX - NullHandling.defaultFloatValue(), - NullHandling.defaultDoubleValue(), - NullHandling.defaultLongValue(), - null, - null, - - // unknownType - 0f, - 0d, - 0L, - "0", - 0 - ), - Lists.newArrayList( - Intervals.ETERNITY.getStart(), - - // FLOAT - 1f, - 1d, - 1L, - "1.0", - 1f, - - // DOUBLE - 1f, - 1d, - 1L, - "1.0", - 1d, - - // LONG - 1f, - 1d, - 1L, - "1", - 1L, - - // STRING - 1f, - 1d, - 1L, - "1", - "1", - - // COMPLEX - NullHandling.defaultFloatValue(), - NullHandling.defaultDoubleValue(), - NullHandling.defaultLongValue(), - null, - null, - - // unknownType - 1f, - 1d, - 1L, - "1", - 1 - ) - ), - walkCursors(cursors, new ArrayList<>(PROCESSORS.values())) - ); + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of( + Lists.newArrayList( + + // FLOAT + 0f, + 0d, + 0L, + "0.0", + 0f, + + // DOUBLE + 0f, + 0d, + 0L, + "0.0", + 0d, + + // LONG + 0f, + 0d, + 0L, + "0", + 0L, + + // STRING + 0f, + 0d, + 0L, + "0", + "0", + + // COMPLEX + NullHandling.defaultFloatValue(), + NullHandling.defaultDoubleValue(), + NullHandling.defaultLongValue(), + null, + null, + + // unknownType + 0f, + 0d, + 0L, + "0", + 0 + ), + Lists.newArrayList( + + // FLOAT + 1f, + 1d, + 1L, + "1.0", + 1f, + + // DOUBLE + 1f, + 1d, + 1L, + "1.0", + 1d, + + // LONG + 1f, + 1d, + 1L, + "1", + 1L, + + // STRING + 1f, + 1d, + 1L, + "1", + "1", + + // COMPLEX + NullHandling.defaultFloatValue(), + NullHandling.defaultDoubleValue(), + NullHandling.defaultLongValue(), + null, + null, + + // unknownType + 1f, + 1d, + 1L, + "1", + 1 + ) + ), + walkCursor(cursor, new ArrayList<>(PROCESSORS.values())) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -780,17 +783,18 @@ public void test_makeCursors_filterOnNonexistentColumnEqualsNonnull() { final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); - final Sequence cursors = adapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) + .setGranularity(Granularities.ALL) + .build(); - Assert.assertEquals( - ImmutableList.of(), - walkCursors(cursors, new ArrayList<>(PROCESSORS.values())) - ); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); + Assert.assertEquals( + ImmutableList.of(), + walkCursor(cursor, new ArrayList<>(PROCESSORS.values())) + ); + } Assert.assertEquals(1, numCloses.get()); } @@ -804,38 +808,78 @@ public void test_makeCursors_eternityIntervalWithMonthGranularity() CursorBuildSpec.builder() .setGranularity(Granularities.MONTH) .build() - ).makeCursors(); + ).makeCursor(); }); } - private static List> walkCursors( - final Sequence cursors, + private static List> walkCursor( + final Cursor cursor, final List>> processors ) { - return cursors.flatMap( - cursor -> { - // Gather test-value suppliers together. - final List> suppliers = new ArrayList<>(); - for (Function> processor : processors) { - suppliers.add(processor.apply(cursor)); - } + final List> suppliers = new ArrayList<>(); + for (Function> processor : processors) { + suppliers.add(processor.apply(cursor)); + } - final List> retVal = new ArrayList<>(); + final List> retVal = new ArrayList<>(); - while (!cursor.isDone()) { - final List row = new ArrayList<>(); + while (!cursor.isDone()) { + final List row = new ArrayList<>(); - for (Supplier supplier : suppliers) { - row.add(supplier.get()); - } + for (Supplier supplier : suppliers) { + row.add(supplier.get()); + } - retVal.add(row); - cursor.advanceUninterruptibly(); - } + retVal.add(row); + cursor.advanceUninterruptibly(); + } - return Sequences.simple(retVal); - } - ).toList(); + return retVal; + } + + private static List> walkCursorGranularized( + final StorageAdapter adapter, + final Cursor cursor, + final CursorBuildSpec buildSpec, + final List>> processors + ) + { + CursorGranularizer granularizer = CursorGranularizer.create( + adapter, + cursor, + buildSpec.getGranularity(), + buildSpec.getInterval(), + buildSpec.isDescending() + ); + + final List> suppliers = new ArrayList<>(); + for (BiFunction> processor : processors) { + suppliers.add(processor.apply(cursor, granularizer)); + } + + final Sequence> theSequence = + Sequences.simple(granularizer.getBucketIterable()) + .flatMap(bucketInterval -> { + if (!granularizer.advanceToBucket(bucketInterval)) { + return Sequences.empty(); + } + final List> retVal = new ArrayList<>(); + while (!cursor.isDone()) { + final List row = new ArrayList<>(); + + for (Supplier supplier : suppliers) { + row.add(supplier.get()); + } + + retVal.add(row); + if (!granularizer.advanceCursorWithinBucketUninterruptedly()) { + break; + } + } + return Sequences.simple(retVal); + }) + .filter(Predicates.notNull()); + return theSequence.toList(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java index c93abe864f16..7d51b9e46e69 100644 --- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java @@ -19,10 +19,6 @@ package org.apache.druid.segment; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; import org.joda.time.DateTime; @@ -41,15 +37,17 @@ public void testTombstoneDefaultInterface() StorageAdapter sa = new StorageAdapter() { @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return null; + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return null; + } + }; } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index d171ea21b2d4..4667ffe0a36e 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.NestedDataTestUtils; @@ -229,14 +228,13 @@ public void test_unnest_adapter_column_capabilities() @Test public void test_unnest_adapters_basic() { - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(UNNEST_STORAGE_ADAPTER.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(UNNEST_STORAGE_ADAPTER.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = UNNEST_STORAGE_ADAPTER.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); @@ -254,22 +252,18 @@ public void test_unnest_adapters_basic() unnest 2 rows -> 16 rows after unnest */ Assert.assertEquals(count, 16); - return null; - }); - + } } @Test public void test_two_levels_of_unnest_adapters() { - Sequence cursorSequence = UNNEST_STORAGE_ADAPTER1.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(UNNEST_STORAGE_ADAPTER1.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = UNNEST_STORAGE_ADAPTER1.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME1)); @@ -295,8 +289,7 @@ public void test_two_levels_of_unnest_adapters() */ Assert.assertEquals(count, 128); Assert.assertEquals(dimSelector.getValueCardinality(), 17); - return null; - }); + } } @Test @@ -322,25 +315,22 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest selector(inputColumn, "2") )); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(baseFilter) - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); - - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(baseFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); + + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); // OR-case so base filter should match the postJoinFilter Assert.assertEquals(baseFilter, postFilter); - return null; - }); + } } @Test @@ -372,25 +362,22 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap )) )); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(baseFilter) - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); - - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(baseFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); + + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); // OR-case so base filter should match the postJoinFilter Assert.assertEquals(baseFilter, postFilter); - return null; - }); + } } @Test @@ -655,19 +642,17 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() final Filter expectedPushDownFilter = selector(inputColumn, "1"); + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); Assert.assertEquals(unnestStorageAdapter.getUnnestFilter(), postFilter); @@ -678,8 +663,7 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() count++; } Assert.assertEquals(1, count); - return null; - }); + } } @@ -700,19 +684,18 @@ public void test_pushdown_filters_unnested_dimension_outside() selector(inputColumn, "1"); final Filter queryFilter = new SelectorFilter(OUTPUT_COLUMN_NAME, "1", null); - final Sequence cursorSequence = unnestStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setFilter(queryFilter) - .setInterval(unnestStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); - - Assert.assertEquals(expectedPushDownFilter, pushDownFilter); - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setFilter(queryFilter) + .setInterval(unnestStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + + try (final CursorMaker maker = unnestStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); + final Filter pushDownFilter = base.getPushDownFilter(); + + Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); Assert.assertEquals(queryFilter, postFilter); @@ -723,8 +706,7 @@ public void test_pushdown_filters_unnested_dimension_outside() count++; } Assert.assertEquals(1, count); - return null; - }); + } } @Test @@ -751,14 +733,12 @@ public void testUnnestValueMatcherValueDoesntExist() new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), null ); - Sequence cursorSequence = withNullsStorageAdapter.asCursorMaker( - CursorBuildSpec.builder() - .setInterval(withNullsStorageAdapter.getInterval()) - .setGranularity(Granularities.ALL) - .build() - ).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + final CursorBuildSpec buildSpec = CursorBuildSpec.builder() + .setInterval(withNullsStorageAdapter.getInterval()) + .setGranularity(Granularities.ALL) + .build(); + try (final CursorMaker maker = withNullsStorageAdapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); DimensionSelector dimSelector = factory.makeDimensionSelector(DefaultDimensionSpec.of(OUTPUT_COLUMN_NAME)); @@ -776,9 +756,7 @@ public void testUnnestValueMatcherValueDoesntExist() count++; } Assert.assertEquals(count, 618); - return null; - }); - + } } public void testComputeBaseAndPostUnnestFilters( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 082c6cfcf451..f232b70b82ba 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -47,8 +47,6 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprType; import org.apache.druid.math.expr.ExpressionType; @@ -72,6 +70,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; @@ -792,29 +791,33 @@ private DimFilter maybeOptimize(final DimFilter dimFilter) return optimize ? dimFilter.optimize(false) : dimFilter; } + private CursorBuildSpec makeCursorBuildSpec(@Nullable Filter filter) + { + return CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(VIRTUAL_COLUMNS) + .setGranularity(Granularities.ALL) + .build(); + + } - private Sequence makeCursorSequence(final Filter filter) + private CursorBuildSpec makeVectorCursorBuildSpec(@Nullable Filter filter) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setVirtualColumns(VIRTUAL_COLUMNS) - .setGranularity(Granularities.ALL) - .build(); - return adapter.asCursorMaker(buildSpec).makeCursors(); + return CursorBuildSpec.builder() + .setFilter(filter) + .setVirtualColumns(virtualColumns) + .setGranularity(Granularities.ALL) + .setQueryContext( + QueryContext.of( + ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) + ) + ) + .build(); } private VectorCursor makeVectorCursor(final Filter filter) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(filter) - .setVirtualColumns(virtualColumns) - .setGranularity(Granularities.ALL) - .setQueryContext( - QueryContext.of( - ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, 3) - ) - ) - .build(); + final CursorBuildSpec buildSpec = makeVectorCursorBuildSpec(filter); return adapter.asCursorMaker(buildSpec).makeVectorCursor(); } @@ -823,48 +826,39 @@ private VectorCursor makeVectorCursor(final Filter filter) */ private List selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn) { - final Sequence cursors = makeCursorSequence(makeFilter(filter)); - Sequence> seq = Sequences.map( - cursors, - cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - - final List values = new ArrayList<>(); - - while (!cursor.isDone()) { - IndexedInts row = selector.getRow(); - Preconditions.checkState(row.size() == 1); - values.add(selector.lookupName(row.get(0))); - cursor.advance(); - } + try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(makeFilter(filter)))) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - return values; - } - ); - return seq.toList().get(0); + final List values = new ArrayList<>(); + + while (!cursor.isDone()) { + IndexedInts row = selector.getRow(); + Preconditions.checkState(row.size() == 1); + values.add(selector.lookupName(row.get(0))); + cursor.advance(); + } + return values; + } } private long selectCountUsingFilteredAggregator(final DimFilter filter) { - final Sequence cursors = makeCursorSequence(null); - Sequence aggSeq = Sequences.map( - cursors, - cursor -> { - Aggregator agg = new FilteredAggregatorFactory( - new CountAggregatorFactory("count"), - maybeOptimize(filter) - ).factorize(cursor.getColumnSelectorFactory()); - - for (; !cursor.isDone(); cursor.advance()) { - agg.aggregate(); - } + try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(null))) { + final Cursor cursor = maker.makeCursor(); + Aggregator agg = new FilteredAggregatorFactory( + new CountAggregatorFactory("count"), + maybeOptimize(filter) + ).factorize(cursor.getColumnSelectorFactory()); - return agg; - } - ); - return aggSeq.toList().get(0).getLong(); + for (; !cursor.isDone(); cursor.advance()) { + agg.aggregate(); + } + + return agg.getLong(); + } } private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFilter) @@ -875,7 +869,9 @@ private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFil dimFilter ); - try (final VectorCursor cursor = makeVectorCursor(null)) { + + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(null)); + final VectorCursor cursor = maker.makeVectorCursor()) { final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), maybeOptimize(dimFilter) @@ -941,27 +937,23 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - final Sequence cursors = makeCursorSequence(postFilteringFilter); - Sequence> seq = Sequences.map( - cursors, - cursor -> { - final DimensionSelector selector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - - final List values = new ArrayList<>(); - - while (!cursor.isDone()) { - IndexedInts row = selector.getRow(); - Preconditions.checkState(row.size() == 1); - values.add(selector.lookupName(row.get(0))); - cursor.advance(); - } + try (final CursorMaker maker = adapter.asCursorMaker(makeCursorBuildSpec(postFilteringFilter))) { + final Cursor cursor = maker.makeCursor(); + final DimensionSelector selector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); - return values; - } - ); - return seq.toList().get(0); + final List values = new ArrayList<>(); + + while (!cursor.isDone()) { + IndexedInts row = selector.getRow(); + Preconditions.checkState(row.size() == 1); + values.add(selector.lookupName(row.get(0))); + cursor.advance(); + } + + return values; + } } private List selectColumnValuesMatchingFilterUsingVectorizedPostFiltering( @@ -1005,7 +997,8 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - try (final VectorCursor cursor = makeVectorCursor(postFilteringFilter)) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(postFilteringFilter)); + final VectorCursor cursor = maker.makeVectorCursor()) { final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1029,7 +1022,8 @@ private List selectColumnValuesMatchingFilterUsingVectorCursor( final String selectColumn ) { - try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter))); + final VectorCursor cursor = maker.makeVectorCursor()) { final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() .makeSingleValueDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn)); @@ -1055,7 +1049,8 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur ) { final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); - try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { + try (final CursorMaker maker = adapter.asCursorMaker(makeVectorCursorBuildSpec(makeFilter(filter))); + final VectorCursor cursor = maker.makeVectorCursor()) { final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); final List values = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index c620917eb01e..9521257c25b1 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -149,11 +149,13 @@ public void testSanity() throws Exception .addOrderByColumn("billy") .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); final Sequence rows = GroupByQueryEngine.process( query, adapter, maker, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), @@ -223,11 +225,13 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception .addOrderByColumn("billy") .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); final Sequence rows = GroupByQueryEngine.process( query, adapter, maker, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), @@ -280,31 +284,31 @@ public void testResetSanity() throws IOException .setInterval(interval) .isDescending(descending) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - Cursor cursor = cursorSequence.limit(1).toList().get(0); - DimensionSelector dimSelector; - - dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); - - index.add( - new MapBasedInputRow( - t.minus(1).getMillis(), - Collections.singletonList("sally"), - ImmutableMap.of("sally", "ah") - ) - ); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + DimensionSelector dimSelector; + + dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + + index.add( + new MapBasedInputRow( + t.minus(1).getMillis(), + Collections.singletonList("sally"), + ImmutableMap.of("sally", "ah") + ) + ); - // Cursor reset should not be affected by out of order values - cursor.reset(); + // Cursor reset should not be affected by out of order values + cursor.reset(); - dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); - Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("sally", "sally")); + Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0))); + } } } @@ -387,11 +391,13 @@ public void testFilterByNull() throws Exception .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - final CursorMaker maker = adapter.asCursorMaker(query.asCursorBuildSpec(null)); + final CursorBuildSpec buildSpec = query.asCursorBuildSpec(null); + final CursorMaker maker = adapter.asCursorMaker(buildSpec); final Sequence rows = GroupByQueryEngine.process( query, adapter, maker, + buildSpec, processingBuffer.get(), null, new GroupByQueryConfig(), @@ -429,41 +435,38 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .setGranularity(Granularities.ALL) .build(); - Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); - - cursors - .map(cursor -> { - DimensionSelector dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - int cardinality = dimSelector.getValueCardinality(); - - //index gets more rows at this point, while other thread is iterating over the cursor - try { - for (int i = 0; i < 1; i++) { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2" + i))); - } - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - int rowNumInCursor = 0; - // and then, cursoring continues in the other thread - while (!cursor.isDone()) { - IndexedInts row = dimSelector.getRow(); - row.forEach(i -> Assert.assertTrue(i < cardinality)); - cursor.advance(); - rowNumInCursor++; - } - Assert.assertEquals(2, rowNumInCursor); - assertCursorsNotEmpty.incrementAndGet(); - - return null; - }) - .toList(); - Assert.assertEquals(1, assertCursorsNotEmpty.get()); + try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); + DimensionSelector dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + int cardinality = dimSelector.getValueCardinality(); + + //index gets more rows at this point, while other thread is iterating over the cursor + try { + for (int i = 0; i < 1; i++) { + index.add(new MapBasedInputRow( + timestamp, + Collections.singletonList("billy"), + ImmutableMap.of("billy", "v2" + i) + )); + } + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + int rowNumInCursor = 0; + // and then, cursoring continues in the other thread + while (!cursor.isDone()) { + IndexedInts row = dimSelector.getRow(); + row.forEach(i -> Assert.assertTrue(i < cardinality)); + cursor.advance(); + rowNumInCursor++; + } + Assert.assertEquals(2, rowNumInCursor); + } } @Test @@ -491,30 +494,23 @@ public void testCursorDictionaryRaceConditionFix() throws Exception .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .setGranularity(Granularities.ALL) .build(); - Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); - - cursors - .map(cursor -> { - DimensionSelector dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - int cardinality = dimSelector.getValueCardinality(); - - int rowNumInCursor = 0; - while (!cursor.isDone()) { - IndexedInts row = dimSelector.getRow(); - row.forEach(i -> Assert.assertTrue(i < cardinality)); - cursor.advance(); - rowNumInCursor++; - } - Assert.assertEquals(5, rowNumInCursor); - assertCursorsNotEmpty.incrementAndGet(); - - return null; - }) - .toList(); - Assert.assertEquals(1, assertCursorsNotEmpty.get()); + try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); + DimensionSelector dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + int cardinality = dimSelector.getValueCardinality(); + + int rowNumInCursor = 0; + while (!cursor.isDone()) { + IndexedInts row = dimSelector.getRow(); + row.forEach(i -> Assert.assertTrue(i < cardinality)); + cursor.advance(); + rowNumInCursor++; + } + Assert.assertEquals(5, rowNumInCursor); + } } @Test @@ -539,86 +535,84 @@ public void testCursoringAndSnapshot() throws Exception .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .setGranularity(Granularities.ALL) .build(); - Sequence cursors = sa.asCursorMaker(buildSpec).makeCursors(); - final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); - - cursors - .map(cursor -> { - DimensionSelector dimSelector1A = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - int cardinalityA = dimSelector1A.getValueCardinality(); - - //index gets more rows at this point, while other thread is iterating over the cursor - try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1"))); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - DimensionSelector dimSelector1B = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - //index gets more rows at this point, while other thread is iterating over the cursor - try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2"))); - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy2"), ImmutableMap.of("billy2", "v3"))); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - DimensionSelector dimSelector1C = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); - - DimensionSelector dimSelector2D = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2")); - //index gets more rows at this point, while other thread is iterating over the cursor - try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3"))); - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", ""))); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - - DimensionSelector dimSelector3E = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("billy3", "billy3")); - - int rowNumInCursor = 0; - // and then, cursoring continues in the other thread - while (!cursor.isDone()) { - IndexedInts rowA = dimSelector1A.getRow(); - rowA.forEach(i -> Assert.assertTrue(i < cardinalityA)); - IndexedInts rowB = dimSelector1B.getRow(); - rowB.forEach(i -> Assert.assertTrue(i < cardinalityA)); - IndexedInts rowC = dimSelector1C.getRow(); - rowC.forEach(i -> Assert.assertTrue(i < cardinalityA)); - IndexedInts rowD = dimSelector2D.getRow(); - // no null id, so should get empty dims array - Assert.assertEquals(0, rowD.size()); - IndexedInts rowE = dimSelector3E.getRow(); - if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(1, rowE.size()); - // the null id - Assert.assertEquals(0, rowE.get(0)); - } else { - Assert.assertEquals(0, rowE.size()); - } - cursor.advance(); - rowNumInCursor++; - } - Assert.assertEquals(2, rowNumInCursor); - assertCursorsNotEmpty.incrementAndGet(); - - return null; - }) - .toList(); - Assert.assertEquals(1, assertCursorsNotEmpty.get()); + try (final CursorMaker maker = sa.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); + final AtomicInteger assertCursorsNotEmpty = new AtomicInteger(0); + + DimensionSelector dimSelector1A = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + int cardinalityA = dimSelector1A.getValueCardinality(); + + //index gets more rows at this point, while other thread is iterating over the cursor + try { + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1"))); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + DimensionSelector dimSelector1B = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + //index gets more rows at this point, while other thread is iterating over the cursor + try { + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2"))); + index.add(new MapBasedInputRow( + timestamp, + Collections.singletonList("billy2"), + ImmutableMap.of("billy2", "v3") + )); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + DimensionSelector dimSelector1C = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); + + DimensionSelector dimSelector2D = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2")); + //index gets more rows at this point, while other thread is iterating over the cursor + try { + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3"))); + index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", ""))); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + + DimensionSelector dimSelector3E = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec("billy3", "billy3")); + + int rowNumInCursor = 0; + // and then, cursoring continues in the other thread + while (!cursor.isDone()) { + IndexedInts rowA = dimSelector1A.getRow(); + rowA.forEach(i -> Assert.assertTrue(i < cardinalityA)); + IndexedInts rowB = dimSelector1B.getRow(); + rowB.forEach(i -> Assert.assertTrue(i < cardinalityA)); + IndexedInts rowC = dimSelector1C.getRow(); + rowC.forEach(i -> Assert.assertTrue(i < cardinalityA)); + IndexedInts rowD = dimSelector2D.getRow(); + // no null id, so should get empty dims array + Assert.assertEquals(0, rowD.size()); + IndexedInts rowE = dimSelector3E.getRow(); + if (NullHandling.replaceWithDefault()) { + Assert.assertEquals(1, rowE.size()); + // the null id + Assert.assertEquals(0, rowE.get(0)); + } else { + Assert.assertEquals(0, rowE.size()); + } + cursor.advance(); + rowNumInCursor++; + } + Assert.assertEquals(2, rowNumInCursor); + } } private static class DictionaryRaceTestFilter implements Filter diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index 31de72b1110d..24becbd4a298 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -306,12 +306,12 @@ public void test_makeCursors_factToCountryLeft() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -363,12 +363,12 @@ public void test_makeCursors_factToCountryLeftUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -418,12 +418,12 @@ public void test_makeCursors_factToCountryInner() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -468,12 +468,12 @@ public void test_makeCursors_factToCountryInnerUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -520,14 +520,14 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -580,14 +580,14 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -636,14 +636,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -667,14 +667,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -697,14 +697,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -730,14 +730,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -762,14 +762,14 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -795,14 +795,14 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -832,14 +832,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -869,14 +869,14 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -906,14 +906,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -943,14 +943,14 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -993,14 +993,14 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1064,14 +1064,14 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1118,12 +1118,12 @@ public void test_makeCursors_factToRegionToCountryLeft() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1175,12 +1175,12 @@ public void test_makeCursors_factToRegionToCountryInnerIncludeNull() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -1241,14 +1241,14 @@ public void test_makeCursors_factToCountryAlwaysTrue() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1300,14 +1300,14 @@ public void test_makeCursors_factToCountryAlwaysFalse() VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1341,14 +1341,14 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1400,14 +1400,14 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1445,14 +1445,14 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() joinableClauses, virtualColumns ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1496,14 +1496,14 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() joinableClauses, virtualColumns ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setVirtualColumns(virtualColumns).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1544,12 +1544,12 @@ public void test_makeCursors_factToCountryUsingExpression() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1589,12 +1589,12 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1636,14 +1636,14 @@ public void test_makeCursors_factToRegionTheWrongWay() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "regionIsoCode", @@ -1687,12 +1687,12 @@ public void test_makeCursors_errorOnNonEquiJoin() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1722,12 +1722,12 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1757,12 +1757,12 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1792,12 +1792,12 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() VirtualColumns.EMPTY ); - JoinTestHelper.readCursors( + JoinTestHelper.readCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of() ); } @@ -1813,14 +1813,14 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1842,14 +1842,14 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1877,7 +1877,7 @@ public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowIS joinFilterPreAnalysis ).asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors().toList(); + ); } @Test @@ -1895,13 +1895,13 @@ public void test_makeCursors_factToCountryLeftWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1932,13 +1932,13 @@ public void test_makeCursors_factToCountryInnerWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -1968,13 +1968,13 @@ public void test_makeCursors_factToCountryRightWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", @@ -2021,13 +2021,13 @@ public void test_makeCursors_factToCountryFullWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), baseFilter, joinableClauses, joinFilterPreAnalysis - ).asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(), + ).asCursorMaker(CursorBuildSpec.FULL_SCAN), ImmutableList.of( "page", "countryIsoCode", diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java index 8e6ceef8fc40..4d5966662ba7 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java @@ -82,10 +82,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -154,10 +154,10 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -205,10 +205,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -267,10 +267,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -333,10 +333,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -398,14 +398,14 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) .setGranularity(Granularities.ALL) .build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -471,14 +471,14 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) .setGranularity(Granularities.ALL) .build() - ).makeCursors(), + ), ImmutableList.of( "page", "v0" @@ -574,10 +574,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -696,10 +696,10 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -785,10 +785,10 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan joinableClauses, joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -830,10 +830,10 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -927,10 +927,10 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" @@ -1012,10 +1012,10 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" @@ -1081,10 +1081,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1134,10 +1134,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1186,10 +1186,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1237,10 +1237,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1289,10 +1289,10 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1341,10 +1341,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1400,10 +1400,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1456,10 +1456,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1506,10 +1506,10 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1555,10 +1555,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1608,10 +1608,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(filter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1660,10 +1660,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1711,10 +1711,10 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", "countryIsoCode", @@ -1773,10 +1773,10 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1839,10 +1839,10 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1915,10 +1915,10 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR + "Equality{leftExpr=user, rightColumn='regionName', includeNull=false}" ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName" @@ -1971,10 +1971,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2046,10 +2046,10 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2244,10 +2244,10 @@ public boolean supportsRequiredColumnRewrite() expectedVirtualColumns = ImmutableSet.of(); } - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2373,10 +2373,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", @@ -2448,10 +2448,10 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName joinFilterPreAnalysis ); - JoinTestHelper.verifyCursors( + JoinTestHelper.verifyCursor( adapter.asCursorMaker( CursorBuildSpec.builder().setFilter(originalFilter).setGranularity(Granularities.ALL).build() - ).makeCursors(), + ), ImmutableList.of( "page", FACT_TO_REGION_PREFIX + "regionName", diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 0746c1047325..c6edcf1b7cd6 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -34,8 +34,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -48,6 +46,7 @@ import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; @@ -325,47 +324,49 @@ public static RowBasedIndexedTable> createRegionsIndexedTabl ); } - public static List readCursors(final Sequence cursors, final List columns) + public static List readCursor(final CursorMaker cursorMaker, final List columns) { - return cursors.flatMap( - cursor -> { - final List> readers = columns - .stream() - .map( - column -> - ColumnProcessors.makeProcessor( - column, - SIMPLE_READER, - cursor.getColumnSelectorFactory() - ) - ) - .collect(Collectors.toList()); - - final List rows = new ArrayList<>(); - - while (!cursor.isDone()) { - final Object[] row = new Object[columns.size()]; - - for (int i = 0; i < row.length; i++) { - row[i] = readers.get(i).get(); - } - - rows.add(row); - cursor.advance(); - } + try { + final Cursor cursor = cursorMaker.makeCursor(); + final List> readers = columns + .stream() + .map( + column -> + ColumnProcessors.makeProcessor( + column, + SIMPLE_READER, + cursor.getColumnSelectorFactory() + ) + ) + .collect(Collectors.toList()); + + final List rows = new ArrayList<>(); - return Sequences.simple(rows); + while (!cursor.isDone()) { + final Object[] row = new Object[columns.size()]; + + for (int i = 0; i < row.length; i++) { + row[i] = readers.get(i).get(); } - ).toList(); + + rows.add(row); + cursor.advance(); + } + + return rows; + } + finally { + cursorMaker.close(); + } } - public static void verifyCursors( - final Sequence cursors, + public static void verifyCursor( + final CursorMaker cursorMaker, final List columns, final List expectedRows ) { - final List rows = readCursors(cursors, columns); + final List rows = readCursor(cursorMaker, columns); for (int i = 0; i < rows.size(); i++) { try { diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index 3abfb020575b..e99e70f909c1 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.join; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -35,9 +34,9 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; -import org.joda.time.DateTime; import org.junit.Test; +import javax.annotation.Nullable; import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -73,7 +72,21 @@ public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch public CursorMaker asCursorMaker(CursorBuildSpec spec) { final CursorMaker delegate = super.asCursorMaker(spec); - return () -> delegate.makeCursors().map(cursor -> new CursorNoAdvance(cursor, countDownLatch)); + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return new CursorNoAdvance(delegate.makeCursor(), countDownLatch); + } + + @Override + public void close() + { + delegate.close(); + } + }; } private static class CursorNoAdvance implements Cursor @@ -93,12 +106,6 @@ public ColumnSelectorFactory getColumnSelectorFactory() return cursor.getColumnSelectorFactory(); } - @Override - public DateTime getTime() - { - return cursor.getTime(); - } - @Override public void advance() { @@ -222,25 +229,25 @@ public void makeCursorAndAdvance() joinFilterPreAnalysis ); - Cursor cursor = Iterables.getOnlyElement( - hashJoinSegmentStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors().toList() - ); + try (final CursorMaker maker = hashJoinSegmentStorageAdapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = maker.makeCursor(); - ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() - { - @Override - public boolean matches(boolean includeUnknown) + ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() { - return false; - } + @Override + public boolean matches(boolean includeUnknown) + { + return false; + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { - } - }); + } + }); - cursor.advance(); + cursor.advance(); + } } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index 6b3d4698ab98..46852491ce08 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -24,9 +24,6 @@ import org.apache.druid.error.DruidException; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.aggregation.AggregationTestHelper; @@ -35,6 +32,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; @@ -352,10 +350,8 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir .setVirtualColumns(virtualColumns) .setGranularity(Granularities.DAY) .build(); - Sequence cursorSequence = storageAdapter.asCursorMaker(buildSpec).makeCursors(); - final Yielder yielder = Yielders.each(cursorSequence); - closer.register(yielder); - final Cursor cursor = yielder.get(); + final CursorMaker maker = closer.register(storageAdapter.asCursorMaker(buildSpec)); + final Cursor cursor = maker.makeCursor(); return cursor.getColumnSelectorFactory(); } diff --git a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java index 8e3ad9c64d4c..7bec5f1fa33d 100644 --- a/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/selector/TestColumnValueSelector.java @@ -87,12 +87,6 @@ public ColumnCapabilities getColumnCapabilities(String column) }; } - @Override - public DateTime getTime() - { - return time; - } - @Override public void advance() { diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 43dd020971e4..ef0ef087e62e 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -29,7 +29,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; @@ -45,6 +44,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -142,9 +142,9 @@ public void test_single_value_string_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); - cursorSequence.accumulate(null, (accumulated, cursor) -> { ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); ExpressionPlan plan = ExpressionPlanner.plan( adapter, @@ -198,9 +198,7 @@ public void test_single_value_string_bindings() cursor.advance(); } - - return null; - }); + } } } @@ -213,9 +211,8 @@ public void test_multi_value_string_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - cursorSequence.accumulate(null, (ignored, cursor) -> { + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // identifier, uses dimension selector supplier supplier, no null coercion @@ -282,8 +279,7 @@ public void test_multi_value_string_bindings() cursor.advance(); } - return ignored; - }); + } } } @@ -296,9 +292,8 @@ public void test_long_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans ExpressionPlan plan = ExpressionPlanner.plan( @@ -333,9 +328,7 @@ public void test_long_bindings() } cursor.advance(); } - - return null; - }); + } } } @@ -348,10 +341,8 @@ public void test_double_bindings() .setInterval(adapter.getInterval()) .setGranularity(Granularities.ALL) .build(); - Sequence cursorSequence = adapter.asCursorMaker(buildSpec).makeCursors(); - - - cursorSequence.accumulate(null, (accumulated, cursor) -> { + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + Cursor cursor = maker.makeCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); // an assortment of plans ExpressionPlan plan = ExpressionPlanner.plan( @@ -386,9 +377,7 @@ public void test_double_bindings() } cursor.advance(); } - - return null; - }); + } } } @@ -671,9 +660,8 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept ); IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - - Sequence cursors = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN).makeCursors(); - int rowsProcessed = cursors.map(cursor -> { + try (final CursorMaker maker = adapter.asCursorMaker(CursorBuildSpec.FULL_SCAN)) { + Cursor cursor = maker.makeCursor(); DimensionSelector xExprSelector = ExpressionSelectors.makeDimensionSelector( cursor.getColumnSelectorFactory(), Parser.parse("concat(x, 'foo')", ExprMacroTable.nil()), @@ -702,10 +690,9 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept rowCount++; cursor.advance(); } - return rowCount; - }).accumulate(0, (in, acc) -> in + acc); - Assert.assertEquals(2, rowsProcessed); + Assert.assertEquals(2, rowCount); + } } private static DimensionSelector dimensionSelectorFromSupplier( diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java index d4b94104f8a2..8a7067015e33 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; @@ -37,6 +36,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; @@ -240,95 +240,95 @@ public static void sanityTestVectorizedExpressionSelectors( .setGranularity(Granularities.ALL) .setVirtualColumns(virtualColumns) .build(); - final VectorCursor cursor = storageAdapter.asCursorMaker(buildSpec).makeVectorCursor(); - - ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); - - int rowCount = 0; - if (capabilities.isDictionaryEncoded().isTrue()) { - SingleValueDimensionVectorSelector selector = cursor.getColumnSelectorFactory().makeSingleValueDimensionSelector( - DefaultDimensionSpec.of("v") - ); - while (!cursor.isDone()) { - int[] row = selector.getRowVector(); - for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - results.add(selector.lookupName(row[i])); + try (final CursorMaker maker = storageAdapter.asCursorMaker(buildSpec)) { + final VectorCursor cursor = maker.makeVectorCursor(); + + ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); + + int rowCount = 0; + if (capabilities.isDictionaryEncoded().isTrue()) { + SingleValueDimensionVectorSelector selector = cursor.getColumnSelectorFactory() + .makeSingleValueDimensionSelector( + DefaultDimensionSpec.of("v") + ); + while (!cursor.isDone()) { + int[] row = selector.getRowVector(); + for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { + results.add(selector.lookupName(row[i])); + } + cursor.advance(); } - cursor.advance(); - } - } else { - VectorValueSelector selector = null; - VectorObjectSelector objectSelector = null; - if (Types.isNumeric(outputType)) { - selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); } else { - objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v"); - } - GroupByVectorColumnSelector groupBySelector = - cursor.getColumnSelectorFactory().makeGroupByVectorColumnSelector("v", DeferExpressionDimensions.ALWAYS); - while (!cursor.isDone()) { - final List resultsVector = new ArrayList<>(); - boolean[] nulls; - switch (outputType.getType()) { - case LONG: - nulls = selector.getNullVector(); - long[] longs = selector.getLongVector(); - for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(nulls != null && nulls[i] ? null : longs[i]); - } - break; - case DOUBLE: - // special case to test floats just to get coverage on getFloatVector - if ("float2".equals(expression)) { + VectorValueSelector selector = null; + VectorObjectSelector objectSelector = null; + if (Types.isNumeric(outputType)) { + selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); + } else { + objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v"); + } + GroupByVectorColumnSelector groupBySelector = + cursor.getColumnSelectorFactory().makeGroupByVectorColumnSelector("v", DeferExpressionDimensions.ALWAYS); + while (!cursor.isDone()) { + final List resultsVector = new ArrayList<>(); + boolean[] nulls; + switch (outputType.getType()) { + case LONG: nulls = selector.getNullVector(); - float[] floats = selector.getFloatVector(); + long[] longs = selector.getLongVector(); for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(nulls != null && nulls[i] ? null : (double) floats[i]); + resultsVector.add(nulls != null && nulls[i] ? null : longs[i]); } - } else { - nulls = selector.getNullVector(); - double[] doubles = selector.getDoubleVector(); - for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(nulls != null && nulls[i] ? null : doubles[i]); + break; + case DOUBLE: + // special case to test floats just to get coverage on getFloatVector + if ("float2".equals(expression)) { + nulls = selector.getNullVector(); + float[] floats = selector.getFloatVector(); + for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { + resultsVector.add(nulls != null && nulls[i] ? null : (double) floats[i]); + } + } else { + nulls = selector.getNullVector(); + double[] doubles = selector.getDoubleVector(); + for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) { + resultsVector.add(nulls != null && nulls[i] ? null : doubles[i]); + } } - } - break; - case STRING: - Object[] objects = objectSelector.getObjectVector(); - for (int i = 0; i < objectSelector.getCurrentVectorSize(); i++, rowCount++) { - resultsVector.add(objects[i]); - } - break; - } + break; + case STRING: + Object[] objects = objectSelector.getObjectVector(); + for (int i = 0; i < objectSelector.getCurrentVectorSize(); i++, rowCount++) { + resultsVector.add(objects[i]); + } + break; + } - verifyGroupBySelector(groupBySelector, resultsVector); - results.addAll(resultsVector); - cursor.advance(); + verifyGroupBySelector(groupBySelector, resultsVector); + results.addAll(resultsVector); + cursor.advance(); + } + } + closer.register(cursor); + + + final Cursor nonVectorized = maker.makeCursor(); + + final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory() + .makeColumnValueSelector("v"); + int rows = 0; + while (!nonVectorized.isDone()) { + Assert.assertEquals( + "Failed at row " + rows, + nonSelector.getObject(), + results.get(rows) + ); + rows++; + nonVectorized.advance(); } - } - closer.register(cursor); - - Sequence cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors(); - - int rowCountCursor = cursors - .map(nonVectorized -> { - final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory() - .makeColumnValueSelector("v"); - int rows = 0; - while (!nonVectorized.isDone()) { - Assert.assertEquals( - "Failed at row " + rows, - nonSelector.getObject(), - results.get(rows) - ); - rows++; - nonVectorized.advance(); - } - return rows; - }).accumulate(0, (acc, in) -> acc + in); - Assert.assertTrue(rowCountCursor > 0); - Assert.assertEquals(rowCountCursor, rowCount); + Assert.assertTrue(rows > 0); + Assert.assertEquals(rows, rowCount); + } } private static void verifyGroupBySelector( diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index c58c686074d0..2cd239ebde0b 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -27,11 +27,9 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; @@ -39,7 +37,6 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; @@ -332,16 +329,17 @@ public Metadata getMetadata() } @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) + public CursorMaker asCursorMaker(CursorBuildSpec spec) { - return null; + return new CursorMaker() + { + @Nullable + @Override + public Cursor makeCursor() + { + return null; + } + }; } }; diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 940ef3ac1d68..3ad58603ea37 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -72,6 +72,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorMaker; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; @@ -308,64 +309,52 @@ public static void runDump( .setGranularity(Granularities.ALL) .build(); - final Sequence cursors = adapter.asCursorMaker(buildSpec).makeCursors(); + try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) { + final Cursor cursor = maker.makeCursor(); - withOutputStream( - new Function() - { - @Override - public Object apply(final OutputStream out) + withOutputStream( + new Function() { - final Sequence sequence = Sequences.map( - cursors, - new Function() - { - @Override - public Object apply(Cursor cursor) - { - ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); - final List selectors = columnNames - .stream() - .map(columnSelectorFactory::makeColumnValueSelector) - .collect(Collectors.toList()); - - while (!cursor.isDone()) { - final Map row = Maps.newLinkedHashMap(); - - for (int i = 0; i < columnNames.size(); i++) { - final String columnName = columnNames.get(i); - final Object value = selectors.get(i).getObject(); - - if (timeISO8601 && columnNames.get(i).equals(ColumnHolder.TIME_COLUMN_NAME)) { - row.put(columnName, new DateTime(value, DateTimeZone.UTC).toString()); - } else { - row.put(columnName, value); - } - } + @Override + public Object apply(final OutputStream out) + { + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final List selectors = columnNames + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); - try { - out.write(objectMapper.writeValueAsBytes(row)); - out.write('\n'); - } - catch (IOException e) { - throw new RuntimeException(e); - } + while (!cursor.isDone()) { + final Map row = Maps.newLinkedHashMap(); - cursor.advance(); - } + for (int i = 0; i < columnNames.size(); i++) { + final String columnName = columnNames.get(i); + final Object value = selectors.get(i).getObject(); - return null; + if (timeISO8601 && columnNames.get(i).equals(ColumnHolder.TIME_COLUMN_NAME)) { + row.put(columnName, new DateTime(value, DateTimeZone.UTC).toString()); + } else { + row.put(columnName, value); } } - ); - evaluateSequenceForSideEffects(sequence); + try { + out.write(objectMapper.writeValueAsBytes(row)); + out.write('\n'); + } + catch (IOException e) { + throw new RuntimeException(e); + } - return null; - } - }, - outputFileName - ); + cursor.advance(); + } + + return null; + } + }, + outputFileName + ); + } } @VisibleForTesting