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 d65d13a324d8..cda80475f53f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -161,9 +161,8 @@ public void queryUsingNative(Blackhole blackhole) private double compute(final Function aggregatorFactory) { - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java index 06f0952b51f0..6672edc5c42c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -148,7 +148,9 @@ public void expressionFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(expressionFilter.toFilter()) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); @@ -166,7 +168,9 @@ public void nativeFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(nativeFilter.toFilter()) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); while (!cursor.isDone()) { 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 8d79a611d0e6..f4dce839cd57 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -42,9 +42,8 @@ import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; -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; @@ -160,7 +159,8 @@ public void timeFloorUsingExpression(Blackhole blackhole) ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); @@ -171,7 +171,8 @@ public void timeFloorUsingExpression(Blackhole blackhole) @Benchmark public void timeFloorUsingExtractionFn(Blackhole blackhole) { - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor @@ -190,15 +191,15 @@ public void timeFloorUsingExtractionFn(Blackhole blackhole) @Benchmark public void timeFloorUsingCursor(Blackhole blackhole) { - final StorageAdapter adapter = new QueryableIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final CursorGranularizer granularizer = CursorGranularizer.create( cursor, QueryableIndexTimeBoundaryInspector.create(index), Cursors.getTimeOrdering(index.getOrdering()), Granularities.HOUR, - adapter.getInterval() + index.getDataInterval() ); final Sequence results = Sequences.simple(granularizer.getBucketIterable()) @@ -241,7 +242,8 @@ public void timeFormatUsingExpression(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector( DefaultDimensionSpec.of("v") @@ -253,7 +255,8 @@ public void timeFormatUsingExpression(Blackhole blackhole) @Benchmark public void timeFormatUsingExtractionFn(Blackhole blackhole) { - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -284,7 +287,8 @@ public void strlenUsingExpressionAsLong(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -307,7 +311,8 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -320,7 +325,8 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) @Benchmark public void strlenUsingExtractionFn(Blackhole blackhole) { - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -346,7 +352,8 @@ public void arithmeticOnLong(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -368,7 +375,8 @@ public void stringConcatAndCompareOnLong(Blackhole blackhole) ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -390,7 +398,8 @@ public void caseSearched1(Blackhole blackhole) ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -412,7 +421,8 @@ public void caseSearched2(Blackhole blackhole) ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -447,7 +457,8 @@ public void caseSearched100(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -476,7 +487,8 @@ public void caseSearchedWithLookup(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -513,7 +525,8 @@ public void caseSearchedWithLookup2(Blackhole blackhole) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java index 5604f7fe8834..2e9ffb644d56 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -155,7 +155,7 @@ public void scan(Blackhole blackhole) .setVirtualColumns(virtualColumns) .build(); final CursorHolder cursorHolder = closer.register( - new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec) + new QueryableIndexCursorFactory(index).makeCursorHolder(buildSpec) ); if (vectorize) { VectorCursor cursor = cursorHolder.asVectorCursor(); 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 65dde3cb65a3..2c0f26901983 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -46,14 +46,14 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; @@ -231,8 +231,8 @@ private IncrementalIndex makeIncIndex() @OutputTimeUnit(TimeUnit.MICROSECONDS) public void stringRead(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, null)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -243,8 +243,8 @@ public void stringRead(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MICROSECONDS) public void longRead(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, null)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -255,8 +255,8 @@ public void longRead(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MICROSECONDS) public void timeFilterNone(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterNone)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterNone)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -267,8 +267,8 @@ public void timeFilterNone(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MICROSECONDS) public void timeFilterHalf(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterHalf)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterHalf)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -279,8 +279,8 @@ public void timeFilterHalf(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MICROSECONDS) public void timeFilterAll(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterAll)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterAll)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -293,8 +293,8 @@ public void readWithPreFilter(Blackhole blackhole) { Filter filter = new SelectorFilter("dimSequential", "199"); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -307,8 +307,8 @@ public void readWithPostFilter(Blackhole blackhole) { Filter filter = new NoBitmapSelectorFilter("dimSequential", "199"); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -321,8 +321,8 @@ public void readWithExFnPreFilter(Blackhole blackhole) { Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -335,8 +335,8 @@ public void readWithExFnPostFilter(Blackhole blackhole) { Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -354,8 +354,8 @@ public void readAndFilter(Blackhole blackhole) ) ); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, andFilter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, andFilter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -370,8 +370,8 @@ public void readOrFilter(Blackhole blackhole) Filter filter2 = new AndFilter(Arrays.asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar"))); Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, orFilter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, orFilter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -386,8 +386,8 @@ public void readOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosionExcept Filter filter2 = new AndFilter(Arrays.asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar"))); Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(orFilter))) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, Filters.toCnf(orFilter))) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -425,8 +425,8 @@ public void readComplexOrFilter(Blackhole blackhole) )) ); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, dimFilter3.toFilter())) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, dimFilter3.toFilter())) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -464,16 +464,16 @@ public void readComplexOrFilterCNF(Blackhole blackhole) throws CNFFilterExplosio )) ); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(dimFilter3.toFilter()))) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, Filters.toCnf(dimFilter3.toFilter()))) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } - private CursorHolder makeCursorHolder(StorageAdapter sa, Filter filter) + private CursorHolder makeCursorHolder(CursorFactory factory, Filter filter) { - return sa.makeCursorHolder( + return factory.makeCursorHolder( CursorBuildSpec.builder() .setFilter(filter) .setInterval(schemaInfo.getDataInterval()) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java index 952e9b188fe9..e654b033dee3 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java @@ -238,7 +238,7 @@ public void hashJoinCursorDimensionSelectors(Blackhole blackhole) private CursorHolder makeCursorHolder() { - return hashJoinSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN); + return hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java index c66ef6baf0a7..d9a98b467d89 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -369,7 +369,7 @@ private static String getLastValue(final Cursor cursor, final String dimension) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void baseSegment(Blackhole blackhole) { - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "countryIsoCode")); } @@ -384,7 +384,7 @@ public void baseSegmentWithFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "countryIsoCode")); } @@ -395,7 +395,7 @@ public void baseSegmentWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupStringKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -411,7 +411,7 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -423,7 +423,7 @@ public void joinLookupStringKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupLongKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -439,7 +439,7 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -451,7 +451,7 @@ public void joinLookupLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableLongKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -467,7 +467,7 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -479,7 +479,7 @@ public void joinIndexedTableLongKeyWithFilter(Blackhole blackhole) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableStringKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -495,7 +495,7 @@ public void joinIndexedTableStringKeyWithFilter(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -510,7 +510,7 @@ public void lookupVirtualColumnStringKey(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); @@ -527,7 +527,7 @@ public void lookupVirtualColumnStringKeyWithFilter(Blackhole blackhole) .setFilter(filter) .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); @@ -542,7 +542,7 @@ public void lookupVirtualColumnLongKey(Blackhole blackhole) final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } @@ -558,7 +558,7 @@ public void lookupVirtualColumnLongKeyWithFilter(Blackhole blackhole) .setVirtualColumns(lookupVirtualColumns) .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java index e059c8a7ea75..e76cfc5b425b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java @@ -23,7 +23,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.MappedByteBufferHandler; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.data.ColumnarLongs; import org.apache.druid.segment.data.CompressedColumnarLongsSupplier; import org.openjdk.jmh.annotations.Benchmark; @@ -118,7 +118,7 @@ public void readSkipping(Blackhole bh) @Benchmark public void readVectorizedContinuous(Blackhole bh) { - long[] vector = new long[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; + long[] vector = new long[QueryContexts.DEFAULT_VECTOR_SIZE]; ColumnarLongs columnarLongs = supplier.get(); int count = columnarLongs.size(); for (int i = 0; i < count; i++) { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java index a8b90e56eea1..a57b7a116c4e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java @@ -284,7 +284,7 @@ public void setupTrial() signature ); final Sequence frameSequence = - FrameSequenceBuilder.fromAdapter(segment.asStorageAdapter()) + FrameSequenceBuilder.fromCursorFactory(segment.asCursorFactory()) .allocator(ArenaMemoryAllocator.createOnHeap(10_000_000)) .frameType(FrameType.ROW_BASED) .frames(); 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 6f0796f412fd..56a41e0fb9b0 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 @@ -35,6 +35,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.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.data.IndexedInts; @@ -44,8 +45,8 @@ import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexCreator; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.serde.ComplexMetrics; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -147,8 +148,8 @@ private IncrementalIndex makeIncIndex() @OutputTimeUnit(TimeUnit.MICROSECONDS) public void read(Blackhole blackhole) { - IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - try (final CursorHolder cursorHolder = makeCursor(sa, null)) { + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(incIndex); + try (final CursorHolder cursorHolder = makeCursor(cursorFactory, null)) { Cursor cursor = cursorHolder.asCursor(); List selectors = new ArrayList<>(); @@ -183,8 +184,8 @@ public void readWithFilters(Blackhole blackhole) ) ); - IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - try (final CursorHolder cursorHolder = makeCursor(sa, filter)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(incIndex); + try (final CursorHolder cursorHolder = makeCursor(cursorFactory, filter)) { Cursor cursor = cursorHolder.asCursor(); List selectors = new ArrayList<>(); @@ -204,14 +205,14 @@ public void readWithFilters(Blackhole blackhole) } } - private CursorHolder makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter) + private CursorHolder makeCursor(CursorFactory factory, DimFilter filter) { CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() .setInterval(schemaInfo.getDataInterval()); if (filter != null) { builder.setFilter(filter.toFilter()); } - return sa.makeCursorHolder(builder.build()); + return factory.makeCursorHolder(builder.build()); } private static DimensionSelector makeDimensionSelector(Cursor cursor, String name) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 06ea2aa9e19d..0e38757f12b1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -57,8 +57,8 @@ import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.generator.GeneratorBasicSchemas; @@ -694,8 +694,8 @@ private static void addSegmentToWalker( } else if (STORAGE_FRAME_ROW.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.ROW_BASED, descriptor.getId() ) @@ -703,8 +703,8 @@ private static void addSegmentToWalker( } else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.COLUMNAR, descriptor.getId() ) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java index 2b5db7413a25..7fa2d38d6868 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java @@ -55,8 +55,8 @@ import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -281,8 +281,8 @@ private static void addSegmentToWalker( } else if (STORAGE_FRAME_ROW.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.ROW_BASED, descriptor.getId() ) @@ -290,8 +290,8 @@ private static void addSegmentToWalker( } else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.COLUMNAR, descriptor.getId() ) diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index c06b56496815..d3edf0827b32 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -35,8 +35,8 @@ import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -104,7 +104,7 @@ public void testTimeseriesWithDistinctCountAgg() throws Exception final Iterable> results = engine.process( query, - new IncrementalIndexStorageAdapter(index), + new IncrementalIndexCursorFactory(index), new IncrementalIndexTimeBoundaryInspector(index), new DefaultTimeseriesQueryMetrics() ).toList(); diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index ff5a0c313634..7e3690cc1cb6 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -33,13 +33,13 @@ import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; -import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.junit.After; import org.junit.Before; @@ -133,8 +133,7 @@ public void testTopNWithDistinctCountAgg() throws Exception final Iterable> results = engine.query( query, - new IncrementalIndexStorageAdapter(index), - new IncrementalIndexTimeBoundaryInspector(index), + new IncrementalIndexSegment(index, SegmentId.dummy(QueryRunnerTestHelper.DATA_SOURCE)), null ).toList(); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index b69b1fe88615..2fea6774e89d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -26,10 +26,10 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -45,13 +45,13 @@ public class ToObjectVectorColumnProcessorFactoryTest extends InitializedNullHandlingTest { - private StorageAdapter adapter; + private CursorFactory cursorFactory; @Before public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - adapter = new QueryableIndexStorageAdapter(index); + cursorFactory = new QueryableIndexCursorFactory(index); } @Test @@ -180,7 +180,7 @@ private CursorHolder makeCursorHolder() ) ) .build(); - return adapter.makeCursorHolder(buildSpec); + return cursorFactory.makeCursorHolder(buildSpec); } private List readColumn(final String column, final int limit) diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java index 3c59bfc366e0..9184d2b14fee 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java @@ -37,8 +37,8 @@ import org.apache.druid.query.extraction.TimeDimExtractionFn; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -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.filter.BaseFilterTest; @@ -92,7 +92,7 @@ public class BloomDimFilterTest extends BaseFilterTest public BloomDimFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) 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 a4b52abee58b..92e5066dc4d7 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 @@ -35,9 +35,7 @@ import org.apache.druid.frame.processor.FrameProcessors; import org.apache.druid.frame.processor.ReturnOrAwait; import org.apache.druid.frame.read.FrameReader; -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.logger.Logger; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.exec.MSQTasks; @@ -46,8 +44,8 @@ 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.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -181,8 +179,8 @@ private void addFrame(final Frame frame) // Reuse input row to avoid redoing allocations. final MSQInputRow inputRow = new MSQInputRow(); - final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final CursorFactory cursorFactory = frameReader.makeCursorFactory(frame); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index a859ea8cd534..470b87d9416d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -153,7 +153,7 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment final Sequence rowSequence = groupingEngine.process( query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), - mappedSegment.asStorageAdapter(), + mappedSegment.asCursorFactory(), mappedSegment.as(TimeBoundaryInspector.class), null ); @@ -187,7 +187,7 @@ protected ReturnOrAwait runWithInputChannel( final Sequence rowSequence = groupingEngine.process( query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), - mappedSegment.asStorageAdapter(), + mappedSegment.asCursorFactory(), mappedSegment.as(TimeBoundaryInspector.class), null ); 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 665cfab89eed..2c4a92efef6c 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 @@ -31,8 +31,7 @@ import org.apache.druid.frame.processor.FrameProcessors; import org.apache.druid.frame.processor.ReturnOrAwait; 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.frame.segment.FrameSegment; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.exec.ResultsContext; import org.apache.druid.segment.BaseObjectColumnValueSelector; @@ -40,13 +39,14 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.run.SqlResults; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.storage.StorageConnector; +import org.apache.druid.timeline.SegmentId; import java.io.IOException; import java.io.OutputStream; @@ -151,8 +151,8 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOEx private void exportFrame(final Frame frame) { - final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Segment segment = new FrameSegment(frame, frameReader, SegmentId.dummy("test")); + try (final CursorHolder cursorHolder = segment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { exportWriter.writeRowEnd(); 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 f402aa604308..fbcae67012a8 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 @@ -65,12 +65,12 @@ import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; @@ -247,15 +247,16 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment if (cursor == null) { final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); - final StorageAdapter adapter = mapSegment(segmentHolder.get()).asStorageAdapter(); - if (adapter == null) { + final Segment mappedSegment = mapSegment(segmentHolder.get()); + final CursorFactory cursorFactory = mappedSegment.asCursorFactory(); + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } final CursorHolder cursorHolder = closer.register( - adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) + cursorFactory.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); final Cursor nextCursor = cursorHolder.asCursor(); @@ -292,15 +293,16 @@ protected ReturnOrAwait runWithInputChannel( final Frame frame = inputChannel.read(); final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("scan")); - final StorageAdapter adapter = mapSegment(frameSegment).asStorageAdapter(); - if (adapter == null) { + final Segment mappedSegment = mapSegment(frameSegment); + final CursorFactory cursorFactory = mappedSegment.asCursorFactory(); + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } final CursorHolder cursorHolder = closer.register( - adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) + cursorFactory.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); final Cursor nextCursor = cursorHolder.asCursor(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java index b0135ccdd4d1..4270fe8bdccc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java @@ -45,8 +45,8 @@ import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; @@ -74,7 +74,7 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private StorageAdapter adapter; + private CursorFactory cursorFactory; private File testDataFile1; private File testDataFile2; private FrameReader frameReader1; @@ -84,11 +84,11 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl public void setUp() throws IOException { final ArenaMemoryAllocator allocator = ArenaMemoryAllocator.createOnHeap(10_000); - adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); // File 1: the entire test dataset. testDataFile1 = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types .allocator(allocator) .frames(), @@ -97,7 +97,7 @@ public void setUp() throws IOException // File 2: just two rows. testDataFile2 = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types .allocator(allocator) .maxRowsPerFrame(1) @@ -106,8 +106,8 @@ public void setUp() throws IOException temporaryFolder.newFile() ); - frameReader1 = FrameReader.create(adapter.getRowSignature()); - frameReader2 = FrameReader.create(adapter.getRowSignature()); + frameReader1 = FrameReader.create(cursorFactory.getRowSignature()); + frameReader2 = FrameReader.create(cursorFactory.getRowSignature()); } @Test @@ -169,7 +169,7 @@ public void testBuildTableAndInlineData() throws IOException Assert.assertEquals(1209, rowsFromStage3.size()); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), Sequences.simple(rowsFromStage3.stream().map(Arrays::asList).collect(Collectors.toList())) ); @@ -178,7 +178,7 @@ public void testBuildTableAndInlineData() throws IOException Assert.assertEquals(2, rowsFromStage4.size()); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false).limit(2), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory).limit(2), Sequences.simple(rowsFromStage4.stream().map(Arrays::asList).collect(Collectors.toList())) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java index 75168e07ad80..439aa148a84c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java @@ -27,15 +27,13 @@ import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameSequenceBuilder; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.StagePartition; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -64,15 +62,15 @@ public void tearDown() throws Exception exec.getExecutorService().awaitTermination(10, TimeUnit.MINUTES); } - protected ReadableInput makeChannelFromAdapter( - final StorageAdapter adapter, + protected ReadableInput makeChannelFromCursorFactory( + final CursorFactory cursorFactory, final List keyColumns, int rowsPerInputFrame ) throws IOException { // Create a single, sorted frame. final FrameSequenceBuilder singleFrameBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(Integer.MAX_VALUE) .sortBy(keyColumns); @@ -86,7 +84,7 @@ protected ReadableInput makeChannelFromAdapter( final FrameReader frameReader = FrameReader.create(signature); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)) + FrameSequenceBuilder.fromCursorFactory(frameReader.makeCursorFactory(frame)) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(rowsPerInputFrame); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java index 0b9b73facde5..9d64fffe23e2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java @@ -47,8 +47,8 @@ import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; @@ -222,14 +222,14 @@ private ReadableInput makeChannelFromRows( signature ); - return makeChannelFromAdapter(segment.asStorageAdapter(), keyColumns); + return makeChannelFromCursorFactory(segment.asCursorFactory(), keyColumns); } - private ReadableInput makeChannelFromAdapter( - final StorageAdapter adapter, + private ReadableInput makeChannelFromCursorFactory( + final CursorFactory cursorFactory, final List keyColumns ) throws IOException { - return makeChannelFromAdapter(adapter, keyColumns, 1000); + return makeChannelFromCursorFactory(cursorFactory, keyColumns, 1000); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java index 4a8ec4358739..dcf5ffe8a87c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java @@ -45,8 +45,8 @@ import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.querykit.FrameProcessorTestBase; import org.apache.druid.msq.test.LimitedFrameWriterFactory; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; -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.join.JoinTestHelper; @@ -1491,17 +1491,17 @@ private ReadableInput makeChannelFromResourceWithLimit( signature ) )) { - final StorageAdapter adapter = segment.asStorageAdapter(); - return makeChannelFromAdapter(adapter, keyColumns); + final CursorFactory cursorFactory = segment.asCursorFactory(); + return makeChannelFromCursorFactory(cursorFactory, keyColumns); } } - private ReadableInput makeChannelFromAdapter( - final StorageAdapter adapter, + private ReadableInput makeChannelFromCursorFactory( + final CursorFactory cursorFactory, final List keyColumns ) throws IOException { - return makeChannelFromAdapter(adapter, keyColumns, rowsPerInputFrame); + return makeChannelFromCursorFactory(cursorFactory, keyColumns, rowsPerInputFrame); } private FrameWriterFactory makeFrameWriterFactory(final RowSignature signature) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java index e300a416705f..632cd66b61a1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java @@ -36,7 +36,7 @@ import org.apache.druid.msq.querykit.FrameProcessorTestBase; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.junit.Assert; import org.junit.Test; @@ -50,11 +50,11 @@ public class QueryResultsFrameProcessorTest extends FrameProcessorTestBase public void sanityTest() throws ExecutionException, InterruptedException, IOException { - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(5) .frameType(FrameType.ROW_BASED) .allocator(ArenaMemoryAllocator.createOnHeap(100_000)); @@ -85,7 +85,7 @@ public void sanityTest() throws ExecutionException, InterruptedException, IOExce FrameReader.create(signature) ); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, signature, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false), rowsFromProcessor ); Assert.assertEquals(Unit.instance(), retVal.get()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java index 205f3495807f..bfb511f949f3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java @@ -46,9 +46,10 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.junit.Assert; import org.junit.Test; @@ -62,11 +63,11 @@ public class ScanQueryFrameProcessorTest extends FrameProcessorTestBase @Test public void test_runWithInputChannel() throws Exception { - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final CursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(5) .frameType(FrameType.ROW_BASED) .allocator(ArenaMemoryAllocator.createOnHeap(100_000)); @@ -86,7 +87,7 @@ public void test_runWithInputChannel() throws Exception Druids.newScanQueryBuilder() .dataSource("test") .intervals(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)) - .columns(adapter.getRowSignature().getColumnNames()) + .columns(cursorFactory.getRowSignature().getColumnNames()) .build(); final StagePartition stagePartition = new StagePartition(new StageId("query", 0), 0); @@ -138,7 +139,7 @@ public void close() ); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, signature, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false), rowsFromProcessor ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java index 927372a3a6ae..0095a8fdb3ae 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java @@ -32,7 +32,7 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -71,9 +71,9 @@ public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest public void setUp() { final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - frameReader = FrameReader.create(adapter.getRowSignature()); - frameList = FrameSequenceBuilder.fromAdapter(adapter) + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + frameReader = FrameReader.create(cursorFactory.getRowSignature()); + frameList = FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) .frames() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index eaa2a9efe5ae..13dac0da6d49 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -61,13 +61,13 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.TestGroupByBuffers; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -440,9 +440,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new QueryableIndexStorageAdapter(index); + return new QueryableIndexCursorFactory(index); } @Override diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index d3ffeda743b3..c17238d4b683 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -134,12 +134,12 @@ import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.TestGroupByBuffers; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; @@ -699,9 +699,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new QueryableIndexStorageAdapter(index); + return new QueryableIndexCursorFactory(index); } @Override @@ -1246,12 +1246,11 @@ public void verifyResults() dataSegment.getDataSource() ); } - final QueryableIndex queryableIndex = indexIO.loadIndex(segmentCacheManager.getSegmentFiles( - dataSegment)); - final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex); + final QueryableIndex queryableIndex = indexIO.loadIndex(segmentCacheManager.getSegmentFiles(dataSegment)); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(queryableIndex); // assert rowSignature - Assert.assertEquals(expectedRowSignature, resultSignatureFromRowSignature(storageAdapter.getRowSignature())); + Assert.assertEquals(expectedRowSignature, resultSignatureFromRowSignature(cursorFactory.getRowSignature())); // assert rollup Assert.assertEquals(expectedRollUp, queryableIndex.getMetadata().isRollup()); @@ -1265,7 +1264,7 @@ public void verifyResults() queryableIndex.getMetadata().getAggregators() ); - for (List row : FrameTestUtil.readRowsFromAdapter(storageAdapter, null, false).toList()) { + for (List row : FrameTestUtil.readRowsFromCursorFactory(cursorFactory).toList()) { // transforming rows for sketch assertions List transformedRow = row.stream() .map(r -> { diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index d1ceccc96a35..5ef56583f7c6 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -47,11 +47,11 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.transform.Transformer; import org.apache.hadoop.fs.Path; @@ -98,12 +98,12 @@ public void initialize(InputSplit split, final TaskAttemptContext context) throw spec = DatasourceInputFormat.getIngestionSpec(context.getConfiguration(), dataSource); logger.info("load schema [%s]", spec); - List adapters = Lists.transform( + List adapters = Lists.transform( segments, - new Function() + new Function() { @Override - public WindowedStorageAdapter apply(WindowedDataSegment segment) + public WindowedCursorFactory apply(WindowedDataSegment segment) { try { logger.info("Getting storage path for segment [%s]", segment.getSegment().getId()); @@ -122,10 +122,7 @@ public WindowedStorageAdapter apply(WindowedDataSegment segment) indexes.add(index); numRows += index.getNumRows(); - return new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(index), - segment.getInterval() - ); + return new WindowedCursorFactory(new QueryableIndexCursorFactory(index), segment.getInterval()); } catch (IOException ex) { throw new RuntimeException(ex); @@ -196,7 +193,7 @@ public static class SegmentReader implements Closeable private Yielder rowYielder; public SegmentReader( - final List adapters, + final List cursorFactories, final TransformSpec transformSpec, final List dims, final List metrics, @@ -207,18 +204,18 @@ public SegmentReader( Sequence rows = Sequences.concat( Iterables.transform( - adapters, - new Function>() + cursorFactories, + new Function>() { @Nullable @Override - public Sequence apply(WindowedStorageAdapter adapter) + public Sequence apply(WindowedCursorFactory windowed) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) - .setInterval(adapter.getInterval()) + .setInterval(windowed.getInterval()) .build(); - final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec); + final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(buildSpec); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.empty(); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 6516b0a0e00d..1e4f62ca6e1f 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -44,12 +44,11 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.LocalDataSegmentPuller; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; @@ -401,10 +400,9 @@ private void testIngestion( new LocalDataSegmentPuller().getSegmentFiles(indexZip, tmpUnzippedSegmentDir); QueryableIndex index = INDEX_IO.loadIndex(tmpUnzippedSegmentDir); - StorageAdapter adapter = new QueryableIndexStorageAdapter(index); DatasourceRecordReader.SegmentReader segmentReader = new DatasourceRecordReader.SegmentReader( - ImmutableList.of(new WindowedStorageAdapter(adapter, windowedDataSegment.getInterval())), + ImmutableList.of(new WindowedCursorFactory(new QueryableIndexCursorFactory(index), windowedDataSegment.getInterval())), TransformSpec.NONE, expectedDimensions, expectedMetrics, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java index 042aa5e9550f..e7f1402606ff 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java @@ -36,22 +36,22 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.filter.SpatialDimFilter; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -61,8 +61,10 @@ import java.io.File; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.stream.Collectors; /** */ @@ -134,10 +136,12 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception .setMaxRowCount(5000) .build() ) { - final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); - final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); + final WindowedCursorFactory ws = new WindowedCursorFactory( + new QueryableIndexCursorFactory(qi), + qi.getDataInterval() + ); final DatasourceRecordReader.SegmentReader segmentReader = new DatasourceRecordReader.SegmentReader( - ImmutableList.of(wsa, wsa), + ImmutableList.of(ws, ws), TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), @@ -162,15 +166,19 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception // Check the index Assert.assertEquals(9, index.size()); - final IncrementalIndexStorageAdapter queryable = new IncrementalIndexStorageAdapter(index); - Assert.assertEquals(2, queryable.getAvailableDimensions().size()); - Assert.assertEquals("host", queryable.getAvailableDimensions().get(0)); - Assert.assertEquals("spatial", queryable.getAvailableDimensions().get(1)); - Assert.assertEquals(ImmutableList.of("visited_sum", "unique_hosts"), queryable.getAvailableMetrics()); + final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test")); + final List dimensions = index.getDimensionNames(false); + Assert.assertEquals(2, dimensions.size()); + Assert.assertEquals("host", dimensions.get(0)); + Assert.assertEquals("spatial", dimensions.get(1)); + Assert.assertEquals( + ImmutableList.of("visited_sum", "unique_hosts"), + Arrays.stream(index.getMetricAggs()).map(AggregatorFactory::getName).collect(Collectors.toList()) + ); // Do a spatial filter final DatasourceRecordReader.SegmentReader segmentReader2 = new DatasourceRecordReader.SegmentReader( - ImmutableList.of(new WindowedStorageAdapter(queryable, Intervals.of("2000/3000"))), + ImmutableList.of(new WindowedCursorFactory(queryable.asCursorFactory(), Intervals.of("2000/3000"))), TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index a85fa73349df..d2da00f38c87 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,15 +49,17 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.utils.CloseableUtils; import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; @@ -118,18 +120,18 @@ public class DruidSegmentReader extends IntermediateRowParsingReader> intermediateRowIterator() throws IOException { final CleanableFile segmentFile = source().fetch(temporaryDirectory, null); - final WindowedStorageAdapter storageAdapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter( - indexIO.loadIndex(segmentFile.file()) - ), + final QueryableIndex queryableIndex = indexIO.loadIndex(segmentFile.file()); + final WindowedCursorFactory windowedCursorFactory = new WindowedCursorFactory( + new QueryableIndexCursorFactory(queryableIndex), intervalFilter ); final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) - .setInterval(storageAdapter.getInterval()) + .setInterval(windowedCursorFactory.getInterval()) .build(); - final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec); + final CursorFactory cursorFactory = windowedCursorFactory.getCursorFactory(); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(cursorBuildSpec); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return CloseableIterators.wrap(Collections.emptyIterator(), cursorHolder); @@ -138,10 +140,7 @@ protected CloseableIterator> intermediateRowIterator() throw // Retain order of columns from the original segments. Useful for preserving dimension order if we're in // schemaless mode. final Set columnsToRead = Sets.newLinkedHashSet( - Iterables.filter( - storageAdapter.getAdapter().getRowSignature().getColumnNames(), - columnsFilter::apply - ) + Iterables.filter(cursorFactory.getRowSignature().getColumnNames(), columnsFilter::apply) ); final Sequence> sequence = cursorToSequence(cursor, columnsToRead).withBaggage(cursorHolder); 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 307a44dbf7ec..cdc7390eb2c3 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 @@ -80,7 +80,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -99,7 +99,7 @@ import org.apache.druid.segment.loading.TombstoneLoadSpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -1702,11 +1702,11 @@ public void testRunWithSpatialDimensions() throws Exception for (DataSegment segment : segments) { final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); @@ -1830,11 +1830,11 @@ public void testRunWithAutoCastDimensions() throws Exception for (DataSegment segment : segments) { final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); @@ -1964,8 +1964,8 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception final File segmentFile = segmentCacheManager.getSegmentFiles(compactSegment); final QueryableIndex queryableIndex = testUtils.getTestIndexIO().loadIndex(segmentFile); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(queryableIndex), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(queryableIndex), compactSegment.getInterval() ); Assert.assertEquals( @@ -1980,7 +1980,7 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception ); try (final CursorHolder cursorHolder = - adapter.getAdapter() + windowed.getCursorFactory() .makeCursorHolder(CursorBuildSpec.builder().setInterval(compactSegment.getInterval()).build())) { final Cursor cursor = cursorHolder.asCursor(); cursor.reset(); @@ -2210,11 +2210,11 @@ private List getCSVFormatRowsFromSegments(List segments) th for (DataSegment segment : segments) { final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index c54446fc4df0..d03ccf465e57 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 @@ -73,7 +73,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnType; @@ -93,7 +93,7 @@ import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -536,11 +536,11 @@ public void testTransformSpec() throws Exception DataSegment segment = segments.get(0); final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(indexIO.loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(indexIO.loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final List> transforms = new ArrayList<>(); @@ -765,12 +765,12 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(indexIO.loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(indexIO.loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final List hashes = new ArrayList<>(); final DimensionSelector selector = cursor.getColumnSelectorFactory() diff --git a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java index 861609ad1882..c79c9b25d677 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java @@ -57,7 +57,7 @@ public int fieldCount() /** * Read a particular field value as an object. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorHolderFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorFactory} * for reading many rows out of a frame. */ public Object readField(final Memory memory, final long rowPosition, final long rowLength, final int fieldNumber) @@ -77,7 +77,7 @@ public Object readField(final Memory memory, final long rowPosition, final long /** * Read an entire row as a list of objects. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorHolderFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorFactory} * for reading many rows out of a frame. */ public List readRow(final Memory memory, final long rowPosition, final long rowLength) diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index f75ced24b12d..4e659b67cff7 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 @@ -29,9 +29,7 @@ import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameCursor; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; @@ -122,8 +120,7 @@ public static FrameCursor makeCursor( // FrameCursorHolderFactory has no closer stuff at all and is totally safe. If this ever changes, this method will // probably need to wrap the cursor in something closeable, or be reworked to just return the CursorHolder so that // callers can deal with closing the stuff. - return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder(cursorBuildSpec) - .asCursor(); + return (FrameCursor) frameReader.makeCursorFactory(frame).makeCursorHolder(cursorBuildSpec).asCursor(); } /** diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index ce169b72b1db..1f23b88fc6f1 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -29,11 +29,11 @@ import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.read.columnar.FrameColumnReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory; -import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.CursorHolderFactory; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -137,15 +137,15 @@ public ColumnCapabilities columnCapabilities(final Frame frame, final String col } /** - * Create a {@link CursorHolderFactory} for the given frame. + * Create a {@link CursorFactory} for the given frame. */ - public CursorHolderFactory makeCursorHolderFactory(final Frame frame) + public CursorFactory makeCursorFactory(final Frame frame) { switch (frame.type()) { case COLUMNAR: - return new ColumnarFrameCursorHolderFactory(frame, signature, columnReaders); + return new ColumnarFrameCursorFactory(frame, signature, columnReaders); case ROW_BASED: - return new RowFrameCursorHolderFactory(frame, this, fieldReaders); + return new RowFrameCursorFactory(frame, this, fieldReaders); default: throw DruidException.defensive("Unrecognized frame type [%s]", frame.type()); } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 82cb092779da..ac5541b893dd 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -19,16 +19,16 @@ package org.apache.druid.frame.segment; -import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory; -import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.query.BaseQuery; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; /** - * An implementation of {@link Cursor} used by {@link RowFrameCursorHolderFactory} - * and {@link ColumnarFrameCursorHolderFactory}. + * An implementation of {@link Cursor} used by {@link RowFrameCursorFactory} + * and {@link ColumnarFrameCursorFactory}. * * Adds the methods {@link #getCurrentRow()} and {@link #setCurrentRow(int)} so the cursor can be moved to * particular rows. diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java index 14b2b0ec6823..3df54bc184da 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java @@ -23,9 +23,9 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -70,9 +70,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new FrameStorageAdapter(frame, frameReader, segmentId.getInterval()); + return frameReader.makeCursorFactory(frame); } @Override 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 deleted file mode 100644 index 51ef56928d40..000000000000 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.frame.segment; - -import org.apache.druid.frame.Frame; -import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.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.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Collections; - -/** - * A {@link StorageAdapter} implementation based on a single {@link Frame}. - * - * This class is used for both columnar and row-based frames. - */ -public class FrameStorageAdapter implements StorageAdapter -{ - private final Frame frame; - private final FrameReader frameReader; - private final Interval interval; - private final CursorHolderFactory cursorFactory; - - public FrameStorageAdapter(Frame frame, FrameReader frameReader, Interval interval) - { - this.frame = frame; - this.frameReader = frameReader; - this.interval = interval; - this.cursorFactory = frameReader.makeCursorHolderFactory(frame); - } - - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public RowSignature getRowSignature() - { - return frameReader.signature(); - } - - @Override - public Indexed getAvailableDimensions() - { - return new ListIndexed<>(frameReader.signature().getColumnNames()); - } - - @Override - public Iterable getAvailableMetrics() - { - return Collections.emptyList(); - } - - @Override - public int getDimensionCardinality(String column) - { - return DimensionDictionarySelector.CARDINALITY_UNKNOWN; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - // It's ok to return null always, because callers are required to handle the case where the min value is not known. - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - // It's ok to return null always, because callers are required to handle the case where the max value is not known. - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return frameReader.columnCapabilities(frame, column); - } - - @Override - public int getNumRows() - { - return frame.numRows(); - } - - @Override - @Nullable - public Metadata getMetadata() - { - return null; - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return cursorFactory.makeCursorHolder(spec); - } -} diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorFactory.java similarity index 86% rename from processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorFactory.java index ee1a01f17fe5..550cbaf88dfa 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorFactory.java @@ -25,7 +25,7 @@ import org.apache.druid.frame.segment.FrameCursor; import org.apache.druid.frame.segment.FrameCursorUtils; import org.apache.druid.frame.segment.FrameFilteredOffset; -import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Order; @@ -35,11 +35,12 @@ import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.vector.FilteredVectorOffset; import org.apache.druid.segment.vector.NoFilterVectorOffset; @@ -54,19 +55,19 @@ import java.util.List; /** - * A {@link CursorHolderFactory} implementation based on a single columnar {@link Frame}. + * A {@link CursorFactory} implementation based on a single columnar {@link Frame}. * * This class is only used for columnar frames. It is not used for row-based frames. * - * @see RowFrameCursorHolderFactory the row-based version + * @see RowFrameCursorFactory the row-based version */ -public class ColumnarFrameCursorHolderFactory implements CursorHolderFactory +public class ColumnarFrameCursorFactory implements CursorFactory { private final Frame frame; private final RowSignature signature; private final List columnReaders; - public ColumnarFrameCursorHolderFactory( + public ColumnarFrameCursorFactory( final Frame frame, final RowSignature signature, final List columnReaders @@ -179,4 +180,25 @@ public void close() } }; } + + @Override + public RowSignature getRowSignature() + { + return signature; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + final int columnNumber = signature.indexOf(column); + + if (columnNumber < 0) { + return null; + } else { + // Better than frameReader.frameSignature().getColumnCapabilities(columnName), because this method has more + // insight into what's actually going on with this column (nulls, multivalue, etc). + return columnReaders.get(columnNumber).readColumn(frame).getCapabilities(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index d3a03ca83bbd..5cf385830792 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java @@ -43,7 +43,7 @@ * A {@link QueryableIndex} implementation based on a single columnar {@link Frame}. There is no internal caching * of columns here, so callers should generally wrap this in a {@link org.apache.druid.segment.ColumnCache}. * - * This class exists so {@link ColumnarFrameCursorHolderFactory} can reuse code meant for regular segment-backed + * This class exists so {@link ColumnarFrameCursorFactory} can reuse code meant for regular segment-backed * {@link QueryableIndex}. Some methods are implemented by throwing {@link UnsupportedOperationException}, wherever * it is not expected that those methods are actually going to be needed. */ diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorFactory.java similarity index 85% rename from processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorFactory.java index f84eb62a4fa8..0132ab720a05 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorFactory.java @@ -26,35 +26,37 @@ import org.apache.druid.frame.segment.FrameCursor; import org.apache.druid.frame.segment.FrameCursorUtils; import org.apache.druid.frame.segment.FrameFilteredOffset; -import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; import org.apache.druid.query.OrderBy; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import java.util.Collections; import java.util.List; /** - * A {@link CursorHolderFactory} implementation based on a single row-based {@link Frame}. + * A {@link CursorFactory} implementation based on a single row-based {@link Frame}. * * This class is only used for row-based frames. * - * @see ColumnarFrameCursorHolderFactory the columnar version + * @see ColumnarFrameCursorFactory the columnar version */ -public class RowFrameCursorHolderFactory implements CursorHolderFactory +public class RowFrameCursorFactory implements CursorFactory { private final Frame frame; private final FrameReader frameReader; private final List fieldReaders; - public RowFrameCursorHolderFactory( + public RowFrameCursorFactory( final Frame frame, final FrameReader frameReader, final List fieldReaders @@ -115,4 +117,17 @@ public List getOrdering() } }; } + + @Override + public RowSignature getRowSignature() + { + return frameReader.signature(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return frameReader.signature().getColumnCapabilities(column); + } } diff --git a/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java b/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java index 62b42b15c782..2d3104a0041f 100644 --- a/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java @@ -25,9 +25,8 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.segment.FilteredStorageAdapter; +import org.apache.druid.segment.FilteredSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.WrappedSegmentReference; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -135,12 +134,7 @@ public Function createSegmentMapFunction( ); return JvmUtils.safeAccumulateThreadCpuTime( cpuTimeAccumulator, - () -> - baseSegment -> - new WrappedSegmentReference( - segmentMapFn.apply(baseSegment), - storageAdapter -> new FilteredStorageAdapter(storageAdapter, filter) - ) + () -> baseSegment -> new FilteredSegment(segmentMapFn.apply(baseSegment), filter) ); } 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 38a8b1e91d5b..68d3db9a36d3 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -21,9 +21,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.planning.DataSourceAnalysis; @@ -88,10 +86,9 @@ public Sequence getRowsAsSequence() Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); - final CursorHolder holder = - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder( - CursorBuildSpec.FULL_SCAN - ); + final CursorHolder holder = frameReader.makeCursorFactory(frame).makeCursorHolder( + CursorBuildSpec.FULL_SCAN + ); return Sequences.simple(Collections.singletonList(holder.asCursor())).withBaggage(holder); } ); diff --git a/processing/src/main/java/org/apache/druid/query/QueryContext.java b/processing/src/main/java/org/apache/druid/query/QueryContext.java index 8b2f7e27e686..52395275967b 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContext.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java @@ -26,7 +26,6 @@ import org.apache.druid.query.QueryContexts.Vectorize; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.TypedInFilter; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import javax.annotation.Nullable; import java.io.IOException; @@ -341,7 +340,7 @@ public Vectorize getVectorizeVirtualColumns(Vectorize defaultValue) public int getVectorSize() { - return getVectorSize(QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE); + return getVectorSize(QueryContexts.DEFAULT_VECTOR_SIZE); } public int getVectorSize(int defaultSize) diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index d3545a8494ae..afdc5a552f0c 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -106,6 +106,7 @@ public class QueryContexts public static final boolean DEFAULT_USE_RESULTLEVEL_CACHE = true; public static final Vectorize DEFAULT_VECTORIZE = Vectorize.TRUE; public static final Vectorize DEFAULT_VECTORIZE_VIRTUAL_COLUMN = Vectorize.TRUE; + public static final int DEFAULT_VECTOR_SIZE = 512; public static final int DEFAULT_PRIORITY = 0; public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0; public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); diff --git a/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java b/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java index b330ba5ce19a..06acca13a754 100644 --- a/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java @@ -26,9 +26,8 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.UnnestStorageAdapter; +import org.apache.druid.segment.UnnestSegment; import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.WrappedSegmentReference; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -146,12 +145,7 @@ public Function createSegmentMapFunction( ); return JvmUtils.safeAccumulateThreadCpuTime( cpuTimeAccumulator, - () -> - baseSegment -> - new WrappedSegmentReference( - segmentMapFn.apply(baseSegment), - storageAdapter -> new UnnestStorageAdapter(storageAdapter, virtualColumn, unnestFilter) - ) + () -> baseSegment -> new UnnestSegment(segmentMapFn.apply(baseSegment), virtualColumn, unnestFilter) ); } diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java index c24f6f4f90af..2b4a0cbb5715 100644 --- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java @@ -85,7 +85,7 @@ private static class DataSourceMetadataQueryRunner implements QueryRunner getToolchest() private static class GroupByQueryRunner implements QueryRunner { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; @Nullable private final TimeBoundaryInspector timeBoundaryInspector; private final GroupingEngine groupingEngine; public GroupByQueryRunner(Segment segment, final GroupingEngine groupingEngine) { - this.adapter = segment.asStorageAdapter(); + this.cursorFactory = segment.asCursorFactory(); this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); this.groupingEngine = groupingEngine; } @@ -110,7 +110,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return groupingEngine.process( (GroupByQuery) query, - adapter, + cursorFactory, timeBoundaryInspector, (GroupByQueryMetrics) queryPlus.getQueryMetrics() ); 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 12b186afc72f..0962e5400e28 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 @@ -70,6 +70,7 @@ import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine; import org.apache.druid.query.groupby.epinephelinae.GroupByResultMergeFn; import org.apache.druid.query.groupby.epinephelinae.GroupByRowProcessor; +import org.apache.druid.query.groupby.epinephelinae.GroupingSelector; import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.LimitSpec; @@ -77,9 +78,9 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -459,30 +460,30 @@ public QueryRunner mergeRunners( } /** - * Process a groupBy query on a single {@link StorageAdapter}. This is used by + * Process a groupBy query on a single {@link CursorFactory}. This is used by * {@link GroupByQueryRunnerFactory#createRunner} to create per-segment * QueryRunners. * * This method is only called on data servers, like Historicals (not the Broker). * * @param query the groupBy query - * @param storageAdapter storage adatper for the segment in question + * @param cursorFactory cursor factory for the segment in question * @param timeBoundaryInspector time boundary inspector for the segment in question * - * @return result sequence for the storage adapter + * @return result sequence for the cursor factory */ public Sequence process( GroupByQuery query, - StorageAdapter storageAdapter, + CursorFactory cursorFactory, @Nullable TimeBoundaryInspector timeBoundaryInspector, @Nullable GroupByQueryMetrics groupByQueryMetrics ) { final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query); - if (storageAdapter == null) { + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } @@ -505,9 +506,9 @@ public Sequence process( : DateTimes.utc(Long.parseLong(fudgeTimestampString)); final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, groupByQueryMetrics); - final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); + final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec)); - final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); + final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory); // group by specific vectorization check final boolean canVectorize = cursorHolder.canVectorize() && @@ -517,7 +518,6 @@ public Sequence process( if (shouldVectorize) { result = VectorGroupByEngine.process( query, - storageAdapter, timeBoundaryInspector, cursorHolder, bufferHolder.get(), @@ -529,7 +529,6 @@ public Sequence process( } else { result = GroupByQueryEngine.process( query, - storageAdapter, timeBoundaryInspector, cursorHolder, buildSpec, @@ -874,7 +873,8 @@ public static CursorBuildSpec makeCursorBuildSpec(GroupByQuery query, @Nullable public static int getCardinalityForArrayAggregation( GroupByQueryConfig querySpecificConfig, GroupByQuery query, - StorageAdapter storageAdapter, + ColumnInspector columnInspector, + List groupingSelectors, ByteBuffer buffer ) { @@ -893,7 +893,7 @@ public static int getCardinalityForArrayAggregation( } else if (dimensions.size() == 1) { // Only real columns can use array-based aggregation, since virtual columns cannot currently report their // cardinality. We need to check if a virtual column exists with the same name, since virtual columns can shadow - // real columns, and we might miss that since we're going directly to the StorageAdapter (which only knows about + // real columns, and we might miss that since we're going directly to the CursorFactory (which only knows about // real columns). if (query.getVirtualColumns().exists(Iterables.getOnlyElement(dimensions).getDimension())) { return -1; @@ -905,8 +905,8 @@ public static int getCardinalityForArrayAggregation( } final String columnName = Iterables.getOnlyElement(dimensions).getDimension(); - columnCapabilities = storageAdapter.getColumnCapabilities(columnName); - cardinality = storageAdapter.getDimensionCardinality(columnName); + columnCapabilities = columnInspector.getColumnCapabilities(columnName); + cardinality = groupingSelectors.get(0).getValueCardinality(); } else { // Cannot use array-based aggregation with more than one dimension. return -1; 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 c6a32d364163..2e67a892b01d 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 @@ -53,7 +53,6 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -64,18 +63,20 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.stream.Stream; /** - * Contains logic to process a groupBy query on a single {@link StorageAdapter} in a non-vectorized manner. - * Processing returns a {@link Sequence} of {@link ResultRow} objects that are not guaranteed to be in any particular - * order, and may not even be fully grouped. It is expected that a downstream {@link GroupByMergingQueryRunner} will - * finish grouping these results. + * Contains logic to process a groupBy query on a single {@link org.apache.druid.segment.CursorFactory} in a + * non-vectorized manner. Processing returns a {@link Sequence} of {@link ResultRow} objects that are not guaranteed + * to be in any particular order, and may not even be fully grouped. It is expected that a downstream + * {@link GroupByMergingQueryRunner} will finish grouping these results. *

- * This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals. + * This code runs on anything that processes {@link org.apache.druid.segment.CursorFactory} directly, typically data + * servers like Historicals. *

* Used for non-vectorized processing by {@link GroupingEngine#process}. * @@ -95,7 +96,6 @@ private GroupByQueryEngine() public static Sequence process( final GroupByQuery query, - final StorageAdapter storageAdapter, @Nullable final TimeBoundaryInspector timeBoundaryInspector, final CursorHolder cursorHolder, final CursorBuildSpec buildSpec, @@ -126,7 +126,7 @@ public static Sequence process( query.getDimensions(), columnSelectorFactory ); - GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length]; + final GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length]; int curPos = 0; for (int i = 0; i < dims.length; i++) { dims[i] = new GroupByColumnSelectorPlus( @@ -140,7 +140,8 @@ public static Sequence process( final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( querySpecificConfig, query, - storageAdapter, + columnSelectorFactory, + Arrays.asList(dims), processingBuffer ); diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupingSelector.java similarity index 59% rename from processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java rename to processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupingSelector.java index e191f033f593..6d90164136f9 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupingSelector.java @@ -17,9 +17,19 @@ * under the License. */ -package org.apache.druid.segment; +package org.apache.druid.query.groupby.epinephelinae; -public interface CursorHolderFactory +import org.apache.druid.segment.DimensionDictionarySelector; + +public interface GroupingSelector { - CursorHolder makeCursorHolder(CursorBuildSpec spec); + /** + * Returns the value cardinality if it is known (such as when backed by a {@link DimensionDictionarySelector}). If + * there are only null values, this method should return 1. If the cardinality is not known, returns + * {@link DimensionDictionarySelector#CARDINALITY_UNKNOWN}. + */ + default int getValueCardinality() + { + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java index 4eb0e9588a80..23ff05474e4b 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java @@ -20,8 +20,12 @@ package org.apache.druid.query.groupby.epinephelinae.column; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.groupby.epinephelinae.GroupingSelector; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionDictionarySelector; -public class GroupByColumnSelectorPlus extends ColumnSelectorPlus +public class GroupByColumnSelectorPlus extends ColumnSelectorPlus implements + GroupingSelector { /** * Indicates the offset of this dimension's value within ResultRows. @@ -58,4 +62,14 @@ public int getResultRowPosition() { return resultRowPosition; } + + @Override + public int getValueCardinality() + { + final ColumnValueSelector selector = getSelector(); + if (selector instanceof DimensionDictionarySelector) { + return ((DimensionDictionarySelector) selector).getValueCardinality(); + } + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java index 774d51c58691..60ccbfadddb1 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java @@ -21,6 +21,7 @@ import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.epinephelinae.GroupingSelector; import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer; /** @@ -35,7 +36,7 @@ * @see GroupByVectorColumnProcessorFactory * @see org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy the nonvectorized version */ -public interface GroupByVectorColumnSelector +public interface GroupByVectorColumnSelector extends GroupingSelector { /** * Get the size in bytes of the key parts generated by this column. diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java index d7bbb21e59e2..e39653e4b430 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java @@ -41,6 +41,12 @@ public int getGroupingKeySize() return 0; } + @Override + public int getValueCardinality() + { + return 1; + } + @Override public int writeKeys(WritableMemory keySpace, int keySize, int keyOffset, int startRow, int endRow) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java index 738fa1cb7b36..57ad933188f2 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java @@ -39,6 +39,12 @@ public int getGroupingKeySize() return Integer.BYTES; } + @Override + public int getValueCardinality() + { + return selector.getValueCardinality(); + } + @Override public int writeKeys( final WritableMemory keySpace, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index c6f52e0bd218..78fa8e532a33 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -45,7 +45,6 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -63,8 +62,9 @@ import java.util.stream.Collectors; /** - * Contains logic to process a groupBy query on a single {@link StorageAdapter} in a vectorized manner. - * This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals. + * Contains logic to process a groupBy query on a single {@link org.apache.druid.segment.CursorFactory} in a vectorized + * manner. This code runs on anything that processes {@link org.apache.druid.segment.CursorFactory} directly, typically + * data servers like Historicals. *

* Used for vectorized processing by {@link GroupingEngine#process}. * @@ -79,7 +79,6 @@ private VectorGroupByEngine() public static Sequence process( final GroupByQuery query, - final StorageAdapter storageAdapter, @Nullable TimeBoundaryInspector timeBoundaryInspector, final CursorHolder cursorHolder, final ByteBuffer processingBuffer, @@ -145,7 +144,6 @@ public void close() query, config, processingConfig, - storageAdapter, timeBoundaryInspector, cursor, cursorHolder.getTimeOrder(), @@ -209,7 +207,6 @@ static class VectorGroupByEngineIterator implements CloseableIterator private final GroupByQuery query; private final GroupByQueryConfig querySpecificConfig; private final DruidProcessingConfig processingConfig; - private final StorageAdapter storageAdapter; private final VectorCursor cursor; private final List selectors; private final ByteBuffer processingBuffer; @@ -241,7 +238,6 @@ static class VectorGroupByEngineIterator implements CloseableIterator final GroupByQuery query, final GroupByQueryConfig querySpecificConfig, final DruidProcessingConfig processingConfig, - final StorageAdapter storageAdapter, @Nullable TimeBoundaryInspector timeBoundaryInspector, final VectorCursor cursor, final Order timeOrder, @@ -254,7 +250,6 @@ static class VectorGroupByEngineIterator implements CloseableIterator this.query = query; this.querySpecificConfig = querySpecificConfig; this.processingConfig = processingConfig; - this.storageAdapter = storageAdapter; this.cursor = cursor; this.selectors = selectors; this.processingBuffer = processingBuffer; @@ -330,11 +325,13 @@ public void close() throws IOException VectorGrouper makeGrouper() { final VectorGrouper grouper; + final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( querySpecificConfig, query, - storageAdapter, + columnSelectorFactory, + selectors, processingBuffer ); @@ -342,7 +339,7 @@ VectorGrouper makeGrouper() grouper = new BufferArrayGrouper( Suppliers.ofInstance(processingBuffer), AggregatorAdapters.factorizeVector( - cursor.getColumnSelectorFactory(), + columnSelectorFactory, query.getAggregatorSpecs() ), cardinalityForArrayAggregation 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 6d0629c4dcec..d0015ba80277 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 @@ -29,11 +29,12 @@ 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.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -46,7 +47,6 @@ import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -80,32 +80,31 @@ public SegmentAnalyzer(EnumSet analysisTypes) public long numRows(Segment segment) { - return Preconditions.checkNotNull(segment, "segment").asStorageAdapter().getNumRows(); + return Preconditions.checkNotNull(segment.as(PhysicalSegmentInspector.class), "PhysicalSegmentInspector") + .getNumRows(); } public Map analyze(Segment segment) { Preconditions.checkNotNull(segment, "segment"); + final PhysicalSegmentInspector segmentInspector = segment.as(PhysicalSegmentInspector.class); - // index is null for incremental-index-based segments, but storageAdapter is always available - final QueryableIndex index = segment.asQueryableIndex(); - final StorageAdapter storageAdapter = segment.asStorageAdapter(); + // index is null for incremental-index-based segments, but segmentInspector should always be available + final QueryableIndex index = segment.as(QueryableIndex.class); - // get length and column names from storageAdapter - final int numRows = storageAdapter.getNumRows(); + final int numRows = segmentInspector != null ? segmentInspector.getNumRows() : 0; // Use LinkedHashMap to preserve column order. final Map columns = new LinkedHashMap<>(); - final RowSignature rowSignature = storageAdapter.getRowSignature(); + final RowSignature rowSignature = segment.asCursorFactory().getRowSignature(); for (String columnName : rowSignature.getColumnNames()) { final ColumnCapabilities capabilities; - if (storageAdapter instanceof IncrementalIndexStorageAdapter) { - // See javadocs for getSnapshotColumnCapabilities for a discussion of why we need to do this. - capabilities = ((IncrementalIndexStorageAdapter) storageAdapter).getSnapshotColumnCapabilities(columnName); + if (segmentInspector != null) { + capabilities = segmentInspector.getColumnCapabilities(columnName); } else { - capabilities = storageAdapter.getColumnCapabilities(columnName); + capabilities = null; } if (capabilities == null) { @@ -133,7 +132,7 @@ public Map analyze(Segment segment) if (index != null) { analysis = analyzeStringColumn(capabilities, index.getColumnHolder(columnName)); } else { - analysis = analyzeStringColumn(capabilities, storageAdapter, columnName); + analysis = analyzeStringColumn(capabilities, segmentInspector, segment.asCursorFactory(), columnName); } break; case ARRAY: @@ -255,7 +254,8 @@ private ColumnAnalysis analyzeStringColumn( private ColumnAnalysis analyzeStringColumn( final ColumnCapabilities capabilities, - final StorageAdapter storageAdapter, + @Nullable final PhysicalSegmentInspector analysisInspector, + final CursorFactory cursorFactory, final String columnName ) { @@ -265,12 +265,17 @@ private ColumnAnalysis analyzeStringColumn( Comparable min = null; Comparable max = null; - if (analyzingCardinality()) { - cardinality = storageAdapter.getDimensionCardinality(columnName); + if (analyzingCardinality() && analysisInspector != null) { + cardinality = analysisInspector.getDimensionCardinality(columnName); + } + + if (analyzingMinMax() && analysisInspector != null) { + min = analysisInspector.getMinValue(columnName); + max = analysisInspector.getMaxValue(columnName); } if (analyzingSize()) { - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor != null) { @@ -291,11 +296,6 @@ private ColumnAnalysis analyzeStringColumn( } } - if (analyzingMinMax()) { - min = storageAdapter.getMinValue(columnName); - max = storageAdapter.getMaxValue(columnName); - } - return ColumnAnalysis.builder() .withCapabilities(capabilities) .withSize(size) diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 827c3a86b07f..a2edb6638863 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -47,9 +47,11 @@ import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.Segment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -117,7 +119,7 @@ public Sequence run(QueryPlus inQ, ResponseCon final Map aggregators; Metadata metadata = null; if (updatedQuery.hasAggregators()) { - metadata = segment.asStorageAdapter().getMetadata(); + metadata = getMetadata(segment); if (metadata != null && metadata.getAggregators() != null) { aggregators = new HashMap<>(); for (AggregatorFactory aggregator : metadata.getAggregators()) { @@ -133,7 +135,7 @@ public Sequence run(QueryPlus inQ, ResponseCon final TimestampSpec timestampSpec; if (updatedQuery.hasTimestampSpec()) { if (metadata == null) { - metadata = segment.asStorageAdapter().getMetadata(); + metadata = getMetadata(segment); } timestampSpec = metadata != null ? metadata.getTimestampSpec() : null; } else { @@ -143,7 +145,7 @@ public Sequence run(QueryPlus inQ, ResponseCon final Granularity queryGranularity; if (updatedQuery.hasQueryGranularity()) { if (metadata == null) { - metadata = segment.asStorageAdapter().getMetadata(); + metadata = getMetadata(segment); } queryGranularity = metadata != null ? metadata.getQueryGranularity() : null; } else { @@ -153,7 +155,7 @@ public Sequence run(QueryPlus inQ, ResponseCon Boolean rollup = null; if (updatedQuery.hasRollup()) { if (metadata == null) { - metadata = segment.asStorageAdapter().getMetadata(); + metadata = getMetadata(segment); } rollup = metadata != null ? metadata.isRollup() : null; if (rollup == null) { @@ -255,4 +257,14 @@ public QueryToolChest getToolchest() { return toolChest; } + + @Nullable + private Metadata getMetadata(Segment segment) + { + PhysicalSegmentInspector inspector = segment.as(PhysicalSegmentInspector.class); + if (inspector != null) { + return inspector.getMetadata(); + } + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java index 8d3f741087fc..f5c600b96ffb 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -64,7 +64,7 @@ * for the base or leaf datasources to include additional joins. * * The base datasource is the one that will be considered by the core Druid query stack for scanning via - * {@link org.apache.druid.segment.Segment} and {@link org.apache.druid.segment.StorageAdapter}. The other leaf + * {@link org.apache.druid.segment.Segment} and {@link org.apache.druid.segment.CursorFactory}. The other leaf * datasources must be joinable onto the base data. * * The idea here is to keep things simple and dumb. So we focus only on identifying left-leaning join trees, which map diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 7ce3df8e0668..dc7a4a355644 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -41,9 +41,9 @@ import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.NaiveSortMaker; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowAdapter; -import org.apache.druid.segment.RowBasedStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -374,9 +374,9 @@ public NaiveSortMaker toNaiveSortMaker() @SuppressWarnings("unused") @SemanticCreator - public StorageAdapter toStorageAdapter() + public CursorFactory toCursorFactory() { - return new RowBasedStorageAdapter(Sequences.simple(rows), rowAdapter, rowSignature); + return new RowBasedCursorFactory<>(Sequences.simple(rows), rowAdapter, rowSignature); } private class MyClusteredGroupPartitioner implements ClusteredGroupPartitioner diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumns.java similarity index 73% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumns.java index 2d49cc324400..6fa74660f7df 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumns.java @@ -19,6 +19,8 @@ package org.apache.druid.query.rowsandcols; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; import org.apache.druid.frame.write.FrameWriter; @@ -30,8 +32,8 @@ 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.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nonnull; @@ -39,24 +41,25 @@ import java.util.Collections; /** - * Provides {@link RowsAndColumns} on top of a {@link StorageAdapter}. + * Provides {@link RowsAndColumns} on top of a {@link CursorFactory}. */ -public class StorageAdapterRowsAndColumns implements CloseableShapeshifter, RowsAndColumns +public class CursorFactoryRowsAndColumns implements CloseableShapeshifter, RowsAndColumns { - private final StorageAdapter storageAdapter; - private RowsAndColumns materialized; + private final CursorFactory cursorFactory; + private final Supplier materialized; - public StorageAdapterRowsAndColumns(StorageAdapter storageAdapter) + public CursorFactoryRowsAndColumns(CursorFactory cursorFactory) { - this.storageAdapter = storageAdapter; + this.cursorFactory = cursorFactory; + this.materialized = Suppliers.memoize(() -> materialize(cursorFactory)); } @SuppressWarnings("unchecked") @Override public T as(Class clazz) { - if (StorageAdapter.class == clazz) { - return (T) storageAdapter; + if (CursorFactory.class == clazz) { + return (T) cursorFactory; } return null; } @@ -64,19 +67,19 @@ public T as(Class clazz) @Override public Collection getColumnNames() { - return storageAdapter.getRowSignature().getColumnNames(); + return cursorFactory.getRowSignature().getColumnNames(); } @Override public int numRows() { - return storageAdapter.getNumRows(); + return materialized.get().numRows(); } @Override public Column findColumn(String name) { - return getRealRAC().findColumn(name); + return materialized.get().findColumn(name); } @Override @@ -84,25 +87,16 @@ public void close() { } - protected RowsAndColumns getRealRAC() - { - if (materialized == null) { - materialized = materialize(storageAdapter); - } - return materialized; - } - @Nonnull - private static RowsAndColumns materialize(StorageAdapter as) + private static RowsAndColumns materialize(CursorFactory cursorFactory) { - try (final CursorHolder cursorHolder = as.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); + final RowSignature rowSignature = cursorFactory.getRowSignature(); if (cursor == null) { return new EmptyRowsAndColumns(); } - - final RowSignature rowSignature = as.getRowSignature(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 6be9500b9fe3..a05b31dc2cb4 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 @@ -46,8 +46,8 @@ 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.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; @@ -185,11 +185,11 @@ private Pair materialize() throw new ISE("Cannot reorder[%s] scan data right now", ordering); } - final StorageAdapter as = base.as(StorageAdapter.class); + final CursorFactory as = base.as(CursorFactory.class); if (as == null) { return naiveMaterialize(base); } else { - return materializeStorageAdapter(as); + return materializeCursorFactory(as); } } @@ -205,7 +205,7 @@ private void reset(RowsAndColumns rac) } @Nullable - private Pair materializeStorageAdapter(StorageAdapter as) + private Pair materializeCursorFactory(CursorFactory cursorFactory) { final Collection cols; if (viewableColumns != null) { @@ -228,7 +228,7 @@ private Pair materializeStorageAdapter(StorageAdapter as) if (virtualColumns != null) { builder.setVirtualColumns(virtualColumns); } - try (final CursorHolder cursorHolder = as.makeCursorHolder(builder.build())) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(builder.build())) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java index 71c2541b387c..e99a3f7f3139 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java @@ -23,13 +23,11 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.CloseableShapeshifter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -86,8 +84,8 @@ public Column findColumn(String name) @Override public T as(Class clazz) { - if (StorageAdapter.class.equals(clazz)) { - return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY); + if (CursorFactory.class.equals(clazz)) { + return (T) FrameReader.create(signature).makeCursorFactory(frame); } return null; } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java index 73fc72a1ee48..35f42f6b9e4f 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java @@ -26,9 +26,9 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nonnull; @@ -109,9 +109,9 @@ public void close() throws IOException @SuppressWarnings("unused") @SemanticCreator - public StorageAdapter toStorageAdapter() + public CursorFactory toCursorFactory() { - return new QueryableIndexStorageAdapter(index); + return new QueryableIndexCursorFactory(index); } @SuppressWarnings("unused") diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java index fa17984e9ba5..865a24e5d6da 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java @@ -25,12 +25,10 @@ import org.apache.druid.frame.field.FieldReader; import org.apache.druid.frame.field.FieldReaders; 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.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.CloseableShapeshifter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -94,8 +92,8 @@ public Column findColumn(String name) @Override public T as(Class clazz) { - if (StorageAdapter.class.equals(clazz)) { - return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY); + if (CursorFactory.class.equals(clazz)) { + return (T) FrameReader.create(signature).makeCursorFactory(frame); } return null; } 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 9dd72d15fa42..a24dc78468bd 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 @@ -38,10 +38,10 @@ 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.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -75,20 +75,20 @@ public Sequence process( if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(Order.NONE)) { return Sequences.empty(); } + if (segment.isTombstone()) { + return Sequences.empty(); + } + final boolean hasTimeout = query.context().hasTimeout(); final Long timeoutAt = responseContext.getTimeoutTime(); - final StorageAdapter adapter = segment.asStorageAdapter(); + final CursorFactory cursorFactory = segment.asCursorFactory(); - if (adapter == null) { + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } - if (adapter.isFromTombstone()) { - return Sequences.empty(); - } - final List allColumns = new ArrayList<>(); if (query.getColumns() != null && !query.getColumns().isEmpty()) { @@ -99,7 +99,7 @@ public Sequence process( } else { final Set availableColumns = Sets.newLinkedHashSet( Iterables.concat( - adapter.getRowSignature().getColumnNames(), + cursorFactory.getRowSignature().getColumnNames(), Iterables.transform( Arrays.asList(query.getVirtualColumns().getVirtualColumns()), VirtualColumn::getOutputName @@ -118,7 +118,7 @@ public Sequence process( // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); - final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); if (Order.NONE != query.getTimeOrder() && Cursors.getTimeOrdering(cursorHolder.getOrdering()) != query.getTimeOrder()) { final String failureReason = StringUtils.format( 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 b6d0cb5f11a5..01ad04e7aa8a 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -26,10 +26,10 @@ 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.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.filter.Filters; import java.util.List; @@ -51,13 +51,14 @@ private CursorOnlyStrategy(SearchQuery query) @Override public List getExecutionPlan(SearchQuery query, Segment segment) { - final StorageAdapter adapter = segment.asStorageAdapter(); - final List dimensionSpecs = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions()); - return ImmutableList.of(new CursorBasedExecutor( - query, - segment, - dimensionSpecs - )); + final List dimensionSpecs = getDimsToSearch(segment, query.getDimensions()); + return ImmutableList.of( + new CursorBasedExecutor( + query, + segment, + dimensionSpecs + ) + ); } public static class CursorBasedExecutor extends SearchQueryExecutor @@ -74,7 +75,7 @@ public CursorBasedExecutor( @Override public Object2IntRBTreeMap execute(final int limit) { - final StorageAdapter adapter = segment.asStorageAdapter(); + final CursorFactory adapter = segment.asCursorFactory(); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(query.getSingleInterval()) .setFilter( diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java b/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java index 9306301d13e3..cbc0a91fb5c6 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java @@ -23,14 +23,22 @@ import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.Druids; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; import org.joda.time.Interval; +import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; public abstract class SearchStrategy { @@ -49,10 +57,36 @@ protected SearchStrategy(SearchQuery query) public abstract List getExecutionPlan(SearchQuery query, Segment segment); - static List getDimsToSearch(Indexed availableDimensions, List dimensions) + static List getDimsToSearch(Segment segment, List dimensions) { if (dimensions == null || dimensions.isEmpty()) { - return ImmutableList.copyOf(Iterables.transform(availableDimensions, Druids.DIMENSION_IDENTITY)); + final Set dims = new LinkedHashSet<>(); + final QueryableIndex index = segment.as(QueryableIndex.class); + if (index != null) { + for (String dim : index.getAvailableDimensions()) { + dims.add(dim); + } + } else { + // fallback to RowSignature and Metadata if QueryableIndex not available + final PhysicalSegmentInspector segmentInspector = segment.as(PhysicalSegmentInspector.class); + final Metadata metadata = segmentInspector != null ? segmentInspector.getMetadata() : null; + final Set ignore = new HashSet<>(); + ignore.add(ColumnHolder.TIME_COLUMN_NAME); + if (metadata != null && metadata.getAggregators() != null) { + for (AggregatorFactory factory : metadata.getAggregators()) { + ignore.add(factory.getName()); + } + } + final RowSignature rowSignature = segment.asCursorFactory().getRowSignature(); + for (String columnName : rowSignature.getColumnNames()) { + if (!ignore.contains(columnName)) { + dims.add(columnName); + } + } + } + return ImmutableList.copyOf( + Iterables.transform(dims, Druids.DIMENSION_IDENTITY) + ); } else { return dimensions; } diff --git a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java index d5af263a1c7a..bd33d7f97839 100644 --- a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java @@ -35,11 +35,11 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.search.CursorOnlyStrategy.CursorBasedExecutor; import org.apache.druid.segment.ColumnSelectorColumnIndexSelector; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -74,14 +74,15 @@ private UseIndexesStrategy( public List getExecutionPlan(SearchQuery query, Segment segment) { final ImmutableList.Builder builder = ImmutableList.builder(); - final QueryableIndex index = segment.asQueryableIndex(); - final StorageAdapter adapter = segment.asStorageAdapter(); - final List searchDims = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions()); + final QueryableIndex index = segment.as(QueryableIndex.class); + final CursorFactory cursorFactory = segment.asCursorFactory(); + final List searchDims = getDimsToSearch(segment, query.getDimensions()); if (index != null) { // pair of bitmap dims and non-bitmap dims final Pair, List> pair = partitionDimensionList( - adapter, + segment, + cursorFactory, query.getVirtualColumns(), searchDims ); @@ -131,7 +132,8 @@ public List getExecutionPlan(SearchQuery query, Segment seg * Note that the returned lists are free to modify. */ private static Pair, List> partitionDimensionList( - StorageAdapter adapter, + Segment segment, + CursorFactory cursorFactory, VirtualColumns virtualColumns, List dimensions ) @@ -139,10 +141,10 @@ private static Pair, List> partitionDimension final List bitmapDims = new ArrayList<>(); final List nonBitmapDims = new ArrayList<>(); final List dimsToSearch = getDimsToSearch( - adapter.getAvailableDimensions(), + segment, dimensions ); - VirtualizedColumnInspector columnInspector = new VirtualizedColumnInspector(adapter, virtualColumns); + VirtualizedColumnInspector columnInspector = new VirtualizedColumnInspector(cursorFactory, virtualColumns); for (DimensionSpec spec : dimsToSearch) { ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(spec.getDimension()); @@ -263,7 +265,7 @@ public IndexOnlyExecutor( @Override public Object2IntRBTreeMap execute(int limit) { - final QueryableIndex index = segment.asQueryableIndex(); + final QueryableIndex index = segment.as(QueryableIndex.class); Preconditions.checkArgument(index != null, "Index should not be null"); ColumnSelectorColumnIndexSelector indexSelector = new ColumnSelectorColumnIndexSelector( 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 ec9e27c6f38e..f90ff67615e6 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -40,11 +40,10 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.Filters; @@ -95,13 +94,15 @@ public QueryToolChest, TimeBoundaryQuery> getToo private static class TimeBoundaryQueryRunner implements QueryRunner> { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; + private final Interval dataInterval; @Nullable private final TimeBoundaryInspector timeBoundaryInspector; public TimeBoundaryQueryRunner(Segment segment) { - this.adapter = segment.asStorageAdapter(); + this.cursorFactory = segment.asCursorFactory(); + this.dataInterval = segment.getDataInterval(); this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); } @@ -124,9 +125,9 @@ public Sequence> run( @Override public Iterator> make() { - if (adapter == null) { + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } @@ -142,13 +143,13 @@ public Iterator> make() maxTime = timeBoundaryInspector.getMaxTime(); } } else { - final Pair timeBoundary = getTimeBoundary(query, adapter); + final Pair timeBoundary = getTimeBoundary(query, cursorFactory); minTime = timeBoundary.left(); maxTime = timeBoundary.right(); } return query.buildResult( - adapter.getInterval().getStart(), + dataInterval.getStart(), minTime, maxTime ).iterator(); @@ -168,7 +169,7 @@ public void cleanup(Iterator> toClean) * Whether a particular {@link TimeBoundaryQuery} can use {@link TimeBoundaryInspector#getMinTime()} and/or * {@link TimeBoundaryInspector#getMaxTime()}. * - * If false, must use {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)}. + * If false, must use {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}. */ private static boolean canUseTimeBoundaryInspector( final TimeBoundaryQuery query, @@ -211,7 +212,7 @@ public static CursorBuildSpec makeCursorBuildSpec(TimeBoundaryQuery query) private static Pair getTimeBoundary( final TimeBoundaryQuery query, - final CursorHolderFactory cursorHolderFactory + final CursorFactory cursorFactory ) { DateTime minTime = null, maxTime = null; @@ -222,7 +223,7 @@ private static Pair getTimeBoundary( .setPreferredOrdering(Cursors.ascendingTimeOrder()) .build(); - try (final CursorHolder cursorHolder = cursorHolderFactory.makeCursorHolder(cursorSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(cursorSpec)) { if (cursorHolder.getTimeOrder() == Order.ASCENDING) { // Time-ordered cursor, use the first timestamp. minTime = getFirstTimestamp(query, cursorHolder); @@ -239,7 +240,7 @@ private static Pair getTimeBoundary( .setPreferredOrdering(Cursors.descendingTimeOrder()) .build(); - try (final CursorHolder cursorHolder = cursorHolderFactory.makeCursorHolder(cursorSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(cursorSpec)) { if (cursorHolder.getTimeOrder() == Order.DESCENDING) { // Time-ordered cursor, use the first timestamp. maxTime = getFirstTimestamp(query, cursorHolder); 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 95cba2e647f8..dd5a8cb2b58c 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 @@ -42,10 +42,10 @@ 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.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.SegmentMissingException; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -87,21 +87,21 @@ public TimeseriesQueryEngine( */ public Sequence> process( final TimeseriesQuery query, - final StorageAdapter adapter, + final CursorFactory cursorFactory, @Nullable TimeBoundaryInspector timeBoundaryInspector, @Nullable final TimeseriesQueryMetrics timeseriesQueryMetrics ) { - if (adapter == null) { + if (cursorFactory == null) { throw new SegmentMissingException( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } final Interval interval = Iterables.getOnlyElement(query.getIntervals()); final Granularity gran = query.getGranularity(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); try { final Sequence> result; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 788e72015e29..2839f3109626 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -32,8 +32,8 @@ import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import javax.annotation.Nullable; @@ -62,7 +62,7 @@ public TimeseriesQueryRunnerFactory( @Override public QueryRunner> createRunner(final Segment segment) { - return new TimeseriesQueryRunner(engine, segment.asStorageAdapter(), segment.as(TimeBoundaryInspector.class)); + return new TimeseriesQueryRunner(engine, segment.asCursorFactory(), segment.as(TimeBoundaryInspector.class)); } @Override @@ -83,18 +83,18 @@ public QueryToolChest, TimeseriesQuery> getToolche private static class TimeseriesQueryRunner implements QueryRunner> { private final TimeseriesQueryEngine engine; - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; @Nullable private final TimeBoundaryInspector timeBoundaryInspector; private TimeseriesQueryRunner( TimeseriesQueryEngine engine, - StorageAdapter adapter, + CursorFactory cursorFactory, @Nullable TimeBoundaryInspector timeBoundaryInspector ) { this.engine = engine; - this.adapter = adapter; + this.cursorFactory = cursorFactory; this.timeBoundaryInspector = timeBoundaryInspector; } @@ -111,11 +111,10 @@ public Sequence> run( return engine.process( (TimeseriesQuery) input, - adapter, + cursorFactory, timeBoundaryInspector, (TimeseriesQueryMetrics) queryPlus.getQueryMetrics() ); } } - } 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 99759f515dd7..a4605c3f2652 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 @@ -28,7 +28,6 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -47,18 +46,18 @@ */ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm { - private final StorageAdapter storageAdapter; private final TopNQuery query; + private final TopNCursorInspector cursorInspector; private final NonBlockingPool bufferPool; public AggregateTopNMetricFirstAlgorithm( - StorageAdapter storageAdapter, TopNQuery query, + TopNCursorInspector cursorInspector, NonBlockingPool bufferPool ) { - this.storageAdapter = storageAdapter; this.query = query; + this.cursorInspector = cursorInspector; this.bufferPool = bufferPool; } @@ -90,7 +89,7 @@ public void run( .build(); final TopNResultBuilder singleMetricResultBuilder = BaseTopNAlgorithm.makeResultBuilder(params, singleMetricQuery); - PooledTopNAlgorithm singleMetricAlgo = new PooledTopNAlgorithm(storageAdapter, singleMetricQuery, bufferPool); + PooledTopNAlgorithm singleMetricAlgo = new PooledTopNAlgorithm(singleMetricQuery, cursorInspector, bufferPool); PooledTopNAlgorithm.PooledTopNParams singleMetricParam = null; int[] dimValSelector; try { @@ -109,7 +108,7 @@ public void run( singleMetricAlgo.cleanup(singleMetricParam); } - PooledTopNAlgorithm allMetricAlgo = new PooledTopNAlgorithm(storageAdapter, query, bufferPool); + PooledTopNAlgorithm allMetricAlgo = new PooledTopNAlgorithm(query, cursorInspector, bufferPool); PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null; try { // reset cursor since we call run again 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 c8bcbc1a0944..4c0bb066eecb 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 @@ -28,7 +28,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TopNOptimizationInspector; import org.apache.druid.segment.column.ColumnCapabilities; import javax.annotation.Nullable; @@ -63,11 +63,11 @@ protected static BufferAggregator[] makeBufferAggregators(Cursor cursor, List implements TopNMetricSpecBu private final IdLookup idLookup; private final TopNQuery query; - private final StorageAdapter storageAdapter; + private final TopNCursorInspector cursorInspector; public BaseArrayProvider( DimensionSelector dimSelector, TopNQuery query, - StorageAdapter storageAdapter + TopNCursorInspector cursorInspector ) { this.idLookup = dimSelector.idLookup(); this.query = query; - this.storageAdapter = storageAdapter; + this.cursorInspector = cursorInspector; previousStop = null; ignoreAfterThreshold = false; @@ -269,7 +269,8 @@ public BaseArrayProvider( @Override public void skipTo(String previousStop) { - ColumnCapabilities capabilities = storageAdapter.getColumnCapabilities(query.getDimensionSpec().getDimension()); + ColumnCapabilities capabilities = cursorInspector.getColumnInspector() + .getColumnCapabilities(query.getDimensionSpec().getDimension()); if (capabilities != null && capabilities.areDictionaryValuesSorted().isTrue()) { this.previousStop = previousStop; } @@ -315,10 +316,12 @@ public Pair computeStartEnd(int cardinality) int endIndex = Math.min(ignoreFirstN + keepOnlyN, cardinality); + final TopNOptimizationInspector topNOptimizationInspector = cursorInspector.getOptimizationInspector(); if (ignoreAfterThreshold && query.getDimensionsFilter() == null && - !storageAdapter.hasBuiltInFilters() && - query.getIntervals().stream().anyMatch(interval -> interval.contains(storageAdapter.getInterval()))) { + topNOptimizationInspector != null && + topNOptimizationInspector.areAllDictionaryIdsPresent() && + query.getIntervals().stream().anyMatch(interval -> interval.contains(cursorInspector.getDataInterval()))) { endIndex = Math.min(endIndex, startIndex + query.getThreshold()); } 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 6f8ed23bc29d..c8986c0f3427 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 @@ -24,7 +24,6 @@ import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; /** * Heap based topn algorithm that handles aggregates on dimension extractions and numeric typed dimension columns. @@ -38,11 +37,11 @@ public class HeapBasedTopNAlgorithm private final TopNQuery query; public HeapBasedTopNAlgorithm( - StorageAdapter storageAdapter, - TopNQuery query + TopNQuery query, + TopNCursorInspector cursorInspector ) { - super(storageAdapter); + super(cursorInspector); this.query = query; } @@ -68,7 +67,7 @@ protected Aggregator[][] makeDimValSelector(TopNParams params, int numProcessed, throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); } ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); - return selectorPlus.getColumnSelectorStrategy().getRowSelector(query, params, storageAdapter); + return selectorPlus.getColumnSelectorStrategy().getRowSelector(query, params, cursorInspector); } @Override 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 d9213b94e09b..d2ba16746218 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 @@ -35,7 +35,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.FilteredOffset; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.Offset; @@ -212,12 +211,12 @@ private static void computeSpecializedScanAndAggregateImplementations() private static final int AGG_UNROLL_COUNT = 8; // Must be able to fit loop below public PooledTopNAlgorithm( - StorageAdapter storageAdapter, TopNQuery query, + TopNCursorInspector cursorInspector, NonBlockingPool bufferPool ) { - super(storageAdapter); + super(cursorInspector); this.query = query; this.bufferPool = bufferPool; } @@ -232,11 +231,7 @@ public PooledTopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor c throw new UnsupportedOperationException("Cannot operate on a dimension with no dictionary"); } - final TopNMetricSpecBuilder arrayProvider = new BaseArrayProvider( - dimSelector, - query, - storageAdapter - ) + final TopNMetricSpecBuilder arrayProvider = new BaseArrayProvider(dimSelector, query, cursorInspector) { private final int[] positions = new int[cardinality]; 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 7e5d43b96ade..0c79e7c8d31b 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 @@ -25,7 +25,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import java.util.HashMap; @@ -39,9 +38,9 @@ public class TimeExtractionTopNAlgorithm extends BaseTopNAlgorithm dimensionValueConverter; - public TimeExtractionTopNAlgorithm(StorageAdapter storageAdapter, TopNQuery query) + public TimeExtractionTopNAlgorithm(TopNQuery query, TopNCursorInspector cursorInspector) { - super(storageAdapter); + super(cursorInspector); this.query = query; // This strategy is used for ExtractionFns on the __time column. They always return STRING, so we need to convert diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.java b/processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.java new file mode 100644 index 000000000000..c5d5d117a53b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.java @@ -0,0 +1,70 @@ +/* + * 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.topn; + +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.TopNOptimizationInspector; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +public class TopNCursorInspector +{ + private final ColumnInspector columnInspector; + @Nullable + private final TopNOptimizationInspector topNOptimizationInspector; + private final Interval dataInterval; + + private final int dimensionCardinality; + + public TopNCursorInspector( + ColumnInspector columnInspector, + @Nullable TopNOptimizationInspector topNOptimizationInspector, + Interval dataInterval, + int dimensionCardinality + ) + { + this.columnInspector = columnInspector; + this.topNOptimizationInspector = topNOptimizationInspector; + this.dataInterval = dataInterval; + this.dimensionCardinality = dimensionCardinality; + } + + public ColumnInspector getColumnInspector() + { + return columnInspector; + } + + @Nullable + public TopNOptimizationInspector getOptimizationInspector() + { + return topNOptimizationInspector; + } + + public Interval getDataInterval() + { + return dataInterval; + } + + public int getDimensionCardinality() + { + return dimensionCardinality; + } +} 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 dafff313afb1..5a48d4a0d283 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 @@ -23,9 +23,7 @@ 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; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.DimensionHandlerUtils; import javax.annotation.Nullable; @@ -45,19 +43,13 @@ public TopNMapFn( @SuppressWarnings("unchecked") @Nullable - public Result apply(final Cursor cursor, final CursorGranularizer granularizer, final @Nullable TopNQueryMetrics queryMetrics) + public Result apply( + final Cursor cursor, + ColumnSelectorPlus> selectorPlus, + final CursorGranularizer granularizer, + final @Nullable TopNQueryMetrics queryMetrics + ) { - final ColumnSelectorPlus> selectorPlus = - DimensionHandlerUtils.createColumnSelectorPlus( - new TopNColumnAggregatesProcessorFactory(query.getDimensionSpec().getOutputType()), - query.getDimensionSpec(), - cursor.getColumnSelectorFactory() - ); - - if (selectorPlus.getSelector() == null) { - return null; - } - TopNParams params = null; try { params = topNAlgorithm.makeInitParams(selectorPlus, cursor, granularizer); 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 c6561faeee3b..442e04552f17 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 @@ -25,17 +25,25 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; +import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; +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.CursorHolder; +import org.apache.druid.segment.DimensionDictionarySelector; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentMissingException; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; +import org.apache.druid.segment.TopNOptimizationInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.Types; @@ -59,32 +67,55 @@ public TopNQueryEngine(NonBlockingPool bufferPool) } /** - * Do the thing - process a {@link StorageAdapter} into a {@link Sequence} of {@link TopNResultValue}, with one of the + * Do the thing - process a {@link Segment} into a {@link Sequence} of {@link TopNResultValue}, with one of the * fine {@link TopNAlgorithm} available chosen based on the type of column being aggregated. The algorithm provides a * mapping function to process rows from the adapter {@link Cursor} to apply {@link AggregatorFactory} and create or * update {@link TopNResultValue} */ public Sequence> query( final TopNQuery query, - final StorageAdapter adapter, - @Nullable final TimeBoundaryInspector timeBoundaryInspector, + final Segment segment, @Nullable final TopNQueryMetrics queryMetrics ) { - if (adapter == null) { + final CursorFactory cursorFactory = segment.asCursorFactory(); + if (cursorFactory == null) { throw new SegmentMissingException( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } - final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); - final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); } + + final TimeBoundaryInspector timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); + + final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + + final ColumnSelectorPlus> selectorPlus = + DimensionHandlerUtils.createColumnSelectorPlus( + new TopNColumnAggregatesProcessorFactory(query.getDimensionSpec().getOutputType()), + query.getDimensionSpec(), + factory + ); + + final int cardinality; + if (selectorPlus.getSelector() instanceof DimensionDictionarySelector) { + cardinality = ((DimensionDictionarySelector) selectorPlus.getSelector()).getValueCardinality(); + } else { + cardinality = DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } + final TopNCursorInspector cursorInspector = new TopNCursorInspector( + factory, + segment.as(TopNOptimizationInspector.class), + segment.getDataInterval(), + cardinality + ); + final CursorGranularizer granularizer = CursorGranularizer.create( cursor, timeBoundaryInspector, @@ -92,18 +123,20 @@ public Sequence> query( query.getGranularity(), buildSpec.getInterval() ); - if (granularizer == null) { + if (granularizer == null || selectorPlus.getSelector() == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); } + if (queryMetrics != null) { queryMetrics.cursor(cursor); } + final TopNMapFn mapFn = getMapFn(query, cursorInspector, queryMetrics); return Sequences.filter( Sequences.simple(granularizer.getBucketIterable()) .map(bucketInterval -> { granularizer.advanceToBucket(bucketInterval); - return mapFn.apply(cursor, granularizer, queryMetrics); + return mapFn.apply(cursor, selectorPlus, granularizer, queryMetrics); }), Predicates.notNull() ).withBaggage(cursorHolder); @@ -114,14 +147,15 @@ public Sequence> query( */ private TopNMapFn getMapFn( final TopNQuery query, - final StorageAdapter adapter, + final TopNCursorInspector cursorInspector, final @Nullable TopNQueryMetrics queryMetrics ) { final String dimension = query.getDimensionSpec().getDimension(); - final int cardinality = adapter.getDimensionCardinality(dimension); + + if (queryMetrics != null) { - queryMetrics.dimensionCardinality(cardinality); + queryMetrics.dimensionCardinality(cursorInspector.getDimensionCardinality()); } int numBytesPerRecord = 0; @@ -129,27 +163,43 @@ private TopNMapFn getMapFn( numBytesPerRecord += aggregatorFactory.getMaxIntermediateSizeWithNulls(); } - final TopNAlgorithmSelector selector = new TopNAlgorithmSelector(cardinality, numBytesPerRecord); + final TopNAlgorithmSelector selector = new TopNAlgorithmSelector( + cursorInspector.getDimensionCardinality(), + numBytesPerRecord + ); query.initTopNAlgorithmSelector(selector); - final ColumnCapabilities columnCapabilities = query.getVirtualColumns() - .getColumnCapabilitiesWithFallback(adapter, dimension); - + final ColumnCapabilities columnCapabilities = query.getVirtualColumns().getColumnCapabilitiesWithFallback( + cursorInspector.getColumnInspector(), + dimension + ); final TopNAlgorithm topNAlgorithm; - if (canUsePooledAlgorithm(selector, query, columnCapabilities, bufferPool, cardinality, numBytesPerRecord)) { + if (canUsePooledAlgorithm(selector, query, columnCapabilities, bufferPool, cursorInspector.getDimensionCardinality(), numBytesPerRecord)) { // pool based algorithm selection, if we can if (selector.isAggregateAllMetrics()) { // if sorted by dimension we should aggregate all metrics in a single pass, use the regular pooled algorithm for // this - topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); + topNAlgorithm = new PooledTopNAlgorithm( + query, + cursorInspector, + bufferPool + ); } else if (shouldUseAggregateMetricFirstAlgorithm(query, selector)) { // for high cardinality dimensions with larger result sets we aggregate with only the ordering aggregation to // compute the first 'n' values, and then for the rest of the metrics but for only the 'n' values - topNAlgorithm = new AggregateTopNMetricFirstAlgorithm(adapter, query, bufferPool); + topNAlgorithm = new AggregateTopNMetricFirstAlgorithm( + query, + cursorInspector, + bufferPool + ); } else { // anything else, use the regular pooled algorithm - topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); + topNAlgorithm = new PooledTopNAlgorithm( + query, + cursorInspector, + bufferPool + ); } } else { // heap based algorithm selection, if we must @@ -161,9 +211,9 @@ private TopNMapFn getMapFn( // A special TimeExtractionTopNAlgorithm is required since HeapBasedTopNAlgorithm // currently relies on the dimension cardinality to support lexicographic sorting - topNAlgorithm = new TimeExtractionTopNAlgorithm(adapter, query); + topNAlgorithm = new TimeExtractionTopNAlgorithm(query, cursorInspector); } else { - topNAlgorithm = new HeapBasedTopNAlgorithm(adapter, query); + topNAlgorithm = new HeapBasedTopNAlgorithm(query, cursorInspector); } } if (queryMetrics != null) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java index a81d9258c866..0229bdbcc592 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java @@ -34,7 +34,6 @@ import org.apache.druid.query.Result; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.TimeBoundaryInspector; import java.nio.ByteBuffer; @@ -78,8 +77,7 @@ public Sequence> run( TopNQuery query = (TopNQuery) input.getQuery(); return queryEngine.query( query, - segment.asStorageAdapter(), - segment.as(TimeBoundaryInspector.class), + segment, (TopNQueryMetrics) input.getQueryMetrics() ); } 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 4b56a833f8a5..565ad036cea0 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 @@ -23,12 +23,12 @@ 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.TopNCursorInspector; import org.apache.druid.query.topn.TopNParams; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNResultBuilder; import org.apache.druid.segment.BaseNullableColumnValueSelector; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; import java.util.Map; import java.util.function.Function; @@ -79,7 +79,7 @@ public int getCardinality(Selector selector) } @Override - public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, StorageAdapter storageAdapter) + public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, TopNCursorInspector cursorInspector) { return null; } 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 d6f716f2c2b3..9eca369fdc7e 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 @@ -22,6 +22,7 @@ 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.TopNCursorInspector; import org.apache.druid.query.topn.TopNParams; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNResultBuilder; @@ -29,7 +30,6 @@ import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; @@ -61,7 +61,7 @@ public int getCardinality(DimensionSelector selector) } @Override - public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, StorageAdapter storageAdapter) + public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, TopNCursorInspector cursorInspector) { if (params.getCardinality() < 0) { throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); @@ -74,8 +74,8 @@ public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, Storage final BaseTopNAlgorithm.AggregatorArrayProvider provider = new BaseTopNAlgorithm.AggregatorArrayProvider( (DimensionSelector) params.getSelectorPlus().getSelector(), query, - params.getCardinality(), - storageAdapter + cursorInspector, + params.getCardinality() ); return provider.build(); 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 5fd141e64289..e397300ae94b 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 @@ -23,12 +23,12 @@ import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.dimension.ColumnSelectorStrategy; import org.apache.druid.query.topn.HeapBasedTopNAlgorithm; +import org.apache.druid.query.topn.TopNCursorInspector; import org.apache.druid.query.topn.TopNParams; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNResultBuilder; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; import javax.annotation.Nullable; @@ -58,15 +58,15 @@ public interface TopNColumnAggregatesProcessor extends Column * * A dimension type that does not have integer values should return null. * - * @param query The TopN query being served - * @param params Parameters for the TopN query being served - * @param storageAdapter Column storage adapter, to provide information about the column that can be used for - * query optimization, e.g. whether dimension values are sorted or not + * @param query The TopN query being served + * @param params Parameters for the TopN query being served + * @param cursorInspector provide information about the cursor that can be used for query optimization, e.g. whether + * dimension values are sorted or not * * @return an Aggregator[][] for integer-valued dimensions, null otherwise */ @Nullable - Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, StorageAdapter storageAdapter); + Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, TopNCursorInspector cursorInspector); /** * Used by {@link HeapBasedTopNAlgorithm}. The contract of this method requires calling {@link #initAggregateStore()} diff --git a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java index 4fa8d3b45b3b..194914035226 100644 --- a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java @@ -19,39 +19,34 @@ package org.apache.druid.segment; -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.rowsandcols.ArrayListRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; -import org.joda.time.Interval; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; /** * A {@link Segment} that is based on a stream of objects. */ -public class ArrayListSegment implements Segment +public class ArrayListSegment extends RowBasedSegment { - private final SegmentId segmentId; private final ArrayList rows; - private final RowAdapter rowAdapter; + private final RowAdapter adapter; private final RowSignature rowSignature; /** * Create a list-based segment. *

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

* The provided "rowSignature" will be used for reporting available columns and their capabilities to users of - * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's - * storage adapter will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that + * {@link #asCursorFactory()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's + * cursor factory will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that * field, even if it doesn't appear in "rowSignature". * * @param segmentId segment identifier; will be returned by {@link #getId()} @@ -67,40 +62,12 @@ public ArrayListSegment( final RowSignature rowSignature ) { - this.segmentId = Preconditions.checkNotNull(segmentId, "segmentId"); + super(segmentId, Sequences.simple(rows), rowAdapter, rowSignature); this.rows = rows; - this.rowAdapter = rowAdapter; + this.adapter = rowAdapter; this.rowSignature = rowSignature; } - @Override - @Nonnull - public SegmentId getId() - { - return segmentId; - } - - @Override - @Nonnull - public Interval getDataInterval() - { - return Intervals.ETERNITY; - } - - @Nullable - @Override - public QueryableIndex asQueryableIndex() - { - return null; - } - - @Override - @Nonnull - public StorageAdapter asStorageAdapter() - { - return new RowBasedStorageAdapter<>(Sequences.simple(rows), rowAdapter, rowSignature); - } - @Nullable @Override @SuppressWarnings("unchecked") @@ -109,7 +76,7 @@ public T as(Class clazz) if (CloseableShapeshifter.class.equals(clazz)) { return (T) new MyCloseableShapeshifter(); } - return Segment.super.as(clazz); + return super.as(clazz); } @Override @@ -120,7 +87,7 @@ public void close() private RowsAndColumns asRowsAndColumns() { - return new ArrayListRowsAndColumns<>(rows, rowAdapter, rowSignature); + return new ArrayListRowsAndColumns<>(rows, adapter, rowSignature); } private class MyCloseableShapeshifter implements CloseableShapeshifter 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 a1fc14c10540..14a53eb41b7b 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -24,7 +24,7 @@ /** * Cursor is an interface for iteration over a range of data points, used during query execution. Cursors are available - * from {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. + * from {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. *

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

diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
index 551430a00e33..b2547ebe38c0 100644
--- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
@@ -19,100 +19,36 @@
 
 package org.apache.druid.segment;
 
-import org.apache.druid.error.DruidException;
-import org.apache.druid.java.util.common.granularity.Granularity;
-import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.query.QueryMetrics;
-import org.apache.druid.query.filter.Filter;
-import org.apache.druid.segment.vector.VectorCursor;
-import org.joda.time.Interval;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.RowSignature;
 
 import javax.annotation.Nullable;
 
-/**
- * Interface extended by {@link StorageAdapter}, which gives them the power to create cursors.
- *
- * @see StorageAdapter
- *
- * @deprecated This interface is deprecated and no longer implemented by any built-in {@link StorageAdapter}. Callers
- * should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. Implementors should implement
- * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead.
- */
-@Deprecated
-public interface CursorFactory
+public interface CursorFactory extends ColumnInspector
 {
-  /**
-   * Returns true if the provided combination of parameters can be handled by "makeVectorCursor".
-   *
-   * Query engines should use this before running in vectorized mode, and be prepared to fall back to non-vectorized
-   * mode if this method returns false.
-   *
-   * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call
-   * {@link CursorHolder#canVectorize()}.
-   * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead.  This method is
-   * no longer implemented by any built-in factories.
-   */
-  @Deprecated
-  default boolean canVectorize(
-      @Nullable Filter filter,
-      VirtualColumns virtualColumns,
-      boolean descending
-  )
-  {
-    throw DruidException.defensive(
-        "CursorFactory.canVectorize is no longer supported, use CursorHolderFactory.makeCursorHolder instead"
-    );
-  }
+  CursorHolder makeCursorHolder(CursorBuildSpec spec);
 
   /**
-   * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity).
-   *
-   * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call
-   * {@link CursorHolder#asCursor()}.
-   * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead.
-   * This method is no longer implemented by any built-in factories.
+   * Returns the {@link RowSignature} of the data available from this cursor factory. For mutable segments, even though
+   * the signature may evolve over time, any particular object returned by this method is an immutable snapshot.
    */
-  @Deprecated
-  default Sequence makeCursors(
-      @Nullable Filter filter,
-      Interval interval,
-      VirtualColumns virtualColumns,
-      Granularity gran,
-      boolean descending,
-      @Nullable QueryMetrics queryMetrics
-  )
-  {
-    throw DruidException.defensive(
-        "CursorFactory.makeCursors is no longer supported, use CursorHolderFactory.makeCursorHolder instead"
-    );
-  }
+  RowSignature getRowSignature();
 
   /**
-   * Creates a VectorCursor. Unlike the Cursor returned by "makeCursor", there is just one of these. Hence, this method
-   * does not take a "granularity" parameter. Before calling this method, check "canVectorize" to see if the call you
-   * are about to make will throw an error or not.
+   * Returns capabilities of a particular column, if known. May be null if the column doesn't exist, or if
+   * the column does exist but the capabilities are unknown. The latter is possible with dynamically discovered
+   * columns.
+   *
+   * Note that CursorFactory are representations of "real" segments, so they are not aware of any virtual columns
+   * that may be involved in a query. In general, query engines should instead use the method
+   * {@link ColumnSelectorFactory#getColumnCapabilities(String)}, which returns capabilities for virtual columns as
+   * well.
    *
-   * Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval
-   * of this segment).
+   * @param column column name
    *
-   * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call
-   * {@link CursorHolder#asVectorCursor()}. Implementors should implement
-   * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is no longer implemented by any
-   * built-in factories.
+   * @return capabilities, or null
    */
-  @Deprecated
+  @Override
   @Nullable
-  default VectorCursor makeVectorCursor(
-      @Nullable Filter filter,
-      Interval interval,
-      VirtualColumns virtualColumns,
-      boolean descending,
-      int vectorSize,
-      @Nullable QueryMetrics queryMetrics
-  )
-  {
-    throw DruidException.defensive(
-        "CursorFactory.makeVectorCursor is no longer supported, use CursorHolderFactory.makeCursorHolder instead"
-    );
-  }
+  ColumnCapabilities getColumnCapabilities(String column);
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredCursorFactory.java
similarity index 51%
rename from processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java
rename to processing/src/main/java/org/apache/druid/segment/FilteredCursorFactory.java
index 46b9f5fe061f..b1806f66342d 100644
--- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/FilteredCursorFactory.java
@@ -24,22 +24,20 @@
 import org.apache.druid.query.filter.Filter;
 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.filter.AndFilter;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 
-public class FilteredStorageAdapter implements StorageAdapter
+public class FilteredCursorFactory implements CursorFactory
 {
+  private final CursorFactory delegate;
   @Nullable
-  private final DimFilter filterOnDataSource;
-  private final StorageAdapter baseStorageAdapter;
+  private final DimFilter filter;
 
-  public FilteredStorageAdapter(final StorageAdapter adapter, @Nullable final DimFilter filter)
+  public FilteredCursorFactory(CursorFactory delegate, @Nullable DimFilter filter)
   {
-    this.baseStorageAdapter = adapter;
-    this.filterOnDataSource = filter;
+    this.delegate = delegate;
+    this.filter = filter;
   }
 
   @Override
@@ -48,89 +46,32 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
     final CursorBuildSpec.CursorBuildSpecBuilder buildSpecBuilder = CursorBuildSpec.builder(spec);
     final Filter newFilter;
     if (spec.getFilter() == null) {
-      if (filterOnDataSource != null) {
-        newFilter = filterOnDataSource.toFilter();
+      if (filter != null) {
+        newFilter = filter.toFilter();
       } else {
         newFilter = null;
       }
     } else {
-      if (filterOnDataSource != null) {
-        newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter()));
+      if (filter != null) {
+        newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filter.toFilter()));
       } else {
         newFilter = spec.getFilter();
       }
     }
     buildSpecBuilder.setFilter(newFilter);
-    return baseStorageAdapter.makeCursorHolder(buildSpecBuilder.build());
-  }
-
-  @Override
-  public Interval getInterval()
-  {
-    return baseStorageAdapter.getInterval();
+    return delegate.makeCursorHolder(buildSpecBuilder.build());
   }
 
   @Override
   public RowSignature getRowSignature()
   {
-    return baseStorageAdapter.getRowSignature();
-  }
-
-  @Override
-  public Indexed getAvailableDimensions()
-  {
-    return baseStorageAdapter.getAvailableDimensions();
-  }
-
-  @Override
-  public Iterable getAvailableMetrics()
-  {
-    return baseStorageAdapter.getAvailableMetrics();
-  }
-
-  @Override
-  public int getDimensionCardinality(String column)
-  {
-    return baseStorageAdapter.getDimensionCardinality(column);
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMinValue(String column)
-  {
-    return null;
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMaxValue(String column)
-  {
-    return null;
+    return delegate.getRowSignature();
   }
 
   @Nullable
   @Override
   public ColumnCapabilities getColumnCapabilities(String column)
   {
-    return baseStorageAdapter.getColumnCapabilities(column);
-  }
-
-  @Override
-  public int getNumRows()
-  {
-    return 0;
-  }
-
-  @Nullable
-  @Override
-  public Metadata getMetadata()
-  {
-    return baseStorageAdapter.getMetadata();
-  }
-
-  @Override
-  public boolean isFromTombstone()
-  {
-    return baseStorageAdapter.isFromTombstone();
+    return delegate.getColumnCapabilities(column);
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java b/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java
new file mode 100644
index 000000000000..145f41d19e77
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment;
+
+import org.apache.druid.query.filter.DimFilter;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+public class FilteredSegment extends WrappedSegmentReference
+{
+  @Nullable
+  private final DimFilter filter;
+
+  public FilteredSegment(
+      SegmentReference delegate,
+      @Nullable DimFilter filter
+  )
+  {
+    super(delegate);
+    this.filter = filter;
+  }
+
+  @Override
+  public CursorFactory asCursorFactory()
+  {
+    return new FilteredCursorFactory(delegate.asCursorFactory(), filter);
+  }
+
+  @Nullable
+  @Override
+  public  T as(@Nonnull Class clazz)
+  {
+    if (TopNOptimizationInspector.class.equals(clazz)) {
+      return (T) new SimpleTopNOptimizationInspector(filter == null);
+    }
+    return super.as(clazz);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
index 62ecda675ac4..735f6d1e1594 100644
--- a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
@@ -20,7 +20,8 @@
 package org.apache.druid.segment;
 
 import org.apache.druid.segment.incremental.IncrementalIndex;
-import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
+import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory;
+import org.apache.druid.segment.incremental.IncrementalIndexPhysicalSegmentInspector;
 import org.apache.druid.timeline.SegmentId;
 import org.joda.time.Interval;
 
@@ -58,9 +59,9 @@ public QueryableIndex asQueryableIndex()
   }
 
   @Override
-  public StorageAdapter asStorageAdapter()
+  public CursorFactory asCursorFactory()
   {
-    return new IncrementalIndexStorageAdapter(index);
+    return new IncrementalIndexCursorFactory(index);
   }
 
   @Nullable
@@ -71,6 +72,12 @@ public  T as(final Class clazz)
       return (T) new IncrementalIndexTimeBoundaryInspector(index);
     } else if (MaxIngestedEventTimeInspector.class.equals(clazz)) {
       return (T) new IncrementalIndexMaxIngestedEventTimeInspector(index);
+    } else if (Metadata.class.equals(clazz)) {
+      return (T) index.getMetadata();
+    } else if (PhysicalSegmentInspector.class.equals(clazz)) {
+      return (T) new IncrementalIndexPhysicalSegmentInspector(index);
+    } else if (TopNOptimizationInspector.class.equals(clazz)) {
+      return (T) new SimpleTopNOptimizationInspector(true);
     } else {
       return Segment.super.as(clazz);
     }
diff --git a/processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.java b/processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.java
new file mode 100644
index 000000000000..6e72e412463a
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment;
+
+import javax.annotation.Nullable;
+
+/**
+ * Interface for methods describing physical segments such as {@link QueryableIndexSegment} and
+ * {@link IncrementalIndexSegment} that is not typically used at query time (outside of metadata queries).
+ */
+public interface PhysicalSegmentInspector extends ColumnInspector
+{
+  /**
+   * Returns {@link Metadata} which contains details about how the segment was created
+   */
+  @Nullable
+  Metadata getMetadata();
+  /**
+   * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null
+   * if not known. Does not scan the column to find the minimum value.
+   */
+  @Nullable
+  Comparable getMinValue(String column);
+
+  /**
+   * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null
+   * if not known. Does not scan the column to find the maximum value.
+   */
+  @Nullable
+  Comparable getMaxValue(String column);
+
+  /**
+   * Returns the number of distinct values in a column, if known, or
+   * {@link DimensionDictionarySelector#CARDINALITY_UNKNOWN} if not.}
+   */
+  int getDimensionCardinality(String column);
+
+  /**
+   * Returns the number of rows in the segment
+   */
+  int getNumRows();
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java
index 3766db4a4b19..309e36d24b87 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java
@@ -34,9 +34,9 @@
 
 /**
  * Direct interface to memory mapped segments. Not a public API for extensions; site specific queries should be
- * using {@link StorageAdapter}.
+ * using {@link CursorFactory}.
  *
- * @see QueryableIndexStorageAdapter for query path adapter
+ * @see QueryableIndexCursorFactory for query path adapter
  * @see QueryableIndexIndexableAdapter for indexing path adapter
  */
 public interface QueryableIndex extends Closeable, ColumnInspector
@@ -48,7 +48,8 @@ public interface QueryableIndex extends Closeable, ColumnInspector
    */
   Indexed getAvailableDimensions();
   BitmapFactory getBitmapFactoryForDimensions();
-  @Nullable Metadata getMetadata();
+  @Nullable
+  Metadata getMetadata();
 
   /**
    * Map of column name to {@link DimensionHandler}, whose contents and iteration order matches
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java
new file mode 100644
index 000000000000..458923267117
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment;
+
+import org.apache.druid.query.OrderBy;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import javax.annotation.Nullable;
+import java.util.LinkedHashSet;
+
+public class QueryableIndexCursorFactory implements CursorFactory
+{
+  private final QueryableIndex index;
+
+  public QueryableIndexCursorFactory(QueryableIndex index)
+  {
+    this.index = index;
+  }
+
+  @Override
+  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
+  {
+    return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build());
+  }
+
+  @Override
+  public RowSignature getRowSignature()
+  {
+    final LinkedHashSet columns = new LinkedHashSet<>();
+
+    for (final OrderBy orderBy : index.getOrdering()) {
+      columns.add(orderBy.getColumnName());
+    }
+
+    // Add __time after the defined ordering, if __time wasn't part of it.
+    columns.add(ColumnHolder.TIME_COLUMN_NAME);
+    columns.addAll(index.getColumnNames());
+
+    final RowSignature.Builder builder = RowSignature.builder();
+    for (final String column : columns) {
+      final ColumnType columnType = ColumnType.fromCapabilities(index.getColumnCapabilities(column));
+
+      // index.getOrdering() may include columns that don't exist, such as if they were omitted due to
+      // being 100% nulls. Don't add those to the row signature.
+      if (columnType != null) {
+        builder.add(column, columnType);
+      }
+    }
+
+    return builder.build();
+  }
+
+  @Nullable
+  @Override
+  public ColumnCapabilities getColumnCapabilities(String column)
+  {
+    return index.getColumnCapabilities(column);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java
index 0060b276c4b2..d9913036b5a6 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java
@@ -265,10 +265,10 @@ public RowIteratorImpl getRows()
   /**
    * On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link
    * SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same
-   * column value selectors as in {@link QueryableIndexStorageAdapter}. The approach with "caching" in {@link
+   * column value selectors as in {@link QueryableIndexCursorFactory}. The approach with "caching" in {@link
    * SettableColumnValueSelector}s is chosen for two reasons:
    *  1) Avoid re-reading column values from serialized format multiple times (because they are accessed multiple times)
-   *     For comparison, it's not a factor for {@link QueryableIndexStorageAdapter} because during query processing,
+   *     For comparison, it's not a factor for {@link QueryableIndexCursorFactory} because during query processing,
    *     column values are usually accessed just once per offset, if aggregator or query runner are written sanely.
    *     Avoiding re-reads is especially important for object columns, because object deserialization is potentially
    *     expensive.
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexPhysicalSegmentInspector.java
similarity index 55%
rename from processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
rename to processing/src/main/java/org/apache/druid/segment/QueryableIndexPhysicalSegmentInspector.java
index a0e23e71f747..44ab4343a6a0 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexPhysicalSegmentInspector.java
@@ -19,121 +19,30 @@
 
 package org.apache.druid.segment;
 
-import org.apache.druid.query.OrderBy;
 import org.apache.druid.segment.column.BaseColumn;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.ColumnIndexSupplier;
-import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.DictionaryEncodedColumn;
-import org.apache.druid.segment.column.RowSignature;
-import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
-import org.joda.time.DateTime;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.UncheckedIOException;
-import java.util.LinkedHashSet;
-import java.util.Set;
 
-/**
- *
- */
-public class QueryableIndexStorageAdapter implements StorageAdapter
+public class QueryableIndexPhysicalSegmentInspector implements PhysicalSegmentInspector
 {
-  public static final int DEFAULT_VECTOR_SIZE = 512;
-
   private final QueryableIndex index;
 
-  public QueryableIndexStorageAdapter(QueryableIndex index)
+  public QueryableIndexPhysicalSegmentInspector(QueryableIndex index)
   {
     this.index = index;
   }
 
   @Override
-  public Interval getInterval()
-  {
-    return index.getDataInterval();
-  }
-
-  @Override
-  public RowSignature getRowSignature()
-  {
-    final LinkedHashSet columns = new LinkedHashSet<>();
-
-    for (final OrderBy orderBy : index.getOrdering()) {
-      columns.add(orderBy.getColumnName());
-    }
-
-    // Add __time after the defined ordering, if __time wasn't part of it.
-    columns.add(ColumnHolder.TIME_COLUMN_NAME);
-
-    for (final String dimension : getAvailableDimensions()) {
-      columns.add(dimension);
-    }
-
-    for (final String metric : getAvailableMetrics()) {
-      columns.add(metric);
-    }
-
-    final RowSignature.Builder builder = RowSignature.builder();
-    for (final String column : columns) {
-      final ColumnType columnType = ColumnType.fromCapabilities(index.getColumnCapabilities(column));
-
-      // index.getOrdering() may include columns that don't exist, such as if they were omitted due to
-      // being 100% nulls. Don't add those to the row signature.
-      if (columnType != null) {
-        builder.add(column, columnType);
-      }
-    }
-
-    return builder.build();
-  }
-
-  @Override
-  public Indexed getAvailableDimensions()
-  {
-    return index.getAvailableDimensions();
-  }
-
-  @Override
-  public Iterable getAvailableMetrics()
-  {
-    // Use LinkedHashSet to preserve the original order.
-    final Set columnNames = new LinkedHashSet<>(index.getColumnNames());
-
-    for (final String dimension : index.getAvailableDimensions()) {
-      columnNames.remove(dimension);
-    }
-
-    return columnNames;
-  }
-
-  @Override
-  public int getDimensionCardinality(String dimension)
-  {
-    ColumnHolder columnHolder = index.getColumnHolder(dimension);
-    if (columnHolder == null) {
-      // NullDimensionSelector has cardinality = 1 (one null, nothing else).
-      return 1;
-    }
-    try (BaseColumn col = columnHolder.getColumn()) {
-      if (!(col instanceof DictionaryEncodedColumn)) {
-        return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
-      }
-      return ((DictionaryEncodedColumn) col).getCardinality();
-    }
-    catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
-
-  @Override
-  public int getNumRows()
+  public Metadata getMetadata()
   {
-    return index.getNumRows();
+    return index.getMetadata();
   }
 
   @Override
@@ -163,31 +72,34 @@ public Comparable getMaxValue(String dimension)
   }
 
   @Override
-  @Nullable
-  public ColumnCapabilities getColumnCapabilities(String column)
-  {
-    return index.getColumnCapabilities(column);
-  }
-
-  @Override
-  public DateTime getMaxIngestedEventTime()
+  public int getDimensionCardinality(String column)
   {
-    // For immutable indexes, maxIngestedEventTime is maxTime.
-    return getMaxTime();
+    ColumnHolder columnHolder = index.getColumnHolder(column);
+    if (columnHolder == null) {
+      // NullDimensionSelector has cardinality = 1 (one null, nothing else).
+      return 1;
+    }
+    try (BaseColumn col = columnHolder.getColumn()) {
+      if (!(col instanceof DictionaryEncodedColumn)) {
+        return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
+      }
+      return ((DictionaryEncodedColumn) col).getCardinality();
+    }
+    catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
   }
 
+  @Nullable
   @Override
-  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
+  public ColumnCapabilities getColumnCapabilities(String column)
   {
-    return new QueryableIndexCursorHolder(
-        index,
-        CursorBuildSpec.builder(spec).build()
-    );
+    return index.getColumnCapabilities(column);
   }
 
   @Override
-  public Metadata getMetadata()
+  public int getNumRows()
   {
-    return index.getMetadata();
+    return index.getNumRows();
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
index ccbe0475c33c..ece082a0b330 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
@@ -39,14 +39,14 @@ public class QueryableIndexSegment implements Segment
       .makeAsMap(QueryableIndexSegment.class);
 
   private final QueryableIndex index;
-  private final QueryableIndexStorageAdapter storageAdapter;
+  private final QueryableIndexCursorFactory cursorFactory;
   private final TimeBoundaryInspector timeBoundaryInspector;
   private final SegmentId segmentId;
 
   public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
   {
     this.index = index;
-    this.storageAdapter = new QueryableIndexStorageAdapter(index);
+    this.cursorFactory = new QueryableIndexCursorFactory(index);
     this.timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index);
     this.segmentId = segmentId;
   }
@@ -70,9 +70,9 @@ public QueryableIndex asQueryableIndex()
   }
 
   @Override
-  public StorageAdapter asStorageAdapter()
+  public CursorFactory asCursorFactory()
   {
-    return storageAdapter;
+    return cursorFactory;
   }
 
   @Override
@@ -97,6 +97,12 @@ public  T as(@Nonnull Class clazz)
 
     if (TimeBoundaryInspector.class.equals(clazz)) {
       return (T) timeBoundaryInspector;
+    } else if (Metadata.class.equals(clazz)) {
+      return (T) index.getMetadata();
+    } else if (PhysicalSegmentInspector.class.equals(clazz)) {
+      return (T) new QueryableIndexPhysicalSegmentInspector(index);
+    } else if (TopNOptimizationInspector.class.equals(clazz)) {
+      return (T) new SimpleTopNOptimizationInspector(true);
     }
 
     return Segment.super.as(clazz);
diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
index b6ce3860b2f3..e2969f10d00e 100644
--- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
@@ -112,11 +112,11 @@ public QueryableIndex asQueryableIndex()
     return !isClosed() ? baseObject.asQueryableIndex() : null;
   }
 
-  @Override
   @Nullable
-  public StorageAdapter asStorageAdapter()
+  @Override
+  public CursorFactory asCursorFactory()
   {
-    return !isClosed() ? baseObject.asStorageAdapter() : null;
+    return !isClosed() ? baseObject.asCursorFactory() : null;
   }
 
   @Override
@@ -185,6 +185,12 @@ public  T as(Class clazz)
     return baseObject.as(clazz);
   }
 
+  @Override
+  public boolean isTombstone()
+  {
+    return baseObject.isTombstone();
+  }
+
   @Override
   public String asString()
   {
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 d1a70783f6b7..46e6ffd7e17b 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java
@@ -31,7 +31,7 @@
 import java.util.function.ToLongFunction;
 
 /**
- * A {@link Cursor} that is based on a stream of objects. Generally created by a {@link RowBasedStorageAdapter}.
+ * A {@link Cursor} that is based on a stream of objects. Generally created by a {@link RowBasedCursorFactory}.
  *
  * @see RowBasedSegment#RowBasedSegment for implementation notes
  */
diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursorFactory.java
similarity index 59%
rename from processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
rename to processing/src/main/java/org/apache/druid/segment/RowBasedCursorFactory.java
index eaed700f59cf..13f2da53e82c 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursorFactory.java
@@ -19,11 +19,7 @@
 
 package org.apache.druid.segment;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.guava.SimpleSequence;
@@ -31,121 +27,26 @@
 import org.apache.druid.query.OrderBy;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.RowSignature;
-import org.apache.druid.segment.data.Indexed;
-import org.apache.druid.segment.data.ListIndexed;
 import org.apache.druid.utils.CloseableUtils;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * A {@link StorageAdapter} that is based on a stream of objects. Generally created by a {@link RowBasedSegment}.
- *
- * @see RowBasedSegment#RowBasedSegment for implementation notes
- */
-public class RowBasedStorageAdapter implements StorageAdapter
+public class RowBasedCursorFactory implements CursorFactory
 {
   private final Sequence rowSequence;
   private final RowAdapter rowAdapter;
   private final RowSignature rowSignature;
 
-  public RowBasedStorageAdapter(
-      final Sequence rowSequence,
-      final RowAdapter rowAdapter,
-      final RowSignature rowSignature
+  public RowBasedCursorFactory(
+      Sequence rowSequence,
+      RowAdapter rowAdapter,
+      RowSignature rowSignature
   )
   {
-    this.rowSequence = Preconditions.checkNotNull(rowSequence, "rowSequence");
-    this.rowAdapter = Preconditions.checkNotNull(rowAdapter, "rowAdapter");
-    this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature");
-  }
-
-  /**
-   * Whether the provided time interval and granularity combination is allowed.
-   *
-   * We restrict ETERNITY with non-ALL granularity, because allowing it would involve creating a very high number
-   * of time grains. This would cause queries to take an excessive amount of time or run out of memory.
-   */
-  public static boolean isQueryGranularityAllowed(final Interval interval, final Granularity granularity)
-  {
-    return Granularities.ALL.equals(granularity) || !Intervals.ETERNITY.equals(interval);
-  }
-
-  @Override
-  public Interval getInterval()
-  {
-    return Intervals.ETERNITY;
-  }
-
-  @Override
-  public Indexed getAvailableDimensions()
-  {
-    return new ListIndexed<>(new ArrayList<>(rowSignature.getColumnNames()));
-  }
-
-  @Override
-  public Iterable getAvailableMetrics()
-  {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public RowSignature getRowSignature()
-  {
-    return rowSignature;
-  }
-
-  @Override
-  public int getDimensionCardinality(String column)
-  {
-    return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMinValue(String column)
-  {
-    return null;
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMaxValue(String column)
-  {
-    return null;
-  }
-
-  @Nullable
-  @Override
-  public ColumnCapabilities getColumnCapabilities(String column)
-  {
-    return RowBasedColumnSelectorFactory.getColumnCapabilities(rowSignature, column);
-  }
-
-  @Override
-  public int getNumRows()
-  {
-    if (rowSequence instanceof SimpleSequence) {
-      final Iterable rowIterable = ((SimpleSequence) rowSequence).getIterable();
-
-      if (rowIterable instanceof Collection) {
-        return ((Collection) rowIterable).size();
-      }
-    }
-
-    // getNumRows is only used by tests and by segmentMetadataQuery (which would be odd to call on inline datasources)
-    // so no big deal if it doesn't always work.
-    throw new UnsupportedOperationException("Cannot retrieve number of rows");
-  }
-
-  @Override
-  public Metadata getMetadata()
-  {
-    throw new UnsupportedOperationException("Cannot retrieve metadata");
+    this.rowSequence = rowSequence;
+    this.rowAdapter = rowAdapter;
+    this.rowSignature = rowSignature;
   }
 
   @Override
@@ -199,6 +100,19 @@ public void close()
     };
   }
 
+  @Override
+  public RowSignature getRowSignature()
+  {
+    return rowSignature;
+  }
+
+  @Nullable
+  @Override
+  public ColumnCapabilities getColumnCapabilities(String column)
+  {
+    return RowBasedColumnSelectorFactory.getColumnCapabilities(rowSignature, column);
+  }
+
   /**
    * Reverse a Sequence.
    *
diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
index 26ff6594bbbc..4ba2c98987d4 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
@@ -20,6 +20,7 @@
 package org.apache.druid.segment;
 
 import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.timeline.SegmentId;
@@ -34,18 +35,20 @@
 public class RowBasedSegment implements Segment
 {
   private final SegmentId segmentId;
-  private final StorageAdapter storageAdapter;
+  private final Sequence rowSequence;
+  private final RowAdapter rowAdapter;
+  private final RowSignature rowSignature;
 
   /**
    * Create a row-based segment.
    *
    * The provided "rowIterable" must be in time-order according to the provided {@link RowAdapter#timestampFunction()}.
-   * The cursor returned by {@link RowBasedStorageAdapter#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify
+   * The cursor returned by {@link RowBasedCursorFactory#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify
    * this, and callers will expect it.
    *
    * The provided "rowSignature" will be used for reporting available columns and their capabilities to users of
-   * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's
-   * storage adapter will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that
+   * {@link #asCursorFactory()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's
+   * cursor factory will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that
    * field, even if it doesn't appear in "rowSignature".
    *
    * @param segmentId    segment identifier; will be returned by {@link #getId()}
@@ -62,11 +65,9 @@ public RowBasedSegment(
   )
   {
     this.segmentId = Preconditions.checkNotNull(segmentId, "segmentId");
-    this.storageAdapter = new RowBasedStorageAdapter<>(
-        rowSequence,
-        rowAdapter,
-        rowSignature
-    );
+    this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature");
+    this.rowSequence = Preconditions.checkNotNull(rowSequence, "rowSequence");
+    this.rowAdapter = Preconditions.checkNotNull(rowAdapter, "rowAdapter");
   }
 
   @Override
@@ -80,7 +81,7 @@ public SegmentId getId()
   @Nonnull
   public Interval getDataInterval()
   {
-    return storageAdapter.getInterval();
+    return Intervals.ETERNITY;
   }
 
   @Nullable
@@ -91,10 +92,9 @@ public QueryableIndex asQueryableIndex()
   }
 
   @Override
-  @Nonnull
-  public StorageAdapter asStorageAdapter()
+  public CursorFactory asCursorFactory()
   {
-    return storageAdapter;
+    return new RowBasedCursorFactory<>(rowSequence, rowAdapter, rowSignature);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/RowWalker.java b/processing/src/main/java/org/apache/druid/segment/RowWalker.java
index f55245b3bca3..a43d1e5f51e9 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowWalker.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowWalker.java
@@ -31,7 +31,7 @@
 import java.util.function.ToLongFunction;
 
 /**
- * Used by {@link RowBasedStorageAdapter} and {@link RowBasedCursor} to walk through rows. It allows multiple
+ * Used by {@link RowBasedCursorFactory} and {@link RowBasedCursor} to walk through rows. It allows multiple
  * {@link RowBasedCursor} to share the same underlying Iterable.
  *
  * The class creates a yielder from the sequence to iterate over the rows. However, it doesn't call the sequence's close
diff --git a/processing/src/main/java/org/apache/druid/segment/Segment.java b/processing/src/main/java/org/apache/druid/segment/Segment.java
index 44be82a56ee7..135f4d556421 100644
--- a/processing/src/main/java/org/apache/druid/segment/Segment.java
+++ b/processing/src/main/java/org/apache/druid/segment/Segment.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.segment;
 
+import org.apache.druid.error.DruidException;
 import org.apache.druid.guice.annotations.PublicApi;
 import org.apache.druid.query.datasourcemetadata.DataSourceMetadataResultValue;
 import org.apache.druid.segment.join.table.IndexedTable;
@@ -44,18 +45,26 @@ public interface Segment extends Closeable
   @Nullable
   QueryableIndex asQueryableIndex();
 
-  StorageAdapter asStorageAdapter();
+  @Deprecated
+  default StorageAdapter asStorageAdapter()
+  {
+    throw DruidException.defensive(
+        "asStorageAdapter is no longer supported, use Segment.asCursorFactory to build cursors, or Segment.as(..) to get various metadata information instead"
+    );
+  }
+
+  CursorFactory asCursorFactory();
 
   /**
    * Request an implementation of a particular interface.
    *
-   * If the passed-in interface is {@link QueryableIndex} or {@link StorageAdapter}, then this method behaves
-   * identically to {@link #asQueryableIndex()} or {@link #asStorageAdapter()}. Other interfaces are only
+   * If the passed-in interface is {@link QueryableIndex} or {@link CursorFactory}, then this method behaves
+   * identically to {@link #asQueryableIndex()} or {@link #asCursorFactory()}. Other interfaces are only
    * expected to be requested by callers that have specific knowledge of extra features provided by specific
    * segment types. For example, an extension might provide a custom Segment type that can offer both
    * StorageAdapter and some new interface. That extension can also offer a Query that uses that new interface.
    *
-   * Implementations which accept classes other than {@link QueryableIndex} or {@link StorageAdapter} are limited
+   * Implementations which accept classes other than {@link QueryableIndex} or {@link CursorFactory} are limited
    * to using those classes within the extension. This means that one extension cannot rely on the `Segment.as`
    * behavior of another extension.
    *
@@ -64,18 +73,23 @@ public interface Segment extends Closeable
    *
    * @return instance of clazz, or null if the interface is not supported by this segment
    *
-   * @see StorageAdapter storage adapter for queries. Never null.
+   * @see CursorFactory to make cursors to run queries. Never null.
    * @see QueryableIndex index object, if this is a memory-mapped regular segment.
    * @see IndexedTable table object, if this is a joinable indexed table.
    * @see TimeBoundaryInspector inspector for min/max timestamps, if supported by this segment.
+   * @see PhysicalSegmentInspector inspector for physical segment details, if supported by this segment.
    * @see MaxIngestedEventTimeInspector inspector for {@link DataSourceMetadataResultValue#getMaxIngestedEventTime()}
+   * @see TopNOptimizationInspector inspector containing information for topN specific optimizations
    * @see CloseableShapeshifter stepping stone to {@link org.apache.druid.query.rowsandcols.RowsAndColumns}.
+   *
    */
   @SuppressWarnings({"unused", "unchecked"})
   @Nullable
   default  T as(@Nonnull Class clazz)
   {
-    if (clazz.equals(QueryableIndex.class)) {
+    if (clazz.equals(CursorFactory.class)) {
+      return (T) asCursorFactory();
+    } else if (clazz.equals(QueryableIndex.class)) {
       return (T) asQueryableIndex();
     } else if (clazz.equals(StorageAdapter.class)) {
       return (T) asStorageAdapter();
@@ -83,6 +97,12 @@ default  T as(@Nonnull Class clazz)
     return null;
   }
 
+  default boolean isTombstone()
+  {
+    return false;
+  }
+
+
   default String asString()
   {
     return getClass().toString();
diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java
index 4e39ca00f18f..a025113f7731 100644
--- a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java
+++ b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java
@@ -40,7 +40,7 @@ public interface SegmentWrangler
    * @return Segments that, collectively, contain data for dataSource. May be empty if dataSource does not exist or
    * has no data in the provided intervals. May contain data outside the provided intervals, so callers should
    * filter it down further, e.g. through the "interval" parameter of {@link CursorBuildSpec} for
-   * {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)}
+   * {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}
    */
   Iterable getSegmentsForIntervals(DataSource dataSource, Iterable intervals);
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java
index 507737a2c5ba..3f6c0f5e8252 100644
--- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java
@@ -33,6 +33,7 @@
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 
@@ -61,11 +62,16 @@ public SimpleQueryableIndex(
     Preconditions.checkNotNull(columns.get(ColumnHolder.TIME_COLUMN_NAME));
     this.dataInterval = Preconditions.checkNotNull(dataInterval, "dataInterval");
     ImmutableList.Builder columnNamesBuilder = ImmutableList.builder();
-    for (String column : columns.keySet()) {
-      if (!ColumnHolder.TIME_COLUMN_NAME.equals(column)) {
-        columnNamesBuilder.add(column);
+    LinkedHashSet dimsFirst = new LinkedHashSet<>();
+    for (String dimName : dimNames) {
+      dimsFirst.add(dimName);
+    }
+    for (String columnName : columns.keySet()) {
+      if (!ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) {
+        dimsFirst.add(columnName);
       }
     }
+    columnNamesBuilder.addAll(dimsFirst);
     this.columnNames = columnNamesBuilder.build();
     this.availableDimensions = dimNames;
     this.bitmapFactory = bitmapFactory;
diff --git a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java b/processing/src/main/java/org/apache/druid/segment/SimpleTopNOptimizationInspector.java
similarity index 69%
rename from processing/src/main/java/org/apache/druid/segment/AbstractSegment.java
rename to processing/src/main/java/org/apache/druid/segment/SimpleTopNOptimizationInspector.java
index c41be41ec4d9..184f514b695f 100644
--- a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/SimpleTopNOptimizationInspector.java
@@ -19,11 +19,18 @@
 
 package org.apache.druid.segment;
 
-/**
- * @deprecated use {@link Segment} directly as this does nothing
- */
-@Deprecated
-public abstract class AbstractSegment implements Segment
+public class SimpleTopNOptimizationInspector implements TopNOptimizationInspector
 {
-  // i used to have a purpose
+  private final boolean allDictionaryIdsPresent;
+
+  public SimpleTopNOptimizationInspector(boolean allDictionaryIdsPresent)
+  {
+    this.allDictionaryIdsPresent = allDictionaryIdsPresent;
+  }
+
+  @Override
+  public boolean areAllDictionaryIdsPresent()
+  {
+    return allDictionaryIdsPresent;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
index c0949692e4fe..5fa7286b7ae6 100644
--- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
@@ -19,14 +19,13 @@
 
 package org.apache.druid.segment;
 
-import com.google.common.collect.Iterables;
 import org.apache.druid.error.DruidException;
 import org.apache.druid.guice.annotations.PublicApi;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.query.OrderBy;
+import org.apache.druid.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.column.ColumnHolder;
-import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.vector.VectorCursor;
@@ -34,185 +33,205 @@
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
-import java.util.List;
 
 /**
  *
  */
+@Deprecated
 @PublicApi
-public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHolderFactory
+public interface StorageAdapter extends ColumnInspector
 {
+  /**
+   * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}
+   * and call {@link CursorHolder#canVectorize()} instead.
+   */
+  @Deprecated
+  default boolean canVectorize(
+      @Nullable Filter filter,
+      VirtualColumns virtualColumns,
+      boolean descending
+  )
+  {
+    throw DruidException.defensive(
+        "canVectorize is no longer supported, use Segment.asCursorFactory().makeCursorHolder(..).canVectorize() instead"
+    );
+  }
 
   /**
-   * Build a {@link CursorHolder} which can provide {@link Cursor} and {@link VectorCursor} (if capable) which allows
-   * scanning segments and creating {@link ColumnSelectorFactory} and
-   * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} respectively to read row values at the cursor
-   * position.
+   * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}
+   * and call {@link CursorHolder#asCursor()} instead.
    */
-  @Override
-  default CursorHolder makeCursorHolder(CursorBuildSpec spec)
-  {
-    // For backwards compatibility, the default implementation assumes the underlying rows are sorted by __time.
-    // Built-in implementations of StorageAdapter must override this method.
-    final List ordering;
-    final boolean descending;
-    if (Cursors.preferDescendingTimeOrdering(spec)) {
-      ordering = Cursors.descendingTimeOrder();
-      descending = true;
-    } else {
-      ordering = Cursors.ascendingTimeOrder();
-      descending = false;
-    }
-    return new CursorHolder()
-    {
-      @Override
-      public boolean canVectorize()
-      {
-        return StorageAdapter.this.canVectorize(
-            spec.getFilter(),
-            spec.getVirtualColumns(),
-            descending
-        );
-      }
-
-      @Override
-      public Cursor asCursor()
-      {
-        return Iterables.getOnlyElement(
-            StorageAdapter.this.makeCursors(
-                spec.getFilter(),
-                spec.getInterval(),
-                spec.getVirtualColumns(),
-                Granularities.ALL,
-                descending,
-                spec.getQueryMetrics()
-            ).toList()
-        );
-      }
-
-      @Override
-      public VectorCursor asVectorCursor()
-      {
-        return StorageAdapter.this.makeVectorCursor(
-            spec.getFilter(),
-            spec.getInterval(),
-            spec.getVirtualColumns(),
-            descending,
-            spec.getQueryContext().getVectorSize(),
-            spec.getQueryMetrics()
-        );
-      }
-
-      @Nullable
-      @Override
-      public List getOrdering()
-      {
-        return ordering;
-      }
-
-      @Override
-      public void close()
-      {
-        // consuming sequences of CursorFactory are expected to close themselves.
-      }
-    };
-  }
-
-  Interval getInterval();
-
-  /**
-   * Returns the names of all dimension columns, not including {@link ColumnHolder#TIME_COLUMN_NAME}.
-   */
-  Indexed getAvailableDimensions();
-
-  /**
-   * Returns the names of all metric columns.
-   */
-  Iterable getAvailableMetrics();
-
-  /**
-   * Returns the row signature of the data available from this adapter. For mutable adapters, even though the signature
-   * may evolve over time, any particular object returned by this method is an immutable snapshot.
+  @Deprecated
+  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 Segment.asCursorFactory().makeCursorHolder(..).asCursor() instead"
+    );
+  }
+
+  /**
+   * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}
+   * and call {@link CursorHolder#asVectorCursor()} instead.
    */
-  default RowSignature getRowSignature()
+  @Deprecated
+  @Nullable
+  default VectorCursor makeVectorCursor(
+      @Nullable Filter filter,
+      Interval interval,
+      VirtualColumns virtualColumns,
+      boolean descending,
+      int vectorSize,
+      @Nullable QueryMetrics queryMetrics
+  )
   {
-    final RowSignature.Builder builder = RowSignature.builder();
-    builder.addTimeColumn();
+    throw DruidException.defensive(
+        "makeVectorCursor is no longer supported, use Segment.asCursorFactory().makeCursorHolder(..).asVectorCursor() instead"
+    );
+  }
 
-    for (final String column : Iterables.concat(getAvailableDimensions(), getAvailableMetrics())) {
-      builder.add(column, ColumnType.fromCapabilities(getColumnCapabilities(column)));
-    }
+  /**
+   * @deprecated Callers should use {@link Segment#getDataInterval()} instead.
+   */
+  @Deprecated
+  default Interval getInterval()
+  {
+    throw DruidException.defensive(
+        "getInterval is no longer supported, use Segment.getDataInterval() instead."
+    );
+  }
 
-    return builder.build();
+  /**
+   * @deprecated Callers should use {@link Segment#as(Class)} to construct a {@link Metadata} instead.
+   */
+  @Deprecated
+  default Indexed getAvailableDimensions()
+  {
+    throw DruidException.defensive(
+        "getAvailableDimensions is no longer supported, use Segment.getRowSignature() and or Segment.as(PhysicalSegmentInspector.class) instead"
+    );
   }
 
   /**
-   * Returns the number of distinct values for a column, or {@link DimensionDictionarySelector#CARDINALITY_UNKNOWN}
-   * if unknown.
-   *
-   * If the column doesn't exist, returns 1, because a column that doesn't exist is treated as a column of default
-   * (or null) values.
+   * @deprecated Callers should use {@link Segment#as(Class)} to construct a {@link Metadata} if available and check
+   * {@link Metadata#getAggregators()} instead.
    */
-  int getDimensionCardinality(String column);
+  @Deprecated
+  default Iterable getAvailableMetrics()
+  {
+    throw DruidException.defensive(
+        "getAvailableMetrics is no longer supported, use Segment.as(PhysicalSegmentInspector.class) instead"
+    );
+  }
 
   /**
-   * Use {@link TimeBoundaryInspector#getMinTime()} instead.
+   * @deprecated use {@link Segment#asCursorFactory()} and {@link CursorFactory#getRowSignature()} instead.
+   */
+  @Deprecated
+  default RowSignature getRowSignature()
+  {
+    throw DruidException.defensive(
+        "getRowSignature is no longer supported, use Segment.asCursorFactory().getRowSignature() instead"
+    );
+  }
+
+  /**
+   * @deprecated Callers should use {@link Segment#as(Class)} to construct a {@link PhysicalSegmentInspector} if
+   * available and call {@link PhysicalSegmentInspector#getDimensionCardinality(String)} instead.
+   */
+  @Deprecated
+  default int getDimensionCardinality(String column)
+  {
+    throw DruidException.defensive(
+        "getDimensionCardinality is no longer supported, use Segment.as(SegmentAnalysisInspector.class) instead"
+    );
+  }
+
+  /**
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link TimeBoundaryInspector} if available and call
+   * {@link TimeBoundaryInspector#getMinTime()} instead.
    */
   @Deprecated
   default DateTime getMinTime()
   {
     throw DruidException.defensive(
-        "getMinTime is no longer supported, use Segment.as(MinMaxValueInspector.class) instead"
+        "getMinTime is no longer supported, use Segment.as(TimeBoundaryInspector.class) instead"
     );
   }
 
   /**
-   * Use {@link TimeBoundaryInspector#getMaxTime()} instead.
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link TimeBoundaryInspector} if available and call
+   * {@link TimeBoundaryInspector#getMaxTime()} instead.
    */
   @Deprecated
   default DateTime getMaxTime()
   {
     throw DruidException.defensive(
-        "getMaxTime is no longer supported, use Segment.as(MinMaxValueInspector.class) instead"
+        "getMaxTime is no longer supported, use Segment.as(TimeBoundaryInspector.class) instead"
     );
   }
 
   /**
-   * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null
-   * if not known. Does not scan the column to find the minimum value.
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link PhysicalSegmentInspector} if available and call
+   * {@link PhysicalSegmentInspector#getMinValue(String)}
    */
+  @Deprecated
   @Nullable
-  Comparable getMinValue(String column);
+  default Comparable getMinValue(String column)
+  {
+    throw DruidException.defensive(
+        "getMinValue is no longer supported, use Segment.as(SegmentAnalysisInspector.class) instead"
+    );
+  }
 
   /**
-   * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null
-   * if not known. Does not scan the column to find the maximum value.
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link PhysicalSegmentInspector} if available and call
+   * {@link PhysicalSegmentInspector#getMaxValue(String)}
    */
+  @Deprecated
   @Nullable
-  Comparable getMaxValue(String column);
-
-  /**
-   * Returns capabilities of a particular column, if known. May be null if the column doesn't exist, or if
-   * the column does exist but the capabilities are unknown. The latter is possible with dynamically discovered
-   * columns.
-   *
-   * Note that StorageAdapters are representations of "real" segments, so they are not aware of any virtual columns
-   * that may be involved in a query. In general, query engines should instead use the method
-   * {@link ColumnSelectorFactory#getColumnCapabilities(String)}, which returns capabilities for virtual columns as
-   * well.
-   *
-   * @param column column name
-   *
-   * @return capabilities, or null
+  default Comparable getMaxValue(String column)
+  {
+    throw DruidException.defensive(
+        "getMaxValue is no longer supported, use Segment.as(SegmentAnalysisInspector.class) instead"
+    );
+  }
+
+  /**
+   * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#getColumnCapabilities(String)}
+   * instead.
    */
+  @Deprecated
   @Override
   @Nullable
-  ColumnCapabilities getColumnCapabilities(String column);
+  default ColumnCapabilities getColumnCapabilities(String column)
+  {
+    throw DruidException.defensive(
+        "getColumnCapabilities is no longer supported, use Segment.asCursorFactory().getColumnCapabilities(..) instead"
+    );
+  }
 
-  int getNumRows();
+  /**
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link PhysicalSegmentInspector} if available then call
+   * {@link PhysicalSegmentInspector#getNumRows()} instead.
+   */
+  @Deprecated
+  default int getNumRows()
+  {
+    throw DruidException.defensive(
+        "getNumRows is no longer supported, use Segment.as(PhysicalSegmentInspector.class) instead"
+    );
+  }
 
   /**
-   * Use {@link MaxIngestedEventTimeInspector#getMaxIngestedEventTime()} instead.
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link MaxIngestedEventTimeInspector} if available and call
+   * {@link MaxIngestedEventTimeInspector#getMaxIngestedEventTime()} instead.
    */
   @Deprecated
   default DateTime getMaxIngestedEventTime()
@@ -222,27 +241,38 @@ default DateTime getMaxIngestedEventTime()
     );
   }
 
+  /**
+   * @deprecated Use {@link Segment#as(Class)} to fetch a {@link Metadata} if available
+   */
+  @Deprecated
   @Nullable
-  Metadata getMetadata();
+  default Metadata getMetadata()
+  {
+    throw DruidException.defensive(
+        "getMetadata is no longer supported, use Segment.as(PhysicalSegmentInspector.class) instead"
+    );
+  }
 
   /**
-   * Returns true if this storage adapter can filter some rows out. The actual column cardinality can be lower than
-   * what {@link #getDimensionCardinality} returns if this returns true. Dimension selectors for such storage adapter
-   * can return non-contiguous dictionary IDs because the dictionary IDs in filtered rows will not be returned.
-   * Note that the number of rows accessible via this storage adapter will not necessarily decrease because of
-   * the built-in filters. For inner joins, for example, the number of joined rows can be larger than
-   * the number of rows in the base adapter even though this method returns true.
+   * @deprecated Use {@link Segment#as(Class)} to get a {@link TopNOptimizationInspector} if available and call
+   * {@link TopNOptimizationInspector#areAllDictionaryIdsPresent()} instead.
    */
+  @Deprecated
   default boolean hasBuiltInFilters()
   {
-    return false;
+    throw DruidException.defensive(
+        "hasBuiltInFilters is no longer supported, use Segment.as(FilteredSegmentInspector.class) instead"
+    );
   }
 
   /**
-   * @return true if this index was created from a tombstone or false otherwise
+   * @deprecated Use {@link Segment#isTombstone()}
    */
+  @Deprecated
   default boolean isFromTombstone()
   {
-    return false;
+    throw DruidException.defensive(
+        "isFromTombstone is no longer supported, use Segment.isTombstone instead"
+    );
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java b/processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java
new file mode 100644
index 000000000000..a5a637ad70d2
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment;
+
+/**
+ * oddly specific {@link Segment} inspector for topN optimizations
+ */
+public interface TopNOptimizationInspector
+{
+  /**
+   * Returns true if all values in the dictionary are present in rows. Returns false such as this segment can filter
+   * some rows out even if the query does not specify a filter, making the actual column cardinality be lower than
+   * what {@link DimensionDictionarySelector#getValueCardinality()} returns. Dimension selectors for such cursors can
+   * return non-contiguous dictionary IDs because the dictionary IDs in filtered rows will not be returned. Note that
+   * the number of rows accessible via cursors created on these segments will not necessarily decrease because of the
+   * built-in filters. For inner joins, for example, the number of joined rows can be larger than the number of rows in
+   * the base segment even though this method returns false.
+   */
+  boolean areAllDictionaryIdsPresent();
+}
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 a2a920cba1d5..57aa5683c8dd 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
@@ -203,7 +203,7 @@ public Class classOfObject()
       public ColumnCapabilities getColumnCapabilities(String column)
       {
         if (outputName.equals(column)) {
-          return UnnestStorageAdapter.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn);
+          return UnnestCursorFactory.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn);
         }
 
         return baseColumnSelectorFactory.getColumnCapabilities(column);
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java
similarity index 83%
rename from processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
rename to processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java
index 2f9552a1d3c2..3337f3b43363 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java
@@ -22,7 +22,6 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.query.OrderBy;
@@ -39,8 +38,6 @@
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.column.TypeSignature;
 import org.apache.druid.segment.column.ValueType;
-import org.apache.druid.segment.data.Indexed;
-import org.apache.druid.segment.data.ListIndexed;
 import org.apache.druid.segment.filter.AndFilter;
 import org.apache.druid.segment.filter.BoundFilter;
 import org.apache.druid.segment.filter.Filters;
@@ -51,45 +48,30 @@
 import org.apache.druid.segment.join.PostJoinCursor;
 import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.utils.CloseableUtils;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.function.Supplier;
 
-/**
- * This class serves as the Storage Adapter for the Unnest Segment and is responsible for creating the cursors
- * If the column is dictionary encoded it creates {@link UnnestDimensionCursor} else {@link UnnestColumnValueSelectorCursor}
- * These cursors help navigate the segments for these cases
- */
-public class UnnestStorageAdapter implements StorageAdapter
+public class UnnestCursorFactory implements CursorFactory
 {
-  public StorageAdapter getBaseAdapter()
-  {
-    return baseAdapter;
-  }
-
-  private final StorageAdapter baseAdapter;
+  private final CursorFactory baseCursorFactory;
   private final VirtualColumn unnestColumn;
-  private final String outputColumnName;
-
   @Nullable
-  private final DimFilter unnestFilter;
+  private final DimFilter filter;
 
-  public UnnestStorageAdapter(
-      final StorageAdapter baseAdapter,
-      final VirtualColumn unnestColumn,
-      @Nullable final DimFilter unnestFilter
+  public UnnestCursorFactory(
+      CursorFactory baseCursorFactory,
+      VirtualColumn unnestColumn,
+      @Nullable DimFilter filter
   )
   {
-    this.baseAdapter = baseAdapter;
+    this.baseCursorFactory = baseCursorFactory;
     this.unnestColumn = unnestColumn;
-    this.outputColumnName = unnestColumn.getOutputName();
-    this.unnestFilter = unnestFilter;
+    this.filter = filter;
   }
 
   @Override
@@ -98,10 +80,11 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
     final String input = getUnnestInputIfDirectAccess(unnestColumn);
     final Pair filterPair = computeBaseAndPostUnnestFilters(
         spec.getFilter(),
-        unnestFilter != null ? unnestFilter.toFilter() : null,
+        filter != null ? filter.toFilter() : null,
         spec.getVirtualColumns(),
         input,
-        input == null ? null : spec.getVirtualColumns().getColumnCapabilitiesWithFallback(baseAdapter, input)
+        input == null ? null : spec.getVirtualColumns()
+                                   .getColumnCapabilitiesWithFallback(baseCursorFactory, input)
     );
     final CursorBuildSpec unnestBuildSpec =
         CursorBuildSpec.builder(spec)
@@ -113,7 +96,7 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
     {
       final Closer closer = Closer.create();
       final Supplier cursorHolderSupplier = Suppliers.memoize(
-          () -> closer.register(baseAdapter.makeCursorHolder(unnestBuildSpec))
+          () -> closer.register(baseCursorFactory.makeCursorHolder(unnestBuildSpec))
       );
 
       @Override
@@ -134,14 +117,14 @@ public Cursor asCursor()
               cursor,
               cursor.getColumnSelectorFactory(),
               unnestColumn,
-              outputColumnName
+              unnestColumn.getOutputName()
           );
         } else {
           unnestCursor = new UnnestColumnValueSelectorCursor(
               cursor,
               cursor.getColumnSelectorFactory(),
               unnestColumn,
-              outputColumnName
+              unnestColumn.getOutputName()
           );
         }
         return PostJoinCursor.wrap(
@@ -165,116 +148,55 @@ public void close()
     };
   }
 
-  @Override
-  public Interval getInterval()
-  {
-    return baseAdapter.getInterval();
-  }
-
   @Override
   public RowSignature getRowSignature()
   {
     final RowSignature.Builder builder = RowSignature.builder();
 
-    final RowSignature baseSignature = baseAdapter.getRowSignature();
+    final RowSignature baseSignature = baseCursorFactory.getRowSignature();
     for (int i = 0; i < baseSignature.size(); i++) {
       final String column = baseSignature.getColumnName(i);
-      if (!outputColumnName.equals(column)) {
+      if (!unnestColumn.getOutputName().equals(column)) {
         builder.add(column, ColumnType.fromCapabilities(getColumnCapabilities(column)));
       }
     }
 
-    return builder.add(outputColumnName, ColumnType.fromCapabilities(getColumnCapabilities(outputColumnName))).build();
-  }
-
-  @Override
-  public Indexed getAvailableDimensions()
-  {
-    final LinkedHashSet availableDimensions = new LinkedHashSet<>();
-
-    for (String dim : baseAdapter.getAvailableDimensions()) {
-      availableDimensions.add(dim);
-    }
-    availableDimensions.add(outputColumnName);
-    return new ListIndexed<>(Lists.newArrayList(availableDimensions));
-  }
-
-  @Override
-  public Iterable getAvailableMetrics()
-  {
-    return baseAdapter.getAvailableMetrics();
-  }
-
-  @Nullable
-  public Filter getUnnestFilter()
-  {
-    return unnestFilter == null ? null : unnestFilter.toFilter();
-  }
-
-  @Override
-  public int getDimensionCardinality(String column)
-  {
-    if (!outputColumnName.equals(column)) {
-      return baseAdapter.getDimensionCardinality(column);
-    }
-    return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMinValue(String column)
-  {
-    if (outputColumnName.equals(column)) {
-      return null;
-    }
-
-    return baseAdapter.getMinValue(column);
+    return builder.add(
+        unnestColumn.getOutputName(),
+        ColumnType.fromCapabilities(getColumnCapabilities(unnestColumn.getOutputName()))
+    ).build();
   }
 
-  @Nullable
-  @Override
-  public Comparable getMaxValue(String column)
-  {
-    if (outputColumnName.equals(column)) {
-      return null;
-    }
-
-    return baseAdapter.getMaxValue(column);
-  }
 
   @Nullable
   @Override
   public ColumnCapabilities getColumnCapabilities(String column)
   {
-    if (outputColumnName.equals(column)) {
-      return computeOutputColumnCapabilities(baseAdapter, unnestColumn);
+    if (unnestColumn.getOutputName().equals(column)) {
+      return computeOutputColumnCapabilities(baseCursorFactory, unnestColumn);
     }
 
-    return baseAdapter.getColumnCapabilities(column);
+    return baseCursorFactory.getColumnCapabilities(column);
   }
 
-  @Override
-  public int getNumRows()
+  @VisibleForTesting
+  public VirtualColumn getUnnestColumn()
   {
-    return 0;
+    return unnestColumn;
   }
 
+  /**
+   * Returns the input of {@link #unnestColumn}, if it's a direct access; otherwise returns null.
+   */
+  @VisibleForTesting
   @Nullable
-  @Override
-  public Metadata getMetadata()
-  {
-    return baseAdapter.getMetadata();
-  }
-
-  @Override
-  public boolean isFromTombstone()
-  {
-    return baseAdapter.isFromTombstone();
-  }
-
-  public VirtualColumn getUnnestColumn()
+  public String getUnnestInputIfDirectAccess(VirtualColumn unnestColumn)
   {
-    return unnestColumn;
+    if (unnestColumn instanceof ExpressionVirtualColumn) {
+      return ((ExpressionVirtualColumn) unnestColumn).getParsedExpression().get().getBindingIfIdentifier();
+    } else {
+      return null;
+    }
   }
 
   /**
@@ -287,6 +209,7 @@ public VirtualColumn getUnnestColumn()
    * @param inputColumnCapabilites input column capabilities if known; otherwise null
    * @return pair of pre- and post-unnest filters
    */
+  @VisibleForTesting
   public Pair computeBaseAndPostUnnestFilters(
       @Nullable final Filter queryFilter,
       @Nullable final Filter unnestFilter,
@@ -333,10 +256,10 @@ to generate filters to be passed to base cursor (filtersPushedDownToBaseCursor)
        filtersPushedDownToBaseCursor -> null (as the filter cannot be re-written due to presence of virtual columns)
        filtersForPostUnnestCursor -> d12 IN (a,b) or m1 < 10
      */
-    final FilterSplitter filterSplitter = new FilterSplitter(inputColumn, inputColumnCapabilites, queryVirtualColumns);
+    final FilterSplitter filterSplitter = new FilterSplitter(inputColumn, inputColumnCapabilites, unnestColumn, queryVirtualColumns);
 
     if (queryFilter != null) {
-      if (queryFilter.getRequiredColumns().contains(outputColumnName)) {
+      if (queryFilter.getRequiredColumns().contains(unnestColumn.getOutputName())) {
         // outside filter contains unnested column
         // requires check for OR and And filters, disqualify rewrite for non-unnest filters
         if (queryFilter instanceof BooleanFilter) {
@@ -374,93 +297,6 @@ to generate filters to be passed to base cursor (filtersPushedDownToBaseCursor)
     );
   }
 
-  class FilterSplitter
-  {
-    private String inputColumn;
-    private ColumnCapabilities inputColumnCapabilites;
-    private VirtualColumns queryVirtualColumns;
-
-    private int originalFilterCount = 0;
-    private int preFilterCount = 0;
-
-    public FilterSplitter(
-        String inputColumn,
-        ColumnCapabilities inputColumnCapabilites,
-        VirtualColumns queryVirtualColumns
-    )
-    {
-      this.inputColumn = inputColumn;
-      this.inputColumnCapabilites = inputColumnCapabilites;
-      this.queryVirtualColumns = queryVirtualColumns;
-    }
-
-    final List filtersPushedDownToBaseCursor = new ArrayList<>();
-    final List filtersForPostUnnestCursor = new ArrayList<>();
-
-    void addPostFilterWithPreFilterIfRewritePossible(@Nullable final Filter filter, boolean skipPreFilters)
-    {
-      if (filter == null) {
-        return;
-      }
-      if (!skipPreFilters) {
-        final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(filter, inputColumn, inputColumnCapabilites);
-        if (newFilter != null) {
-          // Add the rewritten filter pre-unnest, so we get the benefit of any indexes, and so we avoid unnesting
-          // any rows that do not match this filter at all.
-          filtersPushedDownToBaseCursor.add(newFilter);
-        }
-      }
-      // Add original filter post-unnest no matter what: we need to filter out any extraneous unnested values.
-      filtersForPostUnnestCursor.add(filter);
-    }
-
-    void addPreFilter(@Nullable final Filter filter)
-    {
-      if (filter == null) {
-        return;
-      }
-
-      final Set requiredColumns = filter.getRequiredColumns();
-
-      // Run filter post-unnest if it refers to any virtual columns. This is a conservative judgement call
-      // that perhaps forces the code to use a ValueMatcher where an index would've been available,
-      // which can have real performance implications. This is an interim choice made to value correctness
-      // over performance. When we need to optimize this performance, we should be able to
-      // create a VirtualColumnDatasource that contains all the virtual columns, in which case the query
-      // itself would stop carrying them and everything should be able to be pushed down.
-      if (queryVirtualColumns.getVirtualColumns().length > 0) {
-        for (String column : requiredColumns) {
-          if (queryVirtualColumns.exists(column)) {
-            filtersForPostUnnestCursor.add(filter);
-            return;
-          }
-        }
-      }
-      filtersPushedDownToBaseCursor.add(filter);
-
-    }
-
-    public void addToOriginalFilterCount(int c)
-    {
-      originalFilterCount += c;
-    }
-
-    public void addToPreFilterCount(int c)
-    {
-      preFilterCount += c;
-    }
-
-    public int getOriginalFilterCount()
-    {
-      return originalFilterCount;
-    }
-
-    public int getPreFilterCount()
-    {
-      return preFilterCount;
-    }
-  }
-
   /**
    * handles the nested rewrite for unnest columns in recursive way,
    * it loops through all and/or filters and rewrite only required filters in the child and add it to preFilter if qualified
@@ -491,7 +327,7 @@ private List recursiveRewriteOnUnnestFilters(
   {
     final List preFilterList = new ArrayList<>();
     for (Filter filter : queryFilter.getFilters()) {
-      if (filter.getRequiredColumns().contains(outputColumnName)) {
+      if (filter.getRequiredColumns().contains(unnestColumn.getOutputName())) {
         if (filter instanceof AndFilter) {
           List andChildFilters = recursiveRewriteOnUnnestFilters(
               (BooleanFilter) filter,
@@ -519,6 +355,7 @@ private List recursiveRewriteOnUnnestFilters(
           // can we rewrite
           final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(
               filter,
+              unnestColumn,
               inputColumn,
               inputColumnCapabilites
           );
@@ -539,26 +376,32 @@ private List recursiveRewriteOnUnnestFilters(
     }
     return preFilterList;
   }
+
   /**
-   * Returns the input of {@link #unnestColumn}, if it's a direct access; otherwise returns null.
+   * Computes ordering of a join {@link CursorHolder} based on the ordering of an underlying {@link CursorHolder}.
    */
-  @Nullable
-  public String getUnnestInputIfDirectAccess(VirtualColumn unnestColumn)
+  private List computeOrdering(final List baseOrdering)
   {
-    if (unnestColumn instanceof ExpressionVirtualColumn) {
-      return ((ExpressionVirtualColumn) unnestColumn).getParsedExpression().get().getBindingIfIdentifier();
-    } else {
-      return null;
+    // Sorted the same way as the base segment, unless the unnested column shadows one of the base columns.
+    int limit = 0;
+    for (; limit < baseOrdering.size(); limit++) {
+      final String columnName = baseOrdering.get(limit).getColumnName();
+      if (columnName.equals(unnestColumn.getOutputName())) {
+        break;
+      }
     }
+
+    return limit == baseOrdering.size() ? baseOrdering : baseOrdering.subList(0, limit);
   }
 
   /**
-   * Rewrites a filter on {@link #outputColumnName} to operate on the input column from
+   * Rewrites a filter on {@link #unnestColumn} to operate on the input column from
    * if possible.
    */
   @Nullable
-  private Filter rewriteFilterOnUnnestColumnIfPossible(
+  private static Filter rewriteFilterOnUnnestColumnIfPossible(
       final Filter filter,
+      final VirtualColumn unnestColumn,
       @Nullable final String inputColumn,
       @Nullable final ColumnCapabilities inputColumnCapabilities
   )
@@ -571,31 +414,14 @@ private Filter rewriteFilterOnUnnestColumnIfPossible(
     }
 
     if (filterMapsOverMultiValueStrings(filter)) {
-      return filter.rewriteRequiredColumns(ImmutableMap.of(outputColumnName, inputColumn));
+      return filter.rewriteRequiredColumns(ImmutableMap.of(unnestColumn.getOutputName(), inputColumn));
     } else {
       return null;
     }
   }
 
   /**
-   * Computes ordering of a join {@link CursorHolder} based on the ordering of an underlying {@link CursorHolder}.
-   */
-  private List computeOrdering(final List baseOrdering)
-  {
-    // Sorted the same way as the base segment, unless the unnested column shadows one of the base columns.
-    int limit = 0;
-    for (; limit < baseOrdering.size(); limit++) {
-      final String columnName = baseOrdering.get(limit).getColumnName();
-      if (columnName.equals(outputColumnName) || columnName.equals(unnestColumn.getOutputName())) {
-        break;
-      }
-    }
-
-    return limit == baseOrdering.size() ? baseOrdering : baseOrdering.subList(0, limit);
-  }
-
-  /**
-   * Computes the capabilities of {@link #outputColumnName}, after unnesting.
+   * Computes the capabilities of {@link #unnestColumn}, after unnesting.
    */
   @Nullable
   public static ColumnCapabilities computeOutputColumnCapabilities(
@@ -629,8 +455,7 @@ public static ColumnCapabilities computeOutputColumnCapabilities(
    * over multi-value strings. (Rather than treat them as arrays.) There isn't a method on the Filter interface that
    * tells us this, so resort to instanceof.
    */
-  @VisibleForTesting
-  static boolean filterMapsOverMultiValueStrings(final Filter filter)
+  private static boolean filterMapsOverMultiValueStrings(final Filter filter)
   {
     if (filter instanceof BooleanFilter) {
       for (Filter child : ((BooleanFilter) filter).getFilters()) {
@@ -678,4 +503,94 @@ private static boolean useDimensionCursor(@Nullable ColumnCapabilities capabilit
     // wasn't a dictionary encoded string, use the value selector
     return false;
   }
+
+  private static class FilterSplitter
+  {
+    private final String inputColumn;
+    private final ColumnCapabilities inputColumnCapabilites;
+    private final VirtualColumn unnestColumn;
+    private final VirtualColumns queryVirtualColumns;
+
+    private int originalFilterCount = 0;
+    private int preFilterCount = 0;
+
+    public FilterSplitter(
+        String inputColumn,
+        ColumnCapabilities inputColumnCapabilites,
+        VirtualColumn unnestColumn,
+        VirtualColumns queryVirtualColumns
+    )
+    {
+      this.inputColumn = inputColumn;
+      this.inputColumnCapabilites = inputColumnCapabilites;
+      this.unnestColumn = unnestColumn;
+      this.queryVirtualColumns = queryVirtualColumns;
+    }
+
+    final List filtersPushedDownToBaseCursor = new ArrayList<>();
+    final List filtersForPostUnnestCursor = new ArrayList<>();
+
+    void addPostFilterWithPreFilterIfRewritePossible(@Nullable final Filter filter, boolean skipPreFilters)
+    {
+      if (filter == null) {
+        return;
+      }
+      if (!skipPreFilters) {
+        final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(filter, unnestColumn, inputColumn, inputColumnCapabilites);
+        if (newFilter != null) {
+          // Add the rewritten filter pre-unnest, so we get the benefit of any indexes, and so we avoid unnesting
+          // any rows that do not match this filter at all.
+          filtersPushedDownToBaseCursor.add(newFilter);
+        }
+      }
+      // Add original filter post-unnest no matter what: we need to filter out any extraneous unnested values.
+      filtersForPostUnnestCursor.add(filter);
+    }
+
+    void addPreFilter(@Nullable final Filter filter)
+    {
+      if (filter == null) {
+        return;
+      }
+
+      final Set requiredColumns = filter.getRequiredColumns();
+
+      // Run filter post-unnest if it refers to any virtual columns. This is a conservative judgement call
+      // that perhaps forces the code to use a ValueMatcher where an index would've been available,
+      // which can have real performance implications. This is an interim choice made to value correctness
+      // over performance. When we need to optimize this performance, we should be able to
+      // create a VirtualColumnDatasource that contains all the virtual columns, in which case the query
+      // itself would stop carrying them and everything should be able to be pushed down.
+      if (queryVirtualColumns.getVirtualColumns().length > 0) {
+        for (String column : requiredColumns) {
+          if (queryVirtualColumns.exists(column)) {
+            filtersForPostUnnestCursor.add(filter);
+            return;
+          }
+        }
+      }
+      filtersPushedDownToBaseCursor.add(filter);
+
+    }
+
+    public void addToOriginalFilterCount(int c)
+    {
+      originalFilterCount += c;
+    }
+
+    public void addToPreFilterCount(int c)
+    {
+      preFilterCount += c;
+    }
+
+    public int getOriginalFilterCount()
+    {
+      return originalFilterCount;
+    }
+
+    public int getPreFilterCount()
+    {
+      return preFilterCount;
+    }
+  }
 }
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 398c0a21deff..3012d31ff2a0 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java
@@ -251,7 +251,7 @@ public ColumnValueSelector makeColumnValueSelector(String columnName)
       public ColumnCapabilities getColumnCapabilities(String column)
       {
         if (outputName.equals(column)) {
-          return UnnestStorageAdapter.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn);
+          return UnnestCursorFactory.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn);
         }
 
         return baseColumnSelectorFactory.getColumnCapabilities(column);
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java b/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java
new file mode 100644
index 000000000000..22b63655f52d
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment;
+
+import org.apache.druid.query.filter.DimFilter;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+public class UnnestSegment extends WrappedSegmentReference
+{
+  private final VirtualColumn unnestColumn;
+  @Nullable
+  private final DimFilter filter;
+
+  public UnnestSegment(
+      SegmentReference delegate,
+      VirtualColumn unnestColumn,
+      @Nullable DimFilter filter
+  )
+  {
+    super(delegate);
+    this.unnestColumn = unnestColumn;
+    this.filter = filter;
+  }
+
+  @Override
+  public CursorFactory asCursorFactory()
+  {
+    return new UnnestCursorFactory(delegate.asCursorFactory(), unnestColumn, filter);
+  }
+
+  @Nullable
+  @Override
+  public  T as(@Nonnull Class clazz)
+  {
+    if (TopNOptimizationInspector.class.equals(clazz)) {
+      return (T) new SimpleTopNOptimizationInspector(filter == null);
+    }
+    return super.as(clazz);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java
index c5f2474efdfa..8b50315e2f4b 100644
--- a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java
@@ -97,8 +97,8 @@ T makeMultiValueDimensionProcessor(
   T makeObjectProcessor(@SuppressWarnings("unused") ColumnCapabilities capabilities, VectorObjectSelector selector);
 
   /**
-   * The processor factory can influence the decision on whether or not to prefer a dictionary encoded column value
-   * selector over a an object selector by examining the {@link ColumnCapabilities}.
+   * The processor factory can influence the decision on whether to prefer a dictionary encoded column value selector
+   * over an object selector by examining the {@link ColumnCapabilities}.
    *
    * By default, all processor factories prefer to use a dictionary encoded selector if the column has a dictionary
    * available ({@link ColumnCapabilities#isDictionaryEncoded()} is true), and there is a unique mapping of dictionary
diff --git a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
index 19db47067b57..97bb1afd4b77 100644
--- a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
+++ b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
@@ -19,7 +19,6 @@
 
 package org.apache.druid.segment;
 
-import org.apache.druid.query.FilteredDataSource;
 import org.apache.druid.timeline.SegmentId;
 import org.joda.time.Interval;
 
@@ -28,25 +27,20 @@
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Optional;
-import java.util.function.Function;
 
 /**
- * This class is used as a wrapper for other classes that just want to
- * modify the storage adapter for a datasource. Examples include:
- * {@link org.apache.druid.query.UnnestDataSource}, {@link FilteredDataSource}
+ * Simple {@link SegmentReference} implementation for a segment that wraps a base segment such as
+ * {@link UnnestSegment} or {@link FilteredSegment}
  */
-public class WrappedSegmentReference implements SegmentReference
+public abstract class WrappedSegmentReference implements SegmentReference
 {
-  private final SegmentReference delegate;
-  private final Function storageAdapterWrapperFunction;
+  protected final SegmentReference delegate;
 
   public WrappedSegmentReference(
-      SegmentReference delegate,
-      Function storageAdapterWrapperFunction
+      SegmentReference delegate
   )
   {
     this.delegate = delegate;
-    this.storageAdapterWrapperFunction = storageAdapterWrapperFunction;
   }
 
   @Override
@@ -74,12 +68,6 @@ public QueryableIndex asQueryableIndex()
     return delegate.asQueryableIndex();
   }
 
-  @Override
-  public StorageAdapter asStorageAdapter()
-  {
-    return storageAdapterWrapperFunction.apply(delegate.asStorageAdapter());
-  }
-
   @Nullable
   @Override
   public  T as(@Nonnull Class clazz)
@@ -91,10 +79,22 @@ public  T as(@Nonnull Class clazz)
     }
   }
 
+  @Override
+  public boolean isTombstone()
+  {
+    return delegate.isTombstone();
+  }
+
   @Override
   public void close() throws IOException
   {
     delegate.close();
   }
+
+  @Override
+  public String asString()
+  {
+    return delegate.asString();
+  }
 }
 
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
index cfcc99c10e6c..fc2a02c47b7b 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
@@ -98,11 +98,11 @@
 
 /**
  * In-memory, row-based data structure used to hold data during ingestion. Realtime tasks query this index using
- * {@link IncrementalIndexStorageAdapter}.
+ * {@link IncrementalIndexCursorFactory}.
  *
  * Concurrency model: {@link #add(InputRow)} and {@link #add(InputRow, boolean)} are not thread-safe, and must be
  * called from a single thread or externally synchronized. However, the methods that support
- * {@link IncrementalIndexStorageAdapter} are thread-safe, and may be called concurrently with each other, and with
+ * {@link IncrementalIndexCursorFactory} are thread-safe, and may be called concurrently with each other, and with
  * the "add" methods. This concurrency model supports real-time queries of the data in the index.
  */
 public abstract class IncrementalIndex implements Iterable, Closeable, ColumnInspector
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java
index f43b81c40adc..86e8c6690c2d 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java
@@ -22,6 +22,7 @@
 import org.apache.druid.query.Order;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.extraction.ExtractionFn;
+import org.apache.druid.segment.ColumnInspector;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionIndexer;
@@ -41,24 +42,32 @@
  */
 class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, RowIdSupplier
 {
-  private final IncrementalIndexStorageAdapter adapter;
+  private final ColumnInspector snapshotColumnInspector;
   private final IncrementalIndex index;
   private final VirtualColumns virtualColumns;
   private final Order timeOrder;
   private final IncrementalIndexRowHolder rowHolder;
 
   IncrementalIndexColumnSelectorFactory(
-      IncrementalIndexStorageAdapter adapter,
+      IncrementalIndex index,
       VirtualColumns virtualColumns,
       Order timeOrder,
       IncrementalIndexRowHolder rowHolder
   )
   {
-    this.adapter = adapter;
-    this.index = adapter.index;
+    this.index = index;
     this.virtualColumns = virtualColumns;
     this.timeOrder = timeOrder;
     this.rowHolder = rowHolder;
+    this.snapshotColumnInspector = new ColumnInspector()
+    {
+      @Nullable
+      @Override
+      public ColumnCapabilities getColumnCapabilities(String column)
+      {
+        return IncrementalIndexCursorFactory.snapshotColumnCapabilities(index, column);
+      }
+    };
   }
 
   @Override
@@ -132,8 +141,8 @@ public ColumnValueSelector makeColumnValueSelector(String columnName)
   @Nullable
   public ColumnCapabilities getColumnCapabilities(String columnName)
   {
-    // Use adapter.getColumnCapabilities instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater)
-    return virtualColumns.getColumnCapabilitiesWithFallback(adapter, columnName);
+    // Use snapshotColumnInspector instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater)
+    return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, columnName);
   }
 
   @Nullable
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java
similarity index 52%
rename from processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java
rename to processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java
index c9a6d209697a..e034f820dfbf 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java
@@ -21,29 +21,19 @@
 
 import com.google.common.collect.Iterables;
 import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
 import org.apache.druid.segment.CursorHolder;
-import org.apache.druid.segment.DimensionDictionarySelector;
-import org.apache.druid.segment.DimensionIndexer;
-import org.apache.druid.segment.Metadata;
 import org.apache.druid.segment.NestedDataColumnIndexerV4;
-import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
-import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
-import org.apache.druid.segment.data.Indexed;
-import org.apache.druid.segment.data.ListIndexed;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 
-/**
- *
- */
-public class IncrementalIndexStorageAdapter implements StorageAdapter
+public class IncrementalIndexCursorFactory implements CursorFactory
 {
-  private static final ColumnCapabilities.CoercionLogic STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC =
+  private static final ColumnCapabilities.CoercionLogic COERCE_LOGIC =
       new ColumnCapabilities.CoercionLogic()
       {
         @Override
@@ -77,51 +67,17 @@ public boolean hasNulls()
         }
       };
 
-  private static final ColumnCapabilities.CoercionLogic SNAPSHOT_STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC =
-      new ColumnCapabilities.CoercionLogic()
-      {
-        @Override
-        public boolean dictionaryEncoded()
-        {
-          return true;
-        }
-
-        @Override
-        public boolean dictionaryValuesSorted()
-        {
-          return true;
-        }
-
-        @Override
-        public boolean dictionaryValuesUnique()
-        {
-          return true;
-        }
+  private final IncrementalIndex index;
 
-        @Override
-        public boolean multipleValues()
-        {
-          return false;
-        }
-
-        @Override
-        public boolean hasNulls()
-        {
-          return false;
-        }
-      };
-
-  final IncrementalIndex index;
-
-  public IncrementalIndexStorageAdapter(IncrementalIndex index)
+  public IncrementalIndexCursorFactory(IncrementalIndex index)
   {
     this.index = index;
   }
 
   @Override
-  public Interval getInterval()
+  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
-    return index.getInterval();
+    return new IncrementalIndexCursorHolder(index, spec);
   }
 
   @Override
@@ -136,67 +92,14 @@ public RowSignature getRowSignature()
     return builder.build();
   }
 
-  @Override
-  public Indexed getAvailableDimensions()
-  {
-    return new ListIndexed<>(index.getDimensionNames(false));
-  }
-
-  @Override
-  public Iterable getAvailableMetrics()
-  {
-    return index.getMetricNames();
-  }
-
-  @Override
-  public int getDimensionCardinality(String dimension)
-  {
-    if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) {
-      return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
-    }
-
-    IncrementalIndex.DimensionDesc desc = index.getDimension(dimension);
-    if (desc == null) {
-      return 0;
-    }
-
-    return desc.getIndexer().getCardinality();
-  }
-
-  @Override
-  public int getNumRows()
-  {
-    return index.size();
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMinValue(String column)
-  {
-    IncrementalIndex.DimensionDesc desc = index.getDimension(column);
-    if (desc == null) {
-      return null;
-    }
-
-    DimensionIndexer indexer = desc.getIndexer();
-    return indexer.getMinValue();
-  }
-
   @Nullable
   @Override
-  public Comparable getMaxValue(String column)
+  public ColumnCapabilities getColumnCapabilities(String column)
   {
-    IncrementalIndex.DimensionDesc desc = index.getDimension(column);
-    if (desc == null) {
-      return null;
-    }
-
-    DimensionIndexer indexer = desc.getIndexer();
-    return indexer.getMaxValue();
+    return snapshotColumnCapabilities(index, column);
   }
 
-  @Override
-  public ColumnCapabilities getColumnCapabilities(String column)
+  static ColumnCapabilities snapshotColumnCapabilities(IncrementalIndex index, String column)
   {
     IncrementalIndex.DimensionDesc desc = index.getDimension(column);
     // nested column indexer is a liar, and behaves like any type if it only processes unnested literals of a single
@@ -211,7 +114,7 @@ public ColumnCapabilities getColumnCapabilities(String column)
     //
     // We don't want to represent this as having-multiple-values in index.getCapabilities, because that's used
     // at index-persisting time to determine if we need a multi-value column or not. However, that means we
-    // need to tweak the capabilities here in the StorageAdapter (a query-time construct), so at query time
+    // need to tweak the capabilities here in the CursorFactory (a query-time construct), so at query time
     // they appear multi-valued.
     //
     // Note that this could be improved if we snapshot the capabilities at cursor creation time and feed those through
@@ -220,33 +123,7 @@ public ColumnCapabilities getColumnCapabilities(String column)
     // be removed.
     return ColumnCapabilitiesImpl.snapshot(
         index.getColumnCapabilities(column),
-        STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC
+        COERCE_LOGIC
     );
   }
-
-  /**
-   * Sad workaround for {@link org.apache.druid.query.metadata.SegmentAnalyzer} to deal with the fact that the
-   * response from {@link #getColumnCapabilities} is not accurate for string columns, in that it reports all string
-   * columns as having multiple values. This method returns the actual capabilities of the underlying
-   * {@link IncrementalIndex} at the time this method is called.
-   */
-  public ColumnCapabilities getSnapshotColumnCapabilities(String column)
-  {
-    return ColumnCapabilitiesImpl.snapshot(
-        index.getColumnCapabilities(column),
-        SNAPSHOT_STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC
-    );
-  }
-
-  @Override
-  public CursorHolder makeCursorHolder(CursorBuildSpec spec)
-  {
-    return new IncrementalIndexCursorHolder(this, index, spec);
-  }
-
-  @Override
-  public Metadata getMetadata()
-  {
-    return index.getMetadata();
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
index 33c08d5920cb..02c09398d8e5 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java
@@ -41,18 +41,15 @@
 
 public class IncrementalIndexCursorHolder implements CursorHolder
 {
-  private final IncrementalIndexStorageAdapter storageAdapter;
   private final IncrementalIndex index;
   private final CursorBuildSpec spec;
   private final List ordering;
 
   public IncrementalIndexCursorHolder(
-      IncrementalIndexStorageAdapter storageAdapter,
       IncrementalIndex index,
       CursorBuildSpec spec
   )
   {
-    this.storageAdapter = storageAdapter;
     this.index = index;
     this.spec = spec;
     if (index.timePosition == 0) {
@@ -81,7 +78,6 @@ public Cursor asCursor()
 
 
     return new IncrementalIndexCursor(
-        storageAdapter,
         index,
         spec.getVirtualColumns(),
         Cursors.getTimeOrdering(ordering),
@@ -110,7 +106,6 @@ static class IncrementalIndexCursor implements Cursor
     private boolean done;
 
     IncrementalIndexCursor(
-        IncrementalIndexStorageAdapter storageAdapter,
         IncrementalIndex index,
         VirtualColumns virtualColumns,
         Order timeOrder,
@@ -120,7 +115,7 @@ static class IncrementalIndexCursor implements Cursor
     {
       currEntry = new IncrementalIndexRowHolder();
       columnSelectorFactory = new IncrementalIndexColumnSelectorFactory(
-          storageAdapter,
+          index,
           virtualColumns,
           timeOrder,
           currEntry
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java
new file mode 100644
index 000000000000..c015206a9da2
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment.incremental;
+
+import org.apache.druid.segment.DimensionDictionarySelector;
+import org.apache.druid.segment.DimensionIndexer;
+import org.apache.druid.segment.Metadata;
+import org.apache.druid.segment.PhysicalSegmentInspector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+
+public class IncrementalIndexPhysicalSegmentInspector implements PhysicalSegmentInspector
+{
+  private static final ColumnCapabilities.CoercionLogic SNAPSHOT_COERCE_LOGIC =
+      new ColumnCapabilities.CoercionLogic()
+      {
+        @Override
+        public boolean dictionaryEncoded()
+        {
+          return true;
+        }
+
+        @Override
+        public boolean dictionaryValuesSorted()
+        {
+          return true;
+        }
+
+        @Override
+        public boolean dictionaryValuesUnique()
+        {
+          return true;
+        }
+
+        @Override
+        public boolean multipleValues()
+        {
+          return false;
+        }
+
+        @Override
+        public boolean hasNulls()
+        {
+          return false;
+        }
+      };
+
+  private final IncrementalIndex index;
+
+  public IncrementalIndexPhysicalSegmentInspector(IncrementalIndex index)
+  {
+    this.index = index;
+  }
+
+  @Nullable
+  @Override
+  public Metadata getMetadata()
+  {
+    return index.getMetadata();
+  }
+
+  @Nullable
+  @Override
+  public Comparable getMinValue(String column)
+  {
+    IncrementalIndex.DimensionDesc desc = index.getDimension(column);
+    if (desc == null) {
+      return null;
+    }
+
+    DimensionIndexer indexer = desc.getIndexer();
+    return indexer.getMinValue();
+  }
+
+  @Nullable
+  @Override
+  public Comparable getMaxValue(String column)
+  {
+    IncrementalIndex.DimensionDesc desc = index.getDimension(column);
+    if (desc == null) {
+      return null;
+    }
+
+    DimensionIndexer indexer = desc.getIndexer();
+    return indexer.getMaxValue();
+  }
+
+  @Override
+  public int getDimensionCardinality(String column)
+  {
+    if (column.equals(ColumnHolder.TIME_COLUMN_NAME)) {
+      return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
+    }
+
+    IncrementalIndex.DimensionDesc desc = index.getDimension(column);
+    if (desc == null) {
+      // non-existent dimension has cardinality = 1 (one null, nothing else).
+      return 1;
+    }
+
+    return desc.getIndexer().getCardinality();
+  }
+
+  @Nullable
+  @Override
+  public ColumnCapabilities getColumnCapabilities(String column)
+  {
+    return ColumnCapabilitiesImpl.snapshot(
+        index.getColumnCapabilities(column),
+        SNAPSHOT_COERCE_LOGIC
+    );
+  }
+
+  @Override
+  public int getNumRows()
+  {
+    return index.size();
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java
index 0e7e82f01c3e..15909ac1b527 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java
@@ -66,7 +66,7 @@ private static RowPointer makeRowPointer(
   {
     ColumnSelectorFactory columnSelectorFactory =
         new IncrementalIndexColumnSelectorFactory(
-            new IncrementalIndexStorageAdapter(incrementalIndex),
+            incrementalIndex,
             VirtualColumns.EMPTY,
             incrementalIndex.timePosition == 0 ? Order.ASCENDING : Order.NONE,
             rowHolder
diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
index 0cf54b8c1484..b6a8da3fbaed 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
@@ -23,12 +23,14 @@
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.filter.Filter;
-import org.apache.druid.query.rowsandcols.StorageAdapterRowsAndColumns;
+import org.apache.druid.query.rowsandcols.CursorFactoryRowsAndColumns;
 import org.apache.druid.segment.CloseableShapeshifter;
+import org.apache.druid.segment.CursorFactory;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.SegmentReference;
-import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.SimpleTopNOptimizationInspector;
 import org.apache.druid.segment.TimeBoundaryInspector;
+import org.apache.druid.segment.TopNOptimizationInspector;
 import org.apache.druid.segment.WrappedTimeBoundaryInspector;
 import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
 import org.apache.druid.timeline.SegmentId;
@@ -36,7 +38,6 @@
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
@@ -104,16 +105,37 @@ public QueryableIndex asQueryableIndex()
   }
 
   @Override
-  public StorageAdapter asStorageAdapter()
+  public CursorFactory asCursorFactory()
   {
-    return new HashJoinSegmentStorageAdapter(
-        baseSegment.asStorageAdapter(),
+    return new HashJoinSegmentCursorFactory(
+        baseSegment.asCursorFactory(),
         baseFilter,
         clauses,
         joinFilterPreAnalysis
     );
   }
 
+  @SuppressWarnings("unchecked")
+  @Override
+  public  T as(Class clazz)
+  {
+    if (CloseableShapeshifter.class.equals(clazz)) {
+      return (T) new CursorFactoryRowsAndColumns(asCursorFactory());
+    } else if (TimeBoundaryInspector.class.equals(clazz)) {
+      return (T) WrappedTimeBoundaryInspector.create(baseSegment.as(TimeBoundaryInspector.class));
+    } else if (TopNOptimizationInspector.class.equals(clazz)) {
+      // if the baseFilter is not null, then rows from underlying cursor can be potentially filtered.
+      // otherwise, a filtering inner or left join can also filter rows.
+      return (T) new SimpleTopNOptimizationInspector(
+          baseFilter == null && clauses.stream().allMatch(
+              clause -> clause.getJoinType().isLefty() || clause.getCondition().isAlwaysTrue()
+          )
+      );
+    } else {
+      return SegmentReference.super.as(clazz);
+    }
+  }
+
   @Override
   public void close() throws IOException
   {
@@ -134,7 +156,7 @@ public Optional acquireReferences()
         if (acquireFailed) {
           break;
         }
-        acquireFailed |= joinClause.acquireReferences().map(closeable -> {
+        acquireFailed = joinClause.acquireReferences().map(closeable -> {
           closer.register(closeable);
           return false;
         }).orElse(true);
@@ -153,17 +175,4 @@ public Optional acquireReferences()
       return Optional.empty();
     }
   }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public  T as(Class clazz)
-  {
-    if (CloseableShapeshifter.class.equals(clazz)) {
-      return (T) new StorageAdapterRowsAndColumns(this.asStorageAdapter());
-    } else if (TimeBoundaryInspector.class.equals(clazz)) {
-      return (T) WrappedTimeBoundaryInspector.create(baseSegment.as(TimeBoundaryInspector.class));
-    } else {
-      return SegmentReference.super.as(clazz);
-    }
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactory.java
similarity index 56%
rename from processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
rename to processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactory.java
index 765ab4fbd917..c359a96fad09 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactory.java
@@ -28,22 +28,18 @@
 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.CursorHolder;
-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.ColumnType;
 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.filter.Filters;
 import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
 import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
 import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
 import org.apache.druid.segment.join.filter.JoinFilterSplit;
 import org.apache.druid.utils.CloseableUtils;
-import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
@@ -51,171 +47,27 @@
 import java.util.List;
 import java.util.Optional;
 
-public class HashJoinSegmentStorageAdapter implements StorageAdapter
+public class HashJoinSegmentCursorFactory implements CursorFactory
 {
-  private final StorageAdapter baseAdapter;
-
+  private final CursorFactory baseCursorFactory;
   @Nullable
   private final Filter baseFilter;
   private final List clauses;
   private final JoinFilterPreAnalysis joinFilterPreAnalysis;
 
-  /**
-   * @param baseAdapter           A StorageAdapter for the left-hand side base segment
-   * @param clauses               The right-hand side clauses. The caller is responsible for ensuring that there are no
-   *                              duplicate prefixes or prefixes that shadow each other across the clauses
-   * @param joinFilterPreAnalysis Pre-analysis for the query we expect to run on this storage adapter
-   */
-  HashJoinSegmentStorageAdapter(
-      final StorageAdapter baseAdapter,
-      final List clauses,
-      final JoinFilterPreAnalysis joinFilterPreAnalysis
-  )
-  {
-    this(baseAdapter, null, clauses, joinFilterPreAnalysis);
-  }
-
-  /**
-   * @param baseAdapter           A StorageAdapter for the left-hand side base segment
-   * @param baseFilter            A filter for the left-hand side base segment
-   * @param clauses               The right-hand side clauses. The caller is responsible for ensuring that there are no
-   *                              duplicate prefixes or prefixes that shadow each other across the clauses
-   * @param joinFilterPreAnalysis Pre-analysis for the query we expect to run on this storage adapter
-   */
-  HashJoinSegmentStorageAdapter(
-      final StorageAdapter baseAdapter,
-      @Nullable final Filter baseFilter,
-      final List clauses,
-      final JoinFilterPreAnalysis joinFilterPreAnalysis
+  public HashJoinSegmentCursorFactory(
+      CursorFactory baseCursorFactory,
+      @Nullable Filter baseFilter,
+      List clauses,
+      JoinFilterPreAnalysis joinFilterPreAnalysis
   )
   {
-    this.baseAdapter = baseAdapter;
+    this.baseCursorFactory = baseCursorFactory;
     this.baseFilter = baseFilter;
     this.clauses = clauses;
     this.joinFilterPreAnalysis = joinFilterPreAnalysis;
   }
 
-  @Override
-  public Interval getInterval()
-  {
-    return baseAdapter.getInterval();
-  }
-
-  @Override
-  public RowSignature getRowSignature()
-  {
-    // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields.
-    final LinkedHashSet columns = new LinkedHashSet<>(baseAdapter.getRowSignature().getColumnNames());
-
-    for (final JoinableClause clause : clauses) {
-      columns.addAll(clause.getAvailableColumnsPrefixed());
-    }
-
-    final RowSignature.Builder builder = RowSignature.builder();
-    for (final String column : columns) {
-      builder.add(column, ColumnType.fromCapabilities(getColumnCapabilities(column)));
-    }
-
-    return builder.build();
-  }
-
-  @Override
-  public Indexed getAvailableDimensions()
-  {
-    // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields.
-    final LinkedHashSet availableDimensions = new LinkedHashSet<>();
-
-    baseAdapter.getAvailableDimensions().forEach(availableDimensions::add);
-
-    for (JoinableClause clause : clauses) {
-      availableDimensions.addAll(clause.getAvailableColumnsPrefixed());
-    }
-
-    return new ListIndexed<>(Lists.newArrayList(availableDimensions));
-  }
-
-  @Override
-  public Iterable getAvailableMetrics()
-  {
-    return baseAdapter.getAvailableMetrics();
-  }
-
-  @Override
-  public int getDimensionCardinality(String column)
-  {
-    final Optional maybeClause = getClauseForColumn(column);
-
-    if (maybeClause.isPresent()) {
-      final JoinableClause clause = maybeClause.get();
-      return clause.getJoinable().getCardinality(clause.unprefix(column));
-    } else {
-      return baseAdapter.getDimensionCardinality(column);
-    }
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMinValue(String column)
-  {
-    if (isBaseColumn(column)) {
-      return baseAdapter.getMinValue(column);
-    } else {
-      return null;
-    }
-  }
-
-  @Nullable
-  @Override
-  public Comparable getMaxValue(String column)
-  {
-    if (isBaseColumn(column)) {
-      return baseAdapter.getMaxValue(column);
-    } else {
-      return null;
-    }
-  }
-
-  @Nullable
-  @Override
-  public ColumnCapabilities getColumnCapabilities(String column)
-  {
-    final Optional maybeClause = getClauseForColumn(column);
-
-    if (maybeClause.isPresent()) {
-      final JoinableClause clause = maybeClause.get();
-      return clause.getJoinable().getColumnCapabilities(clause.unprefix(column));
-    } else {
-      return baseAdapter.getColumnCapabilities(column);
-    }
-  }
-
-  @Override
-  public int getNumRows()
-  {
-    // Cannot determine the number of rows ahead of time for a join segment (rows may be added or removed based
-    // on the join condition). At the time of this writing, this method is only used by the 'segmentMetadata' query,
-    // which isn't meant to support join segments anyway.
-    throw new UnsupportedOperationException("Cannot retrieve number of rows from join segment");
-  }
-
-  @Override
-  public Metadata getMetadata()
-  {
-    // Cannot get meaningful Metadata for this segment, since it isn't real. At the time of this writing, this method
-    // is only used by the 'segmentMetadata' query, which isn't meant to support join segments anyway.
-    throw new UnsupportedOperationException("Cannot retrieve metadata from join segment");
-  }
-
-  @Override
-  public boolean hasBuiltInFilters()
-  {
-    // if the baseFilter is not null, then rows from underlying storage adapter can be potentially filtered.
-    // otherwise, a filtering inner join can also filter rows.
-    return baseFilter != null || clauses.stream().anyMatch(
-        clause -> clause.getJoinType() == JoinType.INNER && !clause.getCondition().isAlwaysTrue()
-    );
-  }
-
   @Override
   public CursorHolder makeCursorHolder(CursorBuildSpec spec)
   {
@@ -229,7 +81,7 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       // if there are no clauses, we can just use the base cursor directly if we apply the combined filter
       final CursorBuildSpec newSpec = cursorBuildSpecBuilder.setFilter(combinedFilter)
                                                             .build();
-      return baseAdapter.makeCursorHolder(newSpec);
+      return baseCursorFactory.makeCursorHolder(newSpec);
     }
 
     return new CursorHolder()
@@ -237,19 +89,19 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
       final Closer joinablesCloser = Closer.create();
 
       /**
-       * Typically the same as {@link HashJoinSegmentStorageAdapter#joinFilterPreAnalysis}, but may differ when
+       * Typically the same as {@link HashJoinSegmentCursorFactory#joinFilterPreAnalysis}, but may differ when
        * an unnest datasource is layered on top of a join datasource.
        */
       final JoinFilterPreAnalysis actualPreAnalysis;
 
       /**
        * Result of {@link JoinFilterAnalyzer#splitFilter} on {@link #actualPreAnalysis} and
-       * {@link HashJoinSegmentStorageAdapter#baseFilter}.
+       * {@link HashJoinSegmentCursorFactory#baseFilter}.
        */
       final JoinFilterSplit joinFilterSplit;
 
       /**
-       * Cursor holder for {@link HashJoinSegmentStorageAdapter#baseAdapter}.
+       * Cursor holder for {@link HashJoinSegmentCursorFactory#baseCursorFactory}.
        */
       final CursorHolder baseCursorHolder;
 
@@ -297,8 +149,7 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
         );
         cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns);
 
-        baseCursorHolder =
-            joinablesCloser.register(baseAdapter.makeCursorHolder(cursorBuildSpecBuilder.build()));
+        baseCursorHolder = joinablesCloser.register(baseCursorFactory.makeCursorHolder(cursorBuildSpecBuilder.build()));
       }
 
       @Override
@@ -338,35 +189,53 @@ public void close()
   }
 
   @Override
-  public boolean isFromTombstone()
+  public RowSignature getRowSignature()
   {
-    return baseAdapter.isFromTombstone();
-  }
+    // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields.
+    final RowSignature baseSignature = baseCursorFactory.getRowSignature();
 
-  /**
-   * Returns whether "column" will be selected from "baseAdapter". This is true if it is not shadowed by any joinables
-   * (i.e. if it does not start with any of their prefixes).
-   */
-  public boolean isBaseColumn(final String column)
-  {
-    return !getClauseForColumn(column).isPresent();
-  }
+    final LinkedHashSet columns = new LinkedHashSet<>(baseSignature.getColumnNames());
+    for (final JoinableClause clause : clauses) {
+      columns.addAll(clause.getAvailableColumnsPrefixed());
+    }
 
-  /**
-   * Returns the JoinableClause corresponding to a particular column, based on the clauses' prefixes.
-   *
-   * @param column column name
-   *
-   * @return the clause, or absent if the column does not correspond to any clause
-   */
-  private Optional getClauseForColumn(final String column)
-  {
+    final RowSignature.Builder builder = RowSignature.builder();
     // Check clauses in reverse, since "makeCursorHolder" creates the cursor in such a way that the last clause
     // gets first dibs to claim a column.
-    return Lists.reverse(clauses)
-                .stream()
-                .filter(clause -> clause.includesColumn(column))
-                .findFirst();
+    LinkedHashSet reverseClauses = new LinkedHashSet<>(Lists.reverse(clauses));
+    for (final String column : columns) {
+      final Optional maybeClause = reverseClauses.stream()
+                                                                 .filter(c -> c.includesColumn(column))
+                                                                 .findFirst();
+      if (maybeClause.isPresent()) {
+        final JoinableClause clause = maybeClause.get();
+        builder.add(
+            column,
+            ColumnType.fromCapabilities(clause.getJoinable().getColumnCapabilities(clause.unprefix(column)))
+        );
+      } else {
+        builder.add(column, baseSignature.getColumnType(column).get());
+      }
+    }
+
+    return builder.build();
+  }
+
+  @Nullable
+  @Override
+  public ColumnCapabilities getColumnCapabilities(String column)
+  {
+    final Optional maybeClause = Lists.reverse(clauses)
+                                                      .stream()
+                                                      .filter(x -> x.includesColumn(column))
+                                                      .findFirst();
+
+    if (maybeClause.isPresent()) {
+      final JoinableClause clause = maybeClause.get();
+      return clause.getJoinable().getColumnCapabilities(clause.unprefix(column));
+    } else {
+      return baseCursorFactory.getColumnCapabilities(column);
+    }
   }
 
   @Nullable
@@ -383,7 +252,7 @@ private List computeOrdering(final List baseOrdering)
     // Sorted the same way as the base segment, unless a joined-in column shadows one of the base columns.
     int limit = 0;
     for (; limit < baseOrdering.size(); limit++) {
-      if (!isBaseColumn(baseOrdering.get(limit).getColumnName())) {
+      if (!baseCursorFactory.getRowSignature().contains(baseOrdering.get(limit).getColumnName())) {
         break;
       }
     }
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 b8ef88996ff1..5d9848d74718 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
@@ -30,7 +30,7 @@
 import javax.annotation.Nullable;
 
 /**
- * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#makeCursorHolder(CursorBuildSpec)} to add post-join
+ * A Cursor decorator used by {@link HashJoinSegmentCursorFactory#makeCursorHolder(CursorBuildSpec)} to add post-join
  * virtual columns and filters.
  */
 public class PostJoinCursor implements Cursor
diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
index 5fe6d8a698a8..9a56ec430cb2 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
@@ -29,13 +29,13 @@
 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.CursorHolder;
 import org.apache.druid.segment.Cursors;
 import org.apache.druid.segment.NilColumnValueSelector;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.QueryableIndexColumnSelectorFactory;
 import org.apache.druid.segment.QueryableIndexSegment;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.SimpleAscendingOffset;
 import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.BaseColumn;
@@ -60,7 +60,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable
   private static final byte CACHE_PREFIX = 0x01;
 
   private final QueryableIndexSegment segment;
-  private final QueryableIndexStorageAdapter adapter;
+  private final CursorFactory cursorFactory;
   private final QueryableIndex queryableIndex;
   private final Set keyColumns;
   private final RowSignature rowSignature;
@@ -76,18 +76,18 @@ public BroadcastSegmentIndexedTable(
     this.keyColumns = keyColumns;
     this.version = version;
     this.segment = Preconditions.checkNotNull(theSegment, "Segment must not be null");
-    this.adapter = Preconditions.checkNotNull(
-        (QueryableIndexStorageAdapter) segment.asStorageAdapter(),
-        "Segment[%s] must have a QueryableIndexStorageAdapter",
+    this.cursorFactory = Preconditions.checkNotNull(
+        segment.asCursorFactory(),
+        "Segment[%s] must have a cursor factory",
         segment.getId()
     );
     this.queryableIndex = Preconditions.checkNotNull(
-        segment.asQueryableIndex(),
-        "Segment[%s] must have a QueryableIndexSegment",
+        segment.as(QueryableIndex.class),
+        "Segment[%s] must have a QueryableIndex",
         segment.getId()
     );
 
-    this.rowSignature = adapter.getRowSignature();
+    this.rowSignature = cursorFactory.getRowSignature();
 
     // initialize keycolumn index builders
     final ArrayList indexBuilders = new ArrayList<>(rowSignature.size());
@@ -107,7 +107,7 @@ public BroadcastSegmentIndexedTable(
       indexBuilders.add(m);
     }
 
-    try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+    try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
       final Cursor cursor = cursorHolder.asCursor();
       if (cursor == null) {
         this.keyColumnsIndexes = Collections.emptyList();
@@ -123,7 +123,7 @@ public BroadcastSegmentIndexedTable(
           .stream()
           .map(columnName -> {
             // multi-value dimensions are not currently supported
-            if (adapter.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) {
+            if (columnSelectorFactory.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) {
               return NilColumnValueSelector.instance();
             }
             return columnSelectorFactory.makeColumnValueSelector(columnName);
@@ -178,7 +178,7 @@ public RowSignature rowSignature()
   @Override
   public int numRows()
   {
-    return adapter.getNumRows();
+    return queryableIndex.getNumRows();
   }
 
   @Override
@@ -193,7 +193,7 @@ public Reader columnReader(int column)
     if (!rowSignature.contains(column)) {
       throw new IAE("Column[%d] is not a valid column for segment[%s]", column, segment.getId());
     }
-    final SimpleAscendingOffset offset = new SimpleAscendingOffset(adapter.getNumRows());
+    final SimpleAscendingOffset offset = new SimpleAscendingOffset(queryableIndex.getNumRows());
     final BaseColumn baseColumn = queryableIndex.getColumnHolder(rowSignature.getColumnName(column)).getColumn();
     final BaseObjectColumnValueSelector selector = baseColumn.makeColumnValueSelector(offset);
 
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 ceaffe846e28..68807c8c2848 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
@@ -25,10 +25,8 @@
 import org.apache.druid.frame.read.FrameReader;
 import org.apache.druid.frame.read.columnar.FrameColumnReader;
 import org.apache.druid.frame.read.columnar.FrameColumnReaders;
-import org.apache.druid.frame.segment.FrameStorageAdapter;
 import org.apache.druid.frame.segment.columnar.FrameQueryableIndex;
 import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
@@ -39,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.CursorFactory;
 import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.NilColumnValueSelector;
 import org.apache.druid.segment.QueryableIndex;
@@ -114,22 +113,22 @@ public FrameBasedIndexedTable(
       indexBuilders.add(m);
     }
 
-    final Sequence storageAdapters = Sequences.simple(
+    final Sequence cursorFactories = Sequences.simple(
         frameBasedInlineDataSource
             .getFrames()
             .stream()
             .map(frameSignaturePair -> {
               Frame frame = frameSignaturePair.getFrame();
               RowSignature rowSignature = frameSignaturePair.getRowSignature();
-              return new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY);
+              return FrameReader.create(rowSignature).makeCursorFactory(frame);
             })
             .collect(Collectors.toList())
     );
 
     final Sequence sequence = Sequences.map(
-        storageAdapters,
-        storageAdapter -> {
-          try (final CursorHolder holder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+        cursorFactories,
+        cursorFactory -> {
+          try (final CursorHolder holder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
             final Cursor cursor = holder.asCursor();
             if (cursor == null) {
               return 0;
diff --git a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
index bb6ad8aaca97..f07c039ea4e3 100644
--- a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
@@ -23,14 +23,18 @@
 import com.google.common.base.Preconditions;
 import org.apache.druid.collections.bitmap.BitmapFactory;
 import org.apache.druid.query.OrderBy;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
+import org.apache.druid.segment.CursorHolder;
 import org.apache.druid.segment.DimensionHandler;
 import org.apache.druid.segment.Metadata;
 import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.SegmentLazyLoadFailCallback;
-import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
@@ -129,16 +133,7 @@ public ColumnHolder getColumnHolder(String columnName)
 
         };
 
-    final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex)
-    {
-      @Override
-      public boolean isFromTombstone()
-      {
-        return true;
-      }
-    };
-
-    Segment segmentObject = new Segment()
+    final Segment segmentObject = new Segment()
     {
       @Override
       public SegmentId getId()
@@ -149,7 +144,7 @@ public SegmentId getId()
       @Override
       public Interval getDataInterval()
       {
-        return asQueryableIndex().getDataInterval();
+        return tombstone.getInterval();
       }
 
       @Nullable
@@ -160,9 +155,43 @@ public QueryableIndex asQueryableIndex()
       }
 
       @Override
-      public StorageAdapter asStorageAdapter()
+      public CursorFactory asCursorFactory()
+      {
+        return new CursorFactory()
+        {
+          @Override
+          public CursorHolder makeCursorHolder(CursorBuildSpec spec)
+          {
+            return new CursorHolder()
+            {
+              @Nullable
+              @Override
+              public Cursor asCursor()
+              {
+                return null;
+              }
+            };
+          }
+
+          @Override
+          public RowSignature getRowSignature()
+          {
+            return RowSignature.empty();
+          }
+
+          @Override
+          @Nullable
+          public ColumnCapabilities getColumnCapabilities(String column)
+          {
+            return null;
+          }
+        };
+      }
+
+      @Override
+      public boolean isTombstone()
       {
-        return storageAdapter;
+        return true;
       }
 
       @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java b/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java
index 762f2e73fc77..7362b38a1c62 100644
--- a/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java
+++ b/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java
@@ -20,8 +20,8 @@
 package org.apache.druid.segment.vector;
 
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.segment.IdLookup;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
@@ -29,12 +29,12 @@
 public class NilVectorSelector
     implements VectorValueSelector, VectorObjectSelector, SingleValueDimensionVectorSelector, IdLookup
 {
-  private static final boolean[] DEFAULT_NULLS_VECTOR = new boolean[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
-  private static final int[] DEFAULT_INT_VECTOR = new int[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
-  private static final long[] DEFAULT_LONG_VECTOR = new long[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
-  private static final float[] DEFAULT_FLOAT_VECTOR = new float[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
-  private static final double[] DEFAULT_DOUBLE_VECTOR = new double[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
-  private static final Object[] DEFAULT_OBJECT_VECTOR = new Object[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
+  private static final boolean[] DEFAULT_NULLS_VECTOR = new boolean[QueryContexts.DEFAULT_VECTOR_SIZE];
+  private static final int[] DEFAULT_INT_VECTOR = new int[QueryContexts.DEFAULT_VECTOR_SIZE];
+  private static final long[] DEFAULT_LONG_VECTOR = new long[QueryContexts.DEFAULT_VECTOR_SIZE];
+  private static final float[] DEFAULT_FLOAT_VECTOR = new float[QueryContexts.DEFAULT_VECTOR_SIZE];
+  private static final double[] DEFAULT_DOUBLE_VECTOR = new double[QueryContexts.DEFAULT_VECTOR_SIZE];
+  private static final Object[] DEFAULT_OBJECT_VECTOR = new Object[QueryContexts.DEFAULT_VECTOR_SIZE];
 
   static {
     Arrays.fill(DEFAULT_NULLS_VECTOR, NullHandling.sqlCompatible());
@@ -69,7 +69,7 @@ private NilVectorSelector(
 
   public static NilVectorSelector create(final VectorSizeInspector vectorSizeInspector)
   {
-    if (vectorSizeInspector.getMaxVectorSize() <= QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE) {
+    if (vectorSizeInspector.getMaxVectorSize() <= QueryContexts.DEFAULT_VECTOR_SIZE) {
       // Reuse static vars when possible.
       return new NilVectorSelector(
           vectorSizeInspector,
diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java
index 103d769fc477..4771605583b8 100644
--- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java
@@ -20,11 +20,11 @@
 package org.apache.druid.segment.vector;
 
 import org.apache.druid.segment.CursorBuildSpec;
-import org.apache.druid.segment.CursorHolderFactory;
+import org.apache.druid.segment.CursorFactory;
 
 /**
  * Vectorized cursor used during query execution. VectorCursors are available from
- * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via
+ * {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} via
  * {@link org.apache.druid.segment.CursorHolder#asVectorCursor()}.
  * 

* See {@link org.apache.druid.query.vector.VectorCursorGranularizer} for a helper that makes it easier for query diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java index 82e8a1f8eb80..1f42002758f8 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java @@ -26,6 +26,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer; import org.apache.druid.query.groupby.epinephelinae.vector.GroupByVectorColumnSelector; +import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.column.RowSignature; import java.util.HashMap; @@ -90,6 +91,15 @@ public int getGroupingKeySize() return exprKeyBytes; } + @Override + public int getValueCardinality() + { + if (subSelectors.size() == 1) { + return subSelectors.get(0).getValueCardinality(); + } + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } + @Override public int writeKeys(WritableMemory keySpace, int keySize, int keyOffset, int startRow, int endRow) { diff --git a/processing/src/test/java/org/apache/druid/frame/FrameTest.java b/processing/src/test/java/org/apache/druid/frame/FrameTest.java index 029e34bc0116..7a26bbacd047 100644 --- a/processing/src/test/java/org/apache/druid/frame/FrameTest.java +++ b/processing/src/test/java/org/apache/druid/frame/FrameTest.java @@ -31,10 +31,10 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.ByteBufferUtils; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -74,7 +74,7 @@ public static class GoodFramesTest extends InitializedNullHandlingTest @Before public void setUp() { - final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); final List sortBy = ImmutableList.of( new KeyColumn("quality", KeyOrder.DESCENDING), @@ -83,7 +83,7 @@ public void setUp() columnarFrame = Iterables.getOnlyElement( FrameSequenceBuilder - .fromAdapter(adapter) + .fromCursorFactory(cursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList() @@ -91,7 +91,7 @@ public void setUp() rowBasedSortedFrame = Iterables.getOnlyElement( FrameSequenceBuilder - .fromAdapter(adapter) + .fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .sortBy(sortBy) .frames() @@ -318,9 +318,9 @@ public static Iterable constructorFeeder() @BeforeClass public static void setUpClass() throws Exception { - final StorageAdapter adapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final Frame frame = - Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(adapter) + Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList()); @@ -401,8 +401,8 @@ public void testBadChecksum() throws Exception private static Frame makeGoodFrame() { - final StorageAdapter adapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - return Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(adapter) + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); + return Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList()); diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayCursorFactory.java similarity index 92% rename from processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java rename to processing/src/test/java/org/apache/druid/frame/TestArrayCursorFactory.java index 173750a74c75..2a6116bfddff 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayCursorFactory.java @@ -19,7 +19,6 @@ package org.apache.druid.frame; -import com.google.common.collect.Iterables; import org.apache.druid.query.OrderBy; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; @@ -32,7 +31,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; @@ -44,11 +43,12 @@ import java.util.List; /** - * Storage adapter around {@link QueryableIndex} that transforms all multi-value strings columns into string arrays. + * {@link org.apache.druid.segment.CursorFactory} around {@link QueryableIndex} that transforms all multi-value strings + * columns into string arrays. */ -public class TestArrayStorageAdapter extends QueryableIndexStorageAdapter +public class TestArrayCursorFactory extends QueryableIndexCursorFactory { - public TestArrayStorageAdapter(QueryableIndex index) + public TestArrayCursorFactory(QueryableIndex index) { super(index); } @@ -88,7 +88,7 @@ public RowSignature getRowSignature() final RowSignature.Builder builder = RowSignature.builder(); builder.addTimeColumn(); - for (final String column : Iterables.concat(super.getAvailableDimensions(), super.getAvailableMetrics())) { + for (final String column : super.getRowSignature().getColumnNames()) { ColumnCapabilities columnCapabilities = super.getColumnCapabilities(column); ColumnType columnType = columnCapabilities == null ? null : columnCapabilities.toColumnType(); //change MV strings columns to Array @@ -188,7 +188,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public ColumnCapabilities getColumnCapabilities(String column) { - return TestArrayStorageAdapter.this.getColumnCapabilities(column); + return TestArrayCursorFactory.this.getColumnCapabilities(column); } }; } diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java index 18168a1776ed..32faac85276b 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java @@ -30,7 +30,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -124,11 +124,11 @@ public void testTruncatedFrameFile() throws IOException final int allocatorSize = 64000; final int truncatedSize = 30000; // Holds two full columnar frames + one partial frame, after compression. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(allocatorSize))) .frameType(FrameType.COLUMNAR) // No particular reason to test with both frame types .frames(), @@ -171,11 +171,11 @@ public void testSetError() throws IOException final int allocatorSize = 64000; final int errorAtBytePosition = 30000; // Holds two full frames + one partial frame, after compression. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(allocatorSize))) .frameType(FrameType.COLUMNAR) // No particular reason to test with both frame types .frames(), @@ -239,11 +239,11 @@ public static Iterable constructorFeeder() public void testWriteFullyThenRead() throws IOException { // Create a frame file. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .frameType(frameType) .frames(), @@ -285,8 +285,8 @@ public void testWriteFullyThenRead() throws IOException } FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); } @@ -294,11 +294,11 @@ public void testWriteFullyThenRead() throws IOException public void testWriteReadInterleaved() throws IOException { // Create a frame file. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .frameType(frameType) .frames(), @@ -364,8 +364,8 @@ public void testWriteReadInterleaved() throws IOException } FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(outChannel.readable(), FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(outChannel.readable(), FrameReader.create(cursorFactory.getRowSignature())) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java index e86a7d634d0c..c2958c4c10c9 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java @@ -24,7 +24,7 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Test; @@ -37,9 +37,9 @@ public class ReadableConcatFrameChannelTest extends InitializedNullHandlingTest @Test public void testChannel() throws Exception { - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); final List frames = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(11) .frames() @@ -58,8 +58,8 @@ public void testChannel() throws Exception final ReadableConcatFrameChannel concatChannel = ReadableConcatFrameChannel.open(channels.iterator()); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(concatChannel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(concatChannel, FrameReader.create(cursorFactory.getRowSignature())) ); } } diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java index b3c0ec205aab..ff9952216c3a 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java @@ -29,7 +29,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -59,8 +59,8 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling @Rule public ExpectedException expectedException = ExpectedException.none(); - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); ExecutorService executorService = Execs.singleThreaded("input-stream-fetcher-test"); @@ -76,10 +76,10 @@ public void testSimpleFrameFile() ); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ) ); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); @@ -101,7 +101,7 @@ public void testEmptyFrameFile() throws IOException Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size(), 0); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); readableInputStreamFrameChannel.close(); @@ -127,7 +127,7 @@ public void testZeroBytesFrameFile() throws IOException () -> FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList() ); @@ -143,11 +143,11 @@ public void testTruncatedFrameFile() throws IOException final int allocatorSize = 64000; final int truncatedSize = 30000; // Holds two full frames + one partial frame, after compression. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(allocatorSize))) .frameType(FrameType.ROW_BASED) .frames(), @@ -173,7 +173,7 @@ public void testTruncatedFrameFile() throws IOException Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size(), 0); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); readableInputStreamFrameChannel.close(); @@ -199,7 +199,7 @@ public void testIncorrectFrameFile() throws IOException Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size(), 0); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); readableInputStreamFrameChannel.close(); @@ -222,10 +222,10 @@ public void closeInputStreamWhileReading() throws IOException expectedException.expect(ISE.class); expectedException.expectMessage("Found error while reading input stream"); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ) ); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); @@ -260,7 +260,7 @@ private InputStream getInputStream() { try { final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter).maxRowsPerFrame(10).frameType(FrameType.ROW_BASED).frames(), + FrameSequenceBuilder.fromCursorFactory(cursorFactory).maxRowsPerFrame(10).frameType(FrameType.ROW_BASED).frames(), temporaryFolder.newFile() ); return Files.newInputStream(file.toPath()); diff --git a/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java b/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java index 70ff7037b8cb..601739b7fd5f 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java @@ -26,8 +26,9 @@ import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.concrete.RowBasedFrameRowsAndColumns; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.BaseColumn; @@ -49,10 +50,10 @@ public class FieldReaderRACTest extends InitializedNullHandlingTest public void testDataSet() throws IOException { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); - final Frame frame = FrameTestUtil.adapterToFrame(storageAdapter, FrameType.ROW_BASED); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + final Frame frame = FrameTestUtil.cursorFactoryToFrame(cursorFactory, FrameType.ROW_BASED); - final RowSignature siggy = storageAdapter.getRowSignature(); + final RowSignature siggy = cursorFactory.getRowSignature(); final RowBasedFrameRowsAndColumns rowBasedRAC = new RowBasedFrameRowsAndColumns(frame, siggy); for (String columnName : siggy.getColumnNames()) { diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java index 06c160e68409..7b0c5f5407c7 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java @@ -31,9 +31,9 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.http.client.response.ClientResponse; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -70,7 +70,7 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes private final int maxRowsPerFrame; - private StorageAdapter adapter; + private CursorFactory cursorFactory; private File file; private ReadableByteChunksFrameChannel channel; private FrameFileHttpResponseHandler handler; @@ -95,9 +95,9 @@ public static Iterable constructorFeeder() @Before public void setUp() throws IOException { - adapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + cursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types .frames(), @@ -134,8 +134,8 @@ public void testNonChunkedResponse() throws Exception channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); // Backpressure future resolves once channel is read. @@ -230,8 +230,8 @@ public void testChunkedResponse() throws Exception channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); // Backpressure future resolves after channel is read. @@ -341,8 +341,8 @@ public void testCaughtExceptionDuringChunkedResponse() throws Exception channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); } @@ -421,8 +421,8 @@ public void testCaughtExceptionDuringChunkedResponseRetryWithSameHandler() throw Assert.assertEquals(allBytes.length, channel.getBytesAdded()); channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java index 79cbe74ba28f..9c92eb14e3e5 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java @@ -23,24 +23,21 @@ import it.unimi.dsi.fastutil.ints.IntObjectPair; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; -import org.apache.druid.frame.TestArrayStorageAdapter; +import org.apache.druid.frame.TestArrayCursorFactory; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.RowAdapters; -import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; -import org.apache.druid.timeline.SegmentId; import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Assume; @@ -73,39 +70,59 @@ enum AdapterType { INCREMENTAL { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() { - return new IncrementalIndexStorageAdapter(TestIndex.getNoRollupIncrementalTestIndex()); + return new IncrementalIndexCursorFactory(TestIndex.getNoRollupIncrementalTestIndex()); + } + + @Override + int getRowCount() + { + return TestIndex.getNoRollupIncrementalTestIndex().size(); } }, MMAP { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() + { + return new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); + } + + @Override + int getRowCount() { - return new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + return TestIndex.getNoRollupMMappedTestIndex().getNumRows(); } }, MV_AS_STRING_ARRAYS { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() { - return new TestArrayStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + return new TestArrayCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); + } + + @Override + int getRowCount() + { + return TestIndex.getNoRollupMMappedTestIndex().getNumRows(); } }, EMPTY { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() + { + return new RowBasedCursorFactory<>(Sequences.empty(), RowAdapters.standardRow(), RowSignature.empty()); + } + + @Override + int getRowCount() { - return new RowBasedSegment<>( - SegmentId.dummy("EMPTY"), - Sequences.empty(), - RowAdapters.standardRow(), - RowSignature.empty() - ).asStorageAdapter(); + return 0; } }; - abstract StorageAdapter getAdapter(); + abstract CursorFactory getCursorFactory(); + abstract int getRowCount(); } @Rule @@ -120,7 +137,8 @@ StorageAdapter getAdapter() private final AdapterType adapterType; private final int maxMmapSize; - private StorageAdapter adapter; + private CursorFactory cursorFactory; + private int rowCount; private File file; public FrameFileTest( @@ -175,12 +193,13 @@ public static Iterable constructorFeeder() @Before public void setUp() throws IOException { - adapter = adapterType.getAdapter(); + cursorFactory = adapterType.getCursorFactory(); + rowCount = adapterType.getRowCount(); if (partitioned) { // Partition every PARTITION_SIZE rows. file = FrameTestUtil.writeFrameFileWithPartitions( - FrameSequenceBuilder.fromAdapter(adapter).frameType(frameType).maxRowsPerFrame(maxRowsPerFrame).frames().map( + FrameSequenceBuilder.fromCursorFactory(cursorFactory).frameType(frameType).maxRowsPerFrame(maxRowsPerFrame).frames().map( new Function>() { private int rows = 0; @@ -202,7 +221,7 @@ public IntObjectPair apply(final Frame frame) } else { file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter).frameType(frameType).maxRowsPerFrame(maxRowsPerFrame).frames(), + FrameSequenceBuilder.fromCursorFactory(cursorFactory).frameType(frameType).maxRowsPerFrame(maxRowsPerFrame).frames(), temporaryFolder.newFile() ); } @@ -232,7 +251,7 @@ public void test_frame_first() throws IOException Assume.assumeThat(frameFile.numFrames(), Matchers.greaterThan(0)); final Frame firstFrame = frameFile.frame(0); - Assert.assertEquals(Math.min(adapter.getNumRows(), maxRowsPerFrame), firstFrame.numRows()); + Assert.assertEquals(Math.min(rowCount, maxRowsPerFrame), firstFrame.numRows()); } } @@ -245,9 +264,9 @@ public void test_frame_last() throws IOException final Frame lastFrame = frameFile.frame(frameFile.numFrames() - 1); Assert.assertEquals( - adapter.getNumRows() % maxRowsPerFrame != 0 - ? adapter.getNumRows() % maxRowsPerFrame - : Math.min(adapter.getNumRows(), maxRowsPerFrame), + rowCount % maxRowsPerFrame != 0 + ? rowCount % maxRowsPerFrame + : Math.min(rowCount, maxRowsPerFrame), lastFrame.numRows() ); } @@ -274,20 +293,20 @@ public void test_frame_outOfBoundsTooLarge() throws IOException } @Test - public void test_frame_readAllDataViaStorageAdapter() throws IOException + public void test_frame_readAllDataViaCursorFactory() throws IOException { - final FrameReader frameReader = FrameReader.create(adapter.getRowSignature()); + final FrameReader frameReader = FrameReader.create(cursorFactory.getRowSignature()); try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { final Sequence> frameFileRows = Sequences.concat( () -> IntStream.range(0, frameFile.numFrames()) .mapToObj(frameFile::frame) - .map(frame -> new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)) - .map(adapter -> FrameTestUtil.readRowsFromAdapter(adapter, null, true)) + .map(frameReader::makeCursorFactory) + .map(FrameTestUtil::readRowsFromCursorFactoryWithRowNumber) .iterator() ); - final Sequence> adapterRows = FrameTestUtil.readRowsFromAdapter(adapter, null, true); + final Sequence> adapterRows = FrameTestUtil.readRowsFromCursorFactoryWithRowNumber(cursorFactory); FrameTestUtil.assertRowsEqual(adapterRows, frameFileRows); } } @@ -373,7 +392,7 @@ public void test_newReference() throws IOException private int computeExpectedNumFrames() { - return IntMath.divide(countRows(adapter), maxRowsPerFrame, RoundingMode.CEILING); + return IntMath.divide(countRows(cursorFactory), maxRowsPerFrame, RoundingMode.CEILING); } private int computeExpectedNumPartitions() @@ -381,7 +400,7 @@ private int computeExpectedNumPartitions() if (partitioned) { return Math.min( computeExpectedNumFrames(), - IntMath.divide(countRows(adapter), PARTITION_SIZE, RoundingMode.CEILING) + IntMath.divide(countRows(cursorFactory), PARTITION_SIZE, RoundingMode.CEILING) ); } else { // 0 = not partitioned. @@ -389,10 +408,10 @@ private int computeExpectedNumPartitions() } } - private static int countRows(final StorageAdapter adapter) + private static int countRows(final CursorFactory cursorFactory) { - // Not using adapter.getNumRows(), because RowBasedStorageAdapter doesn't support it. - return FrameTestUtil.readRowsFromAdapter(adapter, RowSignature.empty(), false) + // Not using adapter.getNumRows(), because RowBasedCursorFactory doesn't support it. + return FrameTestUtil.readRowsFromCursorFactory(cursorFactory, RowSignature.empty(), false) .accumulate(0, (i, in) -> i + 1); } } diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java index 972371d0fde5..58b3a4e09204 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java @@ -26,7 +26,7 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -49,7 +49,7 @@ public class FrameFileWriterTest extends InitializedNullHandlingTest @Test public void test_abort_afterAllFrames() throws IOException { - final Sequence frames = FrameSequenceBuilder.fromAdapter(new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex())) + final Sequence frames = FrameSequenceBuilder.fromCursorFactory(new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex())) .allocator(ArenaMemoryAllocator.createOnHeap(1000000)) .frameType(FrameType.ROW_BASED) .frames(); diff --git a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java index c53ca04a9096..9e0547cae432 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java @@ -26,8 +26,8 @@ import org.apache.druid.frame.field.FieldReaders; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -50,7 +50,7 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest @Before public void setUp() { - final StorageAdapter rowBasedAdapterWithoutComplexColumn = new RowBasedSegment<>( + final CursorFactory rowBasedAdapterWithoutComplexColumn = new RowBasedSegment<>( SegmentId.dummy("test"), Sequences.simple(ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN), columnName -> { @@ -62,16 +62,16 @@ public void setUp() } }, ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE - ).asStorageAdapter(); + ).asCursorFactory(); frameWithoutComplexColumns = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(rowBasedAdapterWithoutComplexColumn) + FrameSequenceBuilder.fromCursorFactory(rowBasedAdapterWithoutComplexColumn) .frameType(FrameType.ROW_BASED) .frames() .toList() ); - final StorageAdapter rowBasedAdapterWithComplexColumn = new RowBasedSegment<>( + final CursorFactory rowBasedAdapterWithComplexColumn = new RowBasedSegment<>( SegmentId.dummy("test"), Sequences.simple(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS), columnName -> { @@ -83,10 +83,10 @@ public void setUp() } }, ByteRowKeyComparatorTest.SIGNATURE - ).asStorageAdapter(); + ).asCursorFactory(); frameWithComplexColumns = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(rowBasedAdapterWithComplexColumn) + FrameSequenceBuilder.fromCursorFactory(rowBasedAdapterWithComplexColumn) .frameType(FrameType.ROW_BASED) .frames() .toList() diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java index 5600389f57b1..f594b73368c4 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java @@ -93,7 +93,7 @@ public void test_openChannel3() throws IOException, ExecutionException, Interrup writableFrameChannel.write(new FrameWithPartition(frame, 1)); writableFrameChannel.close(); - verifySingleFrameReadableChannel(channel.getReadableChannel(), sourceAdapter); + verifySingleFrameReadableChannel(channel.getReadableChannel(), sourceCursorFactory); } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java index 55e5f7bb81c3..0f50624078b4 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java @@ -45,10 +45,11 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.utils.CloseableUtils; import org.hamcrest.CoreMatchers; @@ -116,9 +117,9 @@ public void test_runFully() throws Exception { // 3 input files blasted to 2 outputs (2 copies of the data), then muxed to one file. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - final List inFiles = writeToNFiles(adapter, 3); + final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + final List inFiles = writeToNFiles(cursorFactory, 3); final File outFile = temporaryFolder.newFile(); final BlockingQueueFrameChannel memoryChannel1 = BlockingQueueFrameChannel.minimal(); @@ -144,14 +145,14 @@ public void test_runFully() throws Exception final ListenableFuture blasterFuture = exec.runFully(blaster, null); final ListenableFuture muxerFuture = exec.runFully(muxer, null); - Assert.assertEquals(adapter.getNumRows(), (long) blasterFuture.get()); - Assert.assertEquals(adapter.getNumRows() * 2, (long) muxerFuture.get()); + Assert.assertEquals(index.size(), (long) blasterFuture.get()); + Assert.assertEquals(index.size() * 2, (long) muxerFuture.get()); Assert.assertEquals( - adapter.getNumRows() * 2, + index.size() * 2, FrameTestUtil.readRowsFromFrameChannel( new ReadableFileFrameChannel(FrameFile.open(outFile, null)), - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size() ); } @@ -180,9 +181,9 @@ public static Collection constructorFeeder() @Test public void test_runFully_errors() throws Exception { - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - final File inFile = Iterables.getOnlyElement(writeToNFiles(adapter, 1)); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); + final File inFile = Iterables.getOnlyElement(writeToNFiles(cursorFactory, 1)); final ReadableFrameChannel inChannel = openFileChannel(inFile); final BlockingQueueFrameChannel outChannel = BlockingQueueFrameChannel.minimal(); @@ -274,7 +275,7 @@ public void test_cancel_concurrency() throws Exception // Doesn't matter what's in this frame. final Frame frame = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex())) + FrameSequenceBuilder.fromCursorFactory(new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex())) .frameType(FrameType.ROW_BASED) .frames() .toList() @@ -427,7 +428,7 @@ public void tearDown() throws Exception } } - List writeToNFiles(final StorageAdapter adapter, final int numFiles) throws IOException + List writeToNFiles(final CursorFactory cursorFactory, final int numFiles) throws IOException { final List files = new ArrayList<>(); final List writers = new ArrayList<>(); @@ -465,7 +466,7 @@ public void accept(Frame frame) }; FrameSequenceBuilder - .fromAdapter(adapter) + .fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .allocator(ArenaMemoryAllocator.createOnHeap(1_000_000)) .maxRowsPerFrame(3) 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 1e65ed7142a8..22ad907bec60 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 @@ -27,15 +27,13 @@ import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -49,15 +47,15 @@ public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTe { private final OutputChannelFactory outputChannelFactory; private final long frameSize; - protected final StorageAdapter sourceAdapter; + protected final CursorFactory sourceCursorFactory; protected final Frame frame; public OutputChannelFactoryTest(OutputChannelFactory outputChannelFactory, long frameSize) { this.outputChannelFactory = outputChannelFactory; this.frameSize = frameSize; - this.sourceAdapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - this.frame = Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(sourceAdapter) + this.sourceCursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); + this.frame = Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(sourceCursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList()); @@ -79,7 +77,7 @@ public void test_openChannel() throws IOException, ExecutionException, Interrupt // read back data from the channel verifySingleFrameReadableChannel( channel.getReadableChannel(), - sourceAdapter + sourceCursorFactory ); Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); } @@ -103,7 +101,7 @@ public void test_openPartitionedChannel() throws IOException, ExecutionException for (int partition : partitions) { verifySingleFrameReadableChannel( partitionedReadableFrameChannelSupplier.get().getReadableFrameChannel(partition), - sourceAdapter + sourceCursorFactory ); Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); } @@ -111,7 +109,7 @@ public void test_openPartitionedChannel() throws IOException, ExecutionException protected void verifySingleFrameReadableChannel( ReadableFrameChannel readableFrameChannel, - StorageAdapter adapter + CursorFactory cursorFactory ) throws ExecutionException, InterruptedException { readableFrameChannel.readabilityFuture().get(); @@ -128,22 +126,18 @@ protected void verifySingleFrameReadableChannel( Assert.assertTrue(readableFrameChannel.isFinished()); readableFrameChannel.close(); - FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( - readbackFrame, - FrameReader.create(adapter.getRowSignature()), - Intervals.ETERNITY - ); + CursorFactory frameCursorFactory = FrameReader.create(cursorFactory.getRowSignature()).makeCursorFactory(readbackFrame); // build list of rows from written and read data to verify - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN); - final CursorHolder frameMaker = frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN) + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN); + final CursorHolder frameHolder = frameCursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN) ) { List> writtenData = FrameTestUtil.readRowsFromCursor( cursorHolder.asCursor(), - adapter.getRowSignature() + cursorFactory.getRowSignature() ).toList(); List> readData = FrameTestUtil.readRowsFromCursor( - frameMaker.asCursor(), - frameStorageAdapter.getRowSignature() + frameHolder.asCursor(), + frameCursorFactory.getRowSignature() ).toList(); Assert.assertEquals( diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java index 9025d1820864..88bb7cb07e8c 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java @@ -26,8 +26,8 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -55,16 +55,16 @@ public class ReadableFileFrameChannelTest extends InitializedNullHandlingTest @Before public void setUp() throws IOException { - final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(ROWS_PER_FRAME) .frames(), temporaryFolder.newFile() ); - allRows = FrameTestUtil.readRowsFromAdapter(adapter, adapter.getRowSignature(), false).toList(); - frameReader = FrameReader.create(adapter.getRowSignature()); + allRows = FrameTestUtil.readRowsFromCursorFactory(cursorFactory).toList(); + frameReader = FrameReader.create(cursorFactory.getRowSignature()); frameFile = FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE); } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java b/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java index b82aed94ef81..7cd1e980428e 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java @@ -41,7 +41,7 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -160,7 +160,7 @@ public void test_runAllFully_fiftyThousandProcessors() throws Exception // Doesn't matter what's in this frame. final Frame frame = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex())) + FrameSequenceBuilder.fromCursorFactory(new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex())) .frameType(FrameType.ROW_BASED) .frames() .toList() diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index b40fde04a9a9..2149d6cbf1c7 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -53,8 +53,8 @@ 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.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; @@ -241,7 +241,7 @@ public static class ParameterizedCasesTest extends InitializedNullHandlingTest private final boolean partitionsDeferred; private final long limitHint; - private StorageAdapter adapter; + private CursorFactory cursorFactory; private RowSignature signature; private FrameProcessorExecutor exec; private List inputChannels; @@ -326,7 +326,7 @@ public void setUp() exec = new FrameProcessorExecutor( MoreExecutors.listeningDecorator(Execs.multiThreaded(numThreads, getClass().getSimpleName() + "[%d]")) ); - adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); } @After @@ -352,7 +352,7 @@ private void setUpInputChannels(final ClusterBy clusterBy) throws Exception } final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .sortBy(clusterBy.getColumns()) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(maxBytesPerFrame))) @@ -465,7 +465,7 @@ private void verifySuperSorter( } final Sequence> expectedRows = Sequences.sort( - FrameTestUtil.readRowsFromAdapter(adapter, signature, true), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, true), Comparator.comparing( row -> { final Object[] array = new Object[clusterByPartColumns.length]; diff --git a/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java b/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java index 0a25993a49ad..ae748198aea3 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java @@ -28,11 +28,11 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -44,7 +44,7 @@ private TestFrameProcessorUtils() { } - public static StorageAdapter toStorageAdapter(List inputRows) + public static CursorFactory toCursorFactory(List inputRows) { final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( @@ -70,13 +70,13 @@ public static StorageAdapter toStorageAdapter(List inputRows) throw new RuntimeException(e); } - return new IncrementalIndexStorageAdapter(index); + return new IncrementalIndexCursorFactory(index); } public static Frame toFrame(List inputRows) { - final StorageAdapter storageAdapter = toStorageAdapter(inputRows); - return Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(storageAdapter) + final CursorFactory cursorFactory = toCursorFactory(inputRows); + return Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .frames() .toList()); diff --git a/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java b/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java index bf24b6cb5f02..a5c15a7121c8 100644 --- a/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java @@ -24,8 +24,8 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.HeapMemoryAllocator; import org.apache.druid.frame.testutil.FrameSequenceBuilder; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -42,7 +42,7 @@ public class FrameReaderTest extends InitializedNullHandlingTest { private final FrameType frameType; - private StorageAdapter inputAdapter; + private CursorFactory inputCursorFactory; private Frame frame; private FrameReader frameReader; @@ -66,10 +66,10 @@ public static Iterable constructorFeeder() @Before public void setUp() { - inputAdapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + inputCursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(inputAdapter) + FrameSequenceBuilder.fromCursorFactory(inputCursorFactory) .frameType(frameType) .allocator(HeapMemoryAllocator.unlimited()); @@ -80,16 +80,16 @@ public void setUp() @Test public void testSignature() { - Assert.assertEquals(inputAdapter.getRowSignature(), frameReader.signature()); + Assert.assertEquals(inputCursorFactory.getRowSignature(), frameReader.signature()); } @Test public void testColumnCapabilitiesToColumnType() { - for (final String columnName : inputAdapter.getRowSignature().getColumnNames()) { + for (final String columnName : inputCursorFactory.getRowSignature().getColumnNames()) { Assert.assertEquals( columnName, - inputAdapter.getRowSignature().getColumnCapabilities(columnName).toColumnType(), + inputCursorFactory.getRowSignature().getColumnCapabilities(columnName).toColumnType(), frameReader.columnCapabilities(frame, columnName).toColumnType() ); } diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java similarity index 77% rename from processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java index 2a8f525bf781..08978e20b0c1 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java @@ -37,11 +37,10 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -70,7 +69,7 @@ import java.util.List; import java.util.function.Function; -public class FrameStorageAdapterTest +public class FrameCursorFactoryTest { /** * Basic tests: everything except makeCursor, makeVectorCursor. @@ -80,9 +79,9 @@ public static class BasicTests extends InitializedNullHandlingTest { private final FrameType frameType; - private StorageAdapter queryableAdapter; + private CursorFactory queryableCursorFactory; private FrameSegment frameSegment; - private StorageAdapter frameAdapter; + private CursorFactory frameCursorFactory; public BasicTests(final FrameType frameType) { @@ -105,9 +104,9 @@ public static Iterable constructorFeeder() public void setUp() { - queryableAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); - frameSegment = FrameTestUtil.adapterToFrameSegment(queryableAdapter, frameType); - frameAdapter = frameSegment.asStorageAdapter(); + queryableCursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); + frameSegment = FrameTestUtil.cursorFactoryToFrameSegment(queryableCursorFactory, frameType); + frameCursorFactory = frameSegment.asCursorFactory(); } @After @@ -118,62 +117,18 @@ public void tearDown() } } - @Test - public void test_getInterval() - { - Assert.assertEquals(queryableAdapter.getInterval(), frameAdapter.getInterval()); - } - @Test public void test_getRowSignature() { - Assert.assertEquals(queryableAdapter.getRowSignature(), frameAdapter.getRowSignature()); - } - - @Test - public void test_getAvailableDimensions() - { - // All columns are dimensions to the frameAdapter. - Assert.assertEquals( - queryableAdapter.getRowSignature().getColumnNames(), - ImmutableList.copyOf(frameAdapter.getAvailableDimensions()) - ); - } - - @Test - public void test_getAvailableMetrics() - { - // All columns are dimensions to the frameAdapter. - Assert.assertEquals(Collections.emptyList(), frameAdapter.getAvailableMetrics()); - } - - @Test - public void test_getDimensionCardinality_knownColumns() - { - for (final String columnName : frameAdapter.getRowSignature().getColumnNames()) { - Assert.assertEquals( - columnName, - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - frameAdapter.getDimensionCardinality(columnName) - ); - } - } - - @Test - public void test_getDimensionCardinality_unknownColumn() - { - Assert.assertEquals( - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - frameAdapter.getDimensionCardinality("nonexistent") - ); + Assert.assertEquals(queryableCursorFactory.getRowSignature(), frameCursorFactory.getRowSignature()); } @Test public void test_getColumnCapabilities_typeOfKnownColumns() { - for (final String columnName : frameAdapter.getRowSignature().getColumnNames()) { - final ColumnCapabilities expectedCapabilities = queryableAdapter.getColumnCapabilities(columnName); - final ColumnCapabilities actualCapabilities = frameAdapter.getColumnCapabilities(columnName); + for (final String columnName : frameCursorFactory.getRowSignature().getColumnNames()) { + final ColumnCapabilities expectedCapabilities = queryableCursorFactory.getColumnCapabilities(columnName); + final ColumnCapabilities actualCapabilities = frameCursorFactory.getColumnCapabilities(columnName); Assert.assertEquals( StringUtils.format("column [%s] type", columnName), @@ -204,19 +159,7 @@ public void test_getColumnCapabilities_typeOfKnownColumns() @Test public void test_getColumnCapabilities_unknownColumn() { - Assert.assertNull(frameAdapter.getColumnCapabilities("nonexistent")); - } - - @Test - public void test_getNumRows() - { - Assert.assertEquals(queryableAdapter.getNumRows(), frameAdapter.getNumRows()); - } - - @Test - public void test_getMetadata() - { - Assert.assertNull(frameAdapter.getMetadata()); + Assert.assertNull(frameCursorFactory.getColumnCapabilities("nonexistent")); } } @@ -235,9 +178,9 @@ public static class CursorTests extends InitializedNullHandlingTest private final VirtualColumns virtualColumns; private final boolean descending; - private StorageAdapter queryableAdapter; + private CursorFactory queryableCursorFactory; private FrameSegment frameSegment; - private StorageAdapter frameAdapter; + private CursorFactory frameCursorFactory; private final QueryContext queryContext = QueryContext.of( ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, VECTOR_SIZE) ); @@ -338,9 +281,9 @@ public static Iterable constructorFeeder() @Before public void setUp() { - queryableAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); - frameSegment = FrameTestUtil.adapterToFrameSegment(queryableAdapter, frameType); - frameAdapter = frameSegment.asStorageAdapter(); + queryableCursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); + frameSegment = FrameTestUtil.cursorFactoryToFrameSegment(queryableCursorFactory, frameType); + frameCursorFactory = frameSegment.asCursorFactory(); } @After @@ -354,7 +297,7 @@ public void tearDown() @Test public void test_makeCursor() { - final RowSignature signature = frameAdapter.getRowSignature(); + final RowSignature signature = frameCursorFactory.getRowSignature(); // Frame adapters don't know the order of the underlying frames, so they should ignore the "preferred ordering" // of the cursor build spec. We test this by passing the frameAdapter a build spec with a preferred ordering, @@ -362,8 +305,8 @@ public void test_makeCursor() final CursorBuildSpec queryableBuildSpec = CursorBuildSpec.builder(buildSpec).setPreferredOrdering(Collections.emptyList()).build(); - try (final CursorHolder queryableCursorHolder = queryableAdapter.makeCursorHolder(queryableBuildSpec); - final CursorHolder frameCursorHolder = frameAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder queryableCursorHolder = queryableCursorFactory.makeCursorHolder(queryableBuildSpec); + final CursorHolder frameCursorHolder = frameCursorFactory.makeCursorHolder(buildSpec)) { final Sequence> queryableRows = FrameTestUtil.readRowsFromCursor(advanceAndReset(queryableCursorHolder.asCursor()), signature); final Sequence> frameRows = @@ -378,18 +321,18 @@ public void test_makeVectorCursor() // Conditions for frames to be vectorizable. Assume.assumeThat(frameType, CoreMatchers.equalTo(FrameType.COLUMNAR)); Assume.assumeFalse(descending); - assertVectorCursorsMatch(adapter -> adapter.makeCursorHolder(buildSpec)); + assertVectorCursorsMatch(cursorFactory -> cursorFactory.makeCursorHolder(buildSpec)); } - private void assertVectorCursorsMatch(final Function call) + private void assertVectorCursorsMatch(final Function call) { - final CursorHolder cursorHolder = call.apply(queryableAdapter); - final CursorHolder frameCursorHolder = call.apply(frameAdapter); + final CursorHolder cursorHolder = call.apply(queryableCursorFactory); + final CursorHolder frameCursorHolder = call.apply(frameCursorFactory); Assert.assertTrue("queryable cursor is vectorizable", cursorHolder.canVectorize()); Assert.assertTrue("frame cursor is vectorizable", frameCursorHolder.canVectorize()); - final RowSignature signature = frameAdapter.getRowSignature(); + final RowSignature signature = frameCursorFactory.getRowSignature(); final Sequence> queryableRows = FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(cursorHolder.asVectorCursor()), signature) .withBaggage(cursorHolder); diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java index eb8162251967..d3fbffd7b4af 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java @@ -34,8 +34,8 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -49,7 +49,7 @@ */ public class FrameSequenceBuilder { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; private FrameType frameType = null; private MemoryAllocator allocator = HeapMemoryAllocator.unlimited(); @@ -57,14 +57,14 @@ public class FrameSequenceBuilder private int maxRowsPerFrame = Integer.MAX_VALUE; private boolean populateRowNumber = false; - private FrameSequenceBuilder(StorageAdapter adapter) + private FrameSequenceBuilder(CursorFactory cursorFactory) { - this.adapter = adapter; + this.cursorFactory = cursorFactory; } - public static FrameSequenceBuilder fromAdapter(final StorageAdapter adapter) + public static FrameSequenceBuilder fromCursorFactory(final CursorFactory cursorFactory) { - return new FrameSequenceBuilder(adapter); + return new FrameSequenceBuilder(cursorFactory); } public FrameSequenceBuilder frameType(final FrameType frameType) @@ -109,11 +109,11 @@ public RowSignature signature() if (populateRowNumber) { baseSignature = RowSignature.builder() - .addAll(adapter.getRowSignature()) + .addAll(cursorFactory.getRowSignature()) .add(FrameTestUtil.ROW_NUMBER_COLUMN, ColumnType.LONG) .build(); } else { - baseSignature = adapter.getRowSignature(); + baseSignature = cursorFactory.getRowSignature(); } return FrameWriters.sortableSignature(baseSignature, keyColumns); @@ -139,7 +139,7 @@ public Sequence frames() throw DruidException.defensive("Unrecognized frame type"); } - final CursorHolder cursorHolder = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber); + final CursorHolder cursorHolder = FrameTestUtil.makeCursorForCursorFactory(cursorFactory, populateRowNumber); final Cursor cursor = cursorHolder.asCursor(); return new BaseSequence<>( new BaseSequence.IteratorMaker>() 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 e1981f3618b0..c75a57a86990 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 @@ -30,7 +30,6 @@ import org.apache.druid.frame.file.FrameFileWriter; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameSegment; -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.guava.Sequence; @@ -42,9 +41,9 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -182,10 +181,10 @@ public static void assertRowEqual(final String message, final List expec } } - public static Frame adapterToFrame(final StorageAdapter adapter, final FrameType frameType) + public static Frame cursorFactoryToFrame(final CursorFactory adapter, final FrameType frameType) { return Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(adapter) .allocator(HeapMemoryAllocator.unlimited()) .frameType(frameType) .frames() @@ -193,34 +192,34 @@ public static Frame adapterToFrame(final StorageAdapter adapter, final FrameType ); } - public static FrameSegment adapterToFrameSegment( - final StorageAdapter adapter, + public static FrameSegment cursorFactoryToFrameSegment( + final CursorFactory adapter, final FrameType frameType ) { return new FrameSegment( - adapterToFrame(adapter, frameType), + cursorFactoryToFrame(adapter, frameType), FrameReader.create(adapter.getRowSignature()), - SegmentId.of("TestFrame", adapter.getInterval(), "0", 0) + SegmentId.of("TestFrame", Intervals.ETERNITY, "0", 0) ); } - public static FrameSegment adapterToFrameSegment( - final StorageAdapter adapter, + public static FrameSegment cursorFactoryToFrameSegment( + final CursorFactory cursorFactory, final FrameType frameType, final SegmentId segmentId ) { return new FrameSegment( - adapterToFrame(adapter, frameType), - FrameReader.create(adapter.getRowSignature()), + cursorFactoryToFrame(cursorFactory, frameType), + FrameReader.create(cursorFactory.getRowSignature()), segmentId ); } /** * Reads a sequence of rows from a frame channel using a non-vectorized cursor from - * {@link FrameStorageAdapter#makeCursorHolder(CursorBuildSpec)}. + * {@link FrameReader#makeCursorFactory(Frame)}. * * @param channel the channel * @param frameReader reader for this channel @@ -233,8 +232,8 @@ public static Sequence> readRowsFromFrameChannel( return new FrameChannelSequence(channel) .flatMap( frame -> { - final CursorHolder cursorHolder = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursorHolder(CursorBuildSpec.FULL_SCAN); + final CursorHolder cursorHolder = frameReader.makeCursorFactory(frame) + .makeCursorHolder(CursorBuildSpec.FULL_SCAN); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); @@ -245,23 +244,24 @@ public static Sequence> readRowsFromFrameChannel( } /** - * Reads a sequence of rows from a storage adapter. + * Reads a sequence of rows from a {@link CursorFactory}. * * If {@param populateRowNumberIfPresent} is set, and the provided signature contains {@link #ROW_NUMBER_COLUMN}, - * then that column will be populated with a row number from the adapter. + * then that column will be populated with a row number from the cursor. * - * @param adapter the adapter - * @param signature optional signature for returned rows; will use {@code adapter.rowSignature()} if null + * @param cursorFactory the cursor factory + * @param signature optional signature for returned rows; will use {@code cursorFactory.rowSignature()} if + * null * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static Sequence> readRowsFromAdapter( - final StorageAdapter adapter, + public static Sequence> readRowsFromCursorFactory( + final CursorFactory cursorFactory, @Nullable final RowSignature signature, final boolean populateRowNumber ) { - final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature; - final CursorHolder cursorHolder = makeCursorForAdapter(adapter, populateRowNumber); + final RowSignature signatureToUse = signature == null ? cursorFactory.getRowSignature() : signature; + final CursorHolder cursorHolder = makeCursorForCursorFactory(cursorFactory, populateRowNumber); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); @@ -269,16 +269,26 @@ public static Sequence> readRowsFromAdapter( return readRowsFromCursor(cursor, signatureToUse).withBaggage(cursorHolder); } + public static Sequence> readRowsFromCursorFactory(CursorFactory cursorFactory) + { + return readRowsFromCursorFactory(cursorFactory, null, false); + } + + public static Sequence> readRowsFromCursorFactoryWithRowNumber(CursorFactory cursorFactory) + { + return readRowsFromCursorFactory(cursorFactory, null, true); + } + /** - * Creates a Cursor and from a storage adapter. + * Creates a {@link CursorHolder} from a {@link CursorFactory}. * * If {@param populateRowNumber} is set, the row number will be populated into {@link #ROW_NUMBER_COLUMN}. * - * @param adapter the adapter + * @param cursorFactory the cursor factory * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static CursorHolder makeCursorForAdapter( - final StorageAdapter adapter, + public static CursorHolder makeCursorForCursorFactory( + final CursorFactory cursorFactory, final boolean populateRowNumber ) { @@ -297,7 +307,7 @@ public static CursorHolder makeCursorForAdapter( .setVirtualColumns(virtualColumns) .build(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); if (populateRowNumber) { return new CursorHolder() { diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 34cdab25a48b..4c6f81a2a16c 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 @@ -24,7 +24,7 @@ import org.apache.druid.segment.Cursor; /** - * Used by {@link FrameTestUtil#readRowsFromAdapter} and {@link FrameTestUtil#readRowsFromCursor}. + * Used by {@link FrameTestUtil#readRowsFromCursorFactory} and {@link FrameTestUtil#readRowsFromCursor}. */ public class RowNumberUpdatingCursor implements Cursor { 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 9d7a9b725531..3103348284b5 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 @@ -35,10 +35,8 @@ import org.apache.druid.frame.key.RowKeyComparator; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameSegment; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -50,6 +48,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowBasedSegment; @@ -438,15 +437,11 @@ private void verifyFrame( final RowSignature signature ) { - final FrameStorageAdapter frameAdapter = new FrameStorageAdapter( - frame, - FrameReader.create(signature), - Intervals.ETERNITY - ); + final CursorFactory cursorFactory = FrameReader.create(signature).makeCursorFactory(frame); FrameTestUtil.assertRowsEqual( expectedRows, - FrameTestUtil.readRowsFromAdapter(frameAdapter, signature, false) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false) ); } @@ -577,8 +572,7 @@ private static Pair writeFrame( inputSegment = new FrameSegment(inputFrame, FrameReader.create(signature), SegmentId.dummy("xxx")); } - try (final CursorHolder cursorHolder = inputSegment.asStorageAdapter() - .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = inputSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); int numRows = 0; diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java index 75119bfb8979..863c01bb3905 100644 --- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -30,18 +30,19 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; 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.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -57,8 +58,9 @@ public class CursorGranularizerTest extends InitializedNullHandlingTest @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - private StorageAdapter adapter; + private CursorFactory cursorFactory; private TimeBoundaryInspector timeBoundaryInspector; + private Interval interval; @Before public void setup() throws IOException @@ -152,21 +154,22 @@ public void setup() throws IOException .tmpDir(temporaryFolder.newFolder()); final QueryableIndex index = bob.buildMMappedIndex(); - adapter = new QueryableIndexStorageAdapter(index); + interval = index.getDataInterval(); + cursorFactory = new QueryableIndexCursorFactory(index); timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index); } @Test public void testGranularizeFullScan() { - try (CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( cursor, timeBoundaryInspector, Order.ASCENDING, Granularities.HOUR, - adapter.getInterval() + interval ); final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); @@ -212,14 +215,14 @@ public void testGranularizeFullScanDescending() final CursorBuildSpec descending = CursorBuildSpec.builder() .setPreferredOrdering(Cursors.descendingTimeOrder()) .build(); - try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) { + try (CursorHolder cursorHolder = cursorFactory.makeCursorHolder(descending)) { final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( cursor, timeBoundaryInspector, Order.DESCENDING, Granularities.HOUR, - adapter.getInterval() + interval ); final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java index 2c44d080deb3..ee0f5c052c64 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java @@ -37,13 +37,13 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; @@ -147,7 +147,7 @@ public void testTimeseriesQuery() final Iterable> iiResults = engine.process( query, - new IncrementalIndexStorageAdapter(incrementalIndex), + new IncrementalIndexCursorFactory(incrementalIndex), new IncrementalIndexTimeBoundaryInspector(incrementalIndex), defaultTimeseriesQueryMetrics ).toList(); @@ -155,7 +155,7 @@ public void testTimeseriesQuery() final Iterable> qiResults = engine.process( query, - new QueryableIndexStorageAdapter(queryableIndex), + new QueryableIndexCursorFactory(queryableIndex), QueryableIndexTimeBoundaryInspector.create(queryableIndex), defaultTimeseriesQueryMetrics ).toList(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java index 1c9b4033cd9a..fb189685544e 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java @@ -37,13 +37,13 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; @@ -151,7 +151,7 @@ public void testTimeseriesQuery() final Iterable> iiResults = engine.process( query, - new IncrementalIndexStorageAdapter(incrementalIndex), + new IncrementalIndexCursorFactory(incrementalIndex), new IncrementalIndexTimeBoundaryInspector(incrementalIndex), defaultTimeseriesQueryMetrics ).toList(); @@ -159,7 +159,7 @@ public void testTimeseriesQuery() final Iterable> qiResults = engine.process( query, - new QueryableIndexStorageAdapter(queryableIndex), + new QueryableIndexCursorFactory(queryableIndex), QueryableIndexTimeBoundaryInspector.create(queryableIndex), defaultTimeseriesQueryMetrics ).toList(); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 53050b20ad58..461cb99b6c7f 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -13662,7 +13662,7 @@ private GroupByQuery.Builder makeQueryBuilder(final GroupByQuery query) private void assumeTimeOrdered() { try (final CursorHolder cursorHolder = - originalRunner.getSegment().asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + originalRunner.getSegment().asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) == Order.ASCENDING); } } 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 adbcbc187bb4..a0aef93381ff 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 @@ -659,7 +659,7 @@ private void runResults( boolean allCanVectorize = segments.stream() .allMatch( s -> { - final CursorHolder cursorHolder = s.asStorageAdapter() + final CursorHolder cursorHolder = s.asCursorFactory() .makeCursorHolder(spec); final boolean canVectorize = cursorHolder.canVectorize(); cursorHolder.close(); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java index 15982c2caab2..d570621eca7d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java @@ -31,10 +31,10 @@ import org.apache.druid.query.groupby.epinephelinae.VectorGrouper; import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine.VectorGroupByEngineIterator; import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -62,10 +62,10 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException .setDimensions(new DefaultDimensionSpec("market", null, null)) .setAggregatorSpecs(factory) .build(); - final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); final QueryableIndexTimeBoundaryInspector timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(TestIndex.getMMappedTestIndex()); - final CursorHolder cursorHolder = storageAdapter.makeCursorHolder( + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder( GroupingEngine.makeCursorBuildSpec(query, null) ); final ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[4096]); @@ -83,7 +83,6 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException query, new GroupByQueryConfig(), GroupByQueryRunnerTest.DEFAULT_PROCESSING_CONFIG, - storageAdapter, timeBoundaryInspector, cursor, cursorHolder.getTimeOrder(), 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 142b23b8391d..8b93bc060118 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 @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSortedMap; -import com.google.common.collect.Lists; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.extraction.MapLookupExtractor; @@ -29,14 +28,15 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.RowBasedStorageAdapter; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.timeline.SegmentId; import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -118,6 +118,7 @@ public void test_getDataInterval() @Test public void test_asQueryableIndex() { + Assert.assertNull(LOOKUP_SEGMENT.as(QueryableIndex.class)); Assert.assertNull(LOOKUP_SEGMENT.asQueryableIndex()); } @@ -129,32 +130,14 @@ public void test_asStorageAdapter_getRowSignature() .add("k", ColumnType.STRING) .add("v", ColumnType.STRING) .build(), - LOOKUP_SEGMENT.asStorageAdapter().getRowSignature() - ); - } - - @Test - public void test_asStorageAdapter_getAvailableDimensions() - { - Assert.assertEquals( - ImmutableList.of("k", "v"), - Lists.newArrayList(LOOKUP_SEGMENT.asStorageAdapter().getAvailableDimensions().iterator()) - ); - } - - @Test - public void test_asStorageAdapter_getAvailableMetrics() - { - Assert.assertEquals( - ImmutableList.of(), - Lists.newArrayList(LOOKUP_SEGMENT.asStorageAdapter().getAvailableMetrics()) + LOOKUP_SEGMENT.asCursorFactory().getRowSignature() ); } @Test public void test_asStorageAdapter_getColumnCapabilitiesK() { - final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asStorageAdapter().getColumnCapabilities("k"); + final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asCursorFactory().getColumnCapabilities("k"); Assert.assertEquals(ValueType.STRING, capabilities.getType()); @@ -168,7 +151,7 @@ public void test_asStorageAdapter_getColumnCapabilitiesK() @Test public void test_asStorageAdapter_getColumnCapabilitiesV() { - final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asStorageAdapter().getColumnCapabilities("v"); + final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asCursorFactory().getColumnCapabilities("v"); // Note: the "v" column does not actually have multiple values, but the RowBasedStorageAdapter doesn't allow // reporting complete single-valued capabilities. It would be good to change this in the future, so query engines @@ -178,37 +161,13 @@ public void test_asStorageAdapter_getColumnCapabilitiesV() Assert.assertFalse(capabilities.isDictionaryEncoded().isTrue()); } - @Test - public void test_asStorageAdapter_getInterval() - { - Assert.assertEquals(Intervals.ETERNITY, LOOKUP_SEGMENT.asStorageAdapter().getInterval()); - } - - @Test - public void test_asStorageAdapter_getDimensionCardinalityK() - { - Assert.assertEquals( - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - LOOKUP_SEGMENT.asStorageAdapter().getDimensionCardinality("k") - ); - } - - @Test - public void test_asStorageAdapter_getDimensionCardinalityV() - { - Assert.assertEquals( - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - LOOKUP_SEGMENT.asStorageAdapter().getDimensionCardinality("v") - ); - } - @Test public void test_asStorageAdapter_makeCursor() { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970/PT1H")) .build(); - try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final List> kvs = new ArrayList<>(); @@ -234,8 +193,8 @@ public void test_asStorageAdapter_makeCursor() @Test public void test_asStorageAdapter_isRowBasedAdapter() { - // This allows us to assume that RowBasedStorageAdapterTest is further exercising makeCursor and verifying misc. + // This allows us to assume that LookupSegmentTest is further exercising makeCursor and verifying misc. // methods like getMinTime, getMaxTime, getMetadata, etc, without checking them explicitly in _this_ test class. - Assert.assertThat(LOOKUP_SEGMENT.asStorageAdapter(), CoreMatchers.instanceOf(RowBasedStorageAdapter.class)); + MatcherAssert.assertThat(LOOKUP_SEGMENT.asCursorFactory(), CoreMatchers.instanceOf(RowBasedCursorFactory.class)); } } diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 8a5605bc673c..bd787a9d92a7 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -59,7 +59,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -430,9 +429,6 @@ public void testAnalysisImproperComplex() throws IOException QueryableIndex mockIndex = EasyMock.createMock(QueryableIndex.class); EasyMock.expect(mockIndex.getNumRows()).andReturn(100).atLeastOnce(); EasyMock.expect(mockIndex.getColumnNames()).andReturn(Collections.singletonList("x")).atLeastOnce(); - EasyMock.expect(mockIndex.getAvailableDimensions()) - .andReturn(new ListIndexed<>(Collections.singletonList("x"))) - .atLeastOnce(); EasyMock.expect(mockIndex.getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME)) .andReturn(ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG)) .atLeastOnce(); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumnsTest.java similarity index 69% rename from processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java rename to processing/src/test/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumnsTest.java index 4395ce9dd5b5..6aeac726e6d0 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumnsTest.java @@ -21,24 +21,24 @@ import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumnsTest; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import java.util.function.Function; -public class StorageAdapterRowsAndColumnsTest extends RowsAndColumnsTestBase +public class CursorFactoryRowsAndColumnsTest extends RowsAndColumnsTestBase { - public StorageAdapterRowsAndColumnsTest() + public CursorFactoryRowsAndColumnsTest() { - super(StorageAdapterRowsAndColumns.class); + super(CursorFactoryRowsAndColumns.class); } - public static Function MAKER = - StorageAdapterRowsAndColumnsTest::buildFrame; + public static Function MAKER = + CursorFactoryRowsAndColumnsTest::buildFrame; - private static StorageAdapterRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) + private static CursorFactoryRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) { try (ColumnBasedFrameRowsAndColumns fRAC = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input)) { - return new StorageAdapterRowsAndColumns(fRAC.as(StorageAdapter.class)); + return new CursorFactoryRowsAndColumns(fRAC.as(CursorFactory.class)); } } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java index 7b639b3d48d4..2973df31ab58 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java @@ -69,7 +69,7 @@ private static ArrayList getMakers() new Object[]{ConcatRowsAndColumns.class, ConcatRowsAndColumnsTest.MAKER}, new Object[]{RearrangedRowsAndColumns.class, RearrangedRowsAndColumnsTest.MAKER}, new Object[]{ColumnBasedFrameRowsAndColumns.class, ColumnBasedFrameRowsAndColumnsTest.MAKER}, - new Object[]{StorageAdapterRowsAndColumns.class, StorageAdapterRowsAndColumnsTest.MAKER}, + new Object[]{CursorFactoryRowsAndColumns.class, CursorFactoryRowsAndColumnsTest.MAKER}, new Object[]{RowBasedFrameRowsAndColumns.class, RowBasedFrameRowsAndColumnsTest.MAKER} ); } 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 023a6294ab9d..f90c2ea19172 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 @@ -254,7 +254,7 @@ private void validateDecorated( if (interval != null) { builder.setInterval(interval); } - try (final CursorHolder cursorHolder = seggy.asStorageAdapter().makeCursorHolder(builder.build())) { + try (final CursorHolder cursorHolder = seggy.asCursorFactory().makeCursorHolder(builder.build())) { final Cursor cursor = cursorHolder.asCursor(); vals = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java index c26508694d40..99d5dfabc89b 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java @@ -26,7 +26,7 @@ import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -64,7 +64,7 @@ public void testMaterializeVirtualColumns() final RowsAndColumns base = make(MapOfColumnsRowsAndColumns.fromRowObjects(vals, siggy)); - assumeNotNull("skipping: StorageAdapter not supported", base.as(StorageAdapter.class)); + assumeNotNull("skipping: CursorFactory not supported", base.as(CursorFactory.class)); LazilyDecoratedRowsAndColumns ras = new LazilyDecoratedRowsAndColumns( base, diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index 2bfd47fb6990..776f89cc8b5b 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -158,7 +158,7 @@ public static Iterable constructorFeeder() // Try every limit up to one past the total number of rows. final Set limits = new TreeSet<>(); - final int totalNumRows = SEGMENTS.stream().mapToInt(s -> s.asStorageAdapter().getNumRows()).sum(); + int totalNumRows = 19; for (int i = 0; i <= totalNumRows + 1; i++) { limits.add(i); } 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 727086b7be3f..c99b1f0fb08f 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 @@ -666,7 +666,7 @@ private void runResults( boolean allCanVectorize = segments.stream() .allMatch( s -> { - final CursorHolder cursorHolder = s.asStorageAdapter() + final CursorHolder cursorHolder = s.asCursorFactory() .makeCursorHolder(spec); final boolean canVectorize = cursorHolder.canVectorize(); cursorHolder.close(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java b/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java index 341a3c18078a..5bc121daa7bc 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.topn; import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.segment.StorageAdapter; import org.easymock.EasyMock; import org.junit.Test; @@ -31,14 +30,14 @@ public class PooledTopNAlgorithmTest @Test public void testCleanupWithNullParams() { - PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(EasyMock.mock(StorageAdapter.class), null, null); + PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(null, EasyMock.mock(TopNCursorInspector.class), null); pooledTopNAlgorithm.cleanup(null); } @Test public void cleanup() { - PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(EasyMock.mock(StorageAdapter.class), null, null); + PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(null, EasyMock.mock(TopNCursorInspector.class), null); PooledTopNAlgorithm.PooledTopNParams params = EasyMock.createMock(PooledTopNAlgorithm.PooledTopNParams.class); ResourceHolder resourceHolder = EasyMock.createMock(ResourceHolder.class); EasyMock.expect(params.getResultsBufHolder()).andReturn(resourceHolder).times(1); 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 ae320ac42613..c374602ecf4f 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 @@ -30,24 +30,20 @@ import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.ColumnInspector; 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.SimpleTopNOptimizationInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.Interval; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; import java.util.List; -public class TopNMetricSpecOptimizationsTest +public class TopNMetricSpecOptimizationsTest extends InitializedNullHandlingTest { private static final List AGGS = Lists.newArrayList( Iterables.concat( @@ -76,15 +72,13 @@ public void testShouldOptimizeLexicographic() .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -110,16 +104,14 @@ public void testAlsoShouldOptimizeLexicographic() .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -145,16 +137,14 @@ public void testShouldNotOptimizeLexicographic() .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-31T00:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-31T00:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -181,15 +171,13 @@ public void testAlsoShouldNotOptimizeLexicographic() .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -215,17 +203,13 @@ public void testAgainShouldNotOptimizeLexicographic() .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); - DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); Pair thePair = arrayProviderToTest.computeStartEnd(cardinality); @@ -233,85 +217,22 @@ public void testAgainShouldNotOptimizeLexicographic() Assert.assertEquals(new Integer(cardinality), thePair.rhs); } - private StorageAdapter makeFakeStorageAdapter(String start, String end, int cardinality) + private TopNCursorInspector makeCursorInspector(String start, String end, int cardinality) { - StorageAdapter adapter = new StorageAdapter() - { - @Override - public Interval getInterval() - { - return Intervals.of(start + "/" + end); - } - - @Override - public int getDimensionCardinality(String column) - { - return cardinality; - } - - // stubs below this line not important for tests - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public Metadata getMetadata() - { - return null; - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new CursorHolder() + return new TopNCursorInspector( + new ColumnInspector() { @Nullable @Override - public Cursor asCursor() + public ColumnCapabilities getColumnCapabilities(String column) { return null; } - }; - } - }; - - return adapter; + }, + new SimpleTopNOptimizationInspector(true), + Intervals.of(start + "/" + end), + cardinality + ); } private DimensionSelector makeFakeDimSelector(int cardinality) diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index c674e575d703..285ccf31a60e 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -6381,7 +6381,7 @@ public void testTopNAggregateTopnMetricFirstWithGranularity() private void assumeTimeOrdered() { try (final CursorHolder cursorHolder = - runner.getSegment().asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + runner.getSegment().asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) == Order.ASCENDING); } } 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 788181859ffa..80b6d23d4b85 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -33,8 +33,8 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.nested.StructuredData; @@ -147,8 +147,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 4, true, STRING_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -184,7 +184,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(STRING_COL).toColumnType()); + Assert.assertEquals(ColumnType.STRING, cursorFactory.getColumnCapabilities(STRING_COL).toColumnType()); } } @@ -200,8 +200,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 4, true, LONG_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -267,7 +267,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc ); Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); } - Assert.assertEquals(ColumnType.LONG, storageAdapter.getColumnCapabilities(LONG_COL).toColumnType()); + Assert.assertEquals(ColumnType.LONG, cursorFactory.getColumnCapabilities(LONG_COL).toColumnType()); } } @@ -283,8 +283,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 4, true, DOUBLE_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -350,7 +350,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE ); Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); } - Assert.assertEquals(ColumnType.DOUBLE, storageAdapter.getColumnCapabilities(DOUBLE_COL).toColumnType()); + Assert.assertEquals(ColumnType.DOUBLE, cursorFactory.getColumnCapabilities(DOUBLE_COL).toColumnType()); } } @@ -366,8 +366,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 4, true, STRING_ARRAY_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec( STRING_ARRAY_COL, @@ -397,7 +397,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index Assert.assertNull(valueSelector.getObject()); Assert.assertEquals( ColumnType.STRING_ARRAY, - storageAdapter.getColumnCapabilities(STRING_ARRAY_COL).toColumnType() + cursorFactory.getColumnCapabilities(STRING_ARRAY_COL).toColumnType() ); } } @@ -414,8 +414,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 4, true, VARIANT_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -442,7 +442,7 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize cursor.advance(); Assert.assertNull(valueSelector.getObject()); Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(VARIANT_COL).toColumnType()); + Assert.assertEquals(ColumnType.STRING, cursorFactory.getColumnCapabilities(VARIANT_COL).toColumnType()); } } @@ -458,8 +458,8 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -482,7 +482,7 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee cursor.advance(); Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(ColumnType.NESTED_DATA, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + Assert.assertEquals(ColumnType.NESTED_DATA, cursorFactory.getColumnCapabilities(NESTED_COL).toColumnType()); } } @@ -517,8 +517,8 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -545,7 +545,7 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz Assert.assertNull(valueSelector.getObject()); Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + Assert.assertEquals(ColumnType.STRING, cursorFactory.getColumnCapabilities(NESTED_COL).toColumnType()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 9dbd866654cc..9c88ab0dc8f2 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -394,7 +394,7 @@ public RowBasedSegment buildRowBasedSegmentWithTypeSignature() { // Determine row signature by building an mmapped index first. try (final QueryableIndex index = buildMMappedIndex()) { - final RowSignature signature = new QueryableIndexStorageAdapter(index).getRowSignature(); + final RowSignature signature = new QueryableIndexCursorFactory(index).getRowSignature(); return new RowBasedSegment<>( SegmentId.dummy("IndexBuilder"), @@ -409,8 +409,8 @@ public FrameSegment buildFrameSegment(FrameType frameType) { // Build mmapped index first, then copy over. try (final QueryableIndex index = buildMMappedIndex()) { - return FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + return FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), frameType, SegmentId.dummy("IndexBuilder") ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index b8c9721b5a0b..5c519f884586 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -65,6 +65,7 @@ import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; @@ -262,7 +263,7 @@ public void testPersistPlainNonTimeOrdered() throws Exception ImmutableList.of("3", "4", timestamp, 1L), ImmutableList.of("3", "4", timestamp + 1, 1L) ), - FrameTestUtil.readRowsFromAdapter(new QueryableIndexStorageAdapter(index), null, false).toList() + FrameTestUtil.readRowsFromCursorFactory(new QueryableIndexCursorFactory(index)).toList() ); } @@ -330,7 +331,7 @@ public void testPersistRollupNonTimeOrdered() throws Exception ImmutableList.of("3", "4", timestamp, 2L), ImmutableList.of("3", "4", timestamp + 1, 1L) ), - FrameTestUtil.readRowsFromAdapter(new QueryableIndexStorageAdapter(index), null, false).toList() + FrameTestUtil.readRowsFromCursorFactory(new QueryableIndexCursorFactory(index)).toList() ); } @@ -1705,10 +1706,14 @@ public void testAddMetrics() throws IOException indexSpec, -1 ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); - Assert.assertEquals(ImmutableSet.of("A", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); - + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) + ); } @Test @@ -1777,9 +1782,14 @@ public void testAddMetricsBothSidesNull() throws IOException indexSpec, -1 ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); - Assert.assertEquals(ImmutableSet.of("A", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) + ); } @@ -1844,10 +1854,14 @@ public void testMismatchedMetrics() throws IOException ); // Since D was not present in any of the indices, it is not present in the output - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); - Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); - + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "B", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) + ); } @Test(expected = IAE.class) @@ -1888,10 +1902,14 @@ public void testMismatchedMetricsVarying() throws IOException indexSpec, -1 ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter( - closer.closeLater(indexIO.loadIndex(merged)) + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "B", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) ); - Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); } @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 2ed9e8431501..5d0454314e85 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -36,7 +36,7 @@ * This is used to test the base cursor of an UnnestCursor. * Usages can be found in tests of {@link UnnestColumnValueSelectorCursor} in {@link UnnestColumnValueSelectorCursorTest} * However this cannot help with {@link UnnestDimensionCursor}. - * Tests for {@link UnnestDimensionCursor} are done alongside tests for {@link UnnestStorageAdapterTest} + * Tests for {@link UnnestDimensionCursor} are done alongside tests for {@link UnnestCursorFactoryTest} */ public class ListCursor implements Cursor { 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 f13b6c5632a6..2e9deab42b49 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -32,8 +32,8 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.nested.StructuredData; @@ -146,9 +146,9 @@ public void testNestedColumnIndexerSchemaDiscoveryRootString() throws IndexSizeE index.add(makeInputRow(minTimestamp + 4, true, STRING_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -198,8 +198,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootLong() throws IndexSizeExc index.add(makeInputRow(minTimestamp + 4, true, LONG_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -279,8 +279,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootDouble() throws IndexSizeE index.add(makeInputRow(minTimestamp + 4, true, DOUBLE_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -360,8 +360,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootStringArray() throws Index index.add(makeInputRow(minTimestamp + 4, true, STRING_ARRAY_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec( STRING_ARRAY_COL, @@ -403,8 +403,8 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize index.add(makeInputRow(minTimestamp + 4, true, VARIANT_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -446,8 +446,8 @@ public void testNestedColumnIndexerSchemaDiscoveryNested() throws IndexSizeExcee index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorFactoryTest.java similarity index 94% rename from processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorFactoryTest.java index 4cde2a796585..05de6845e53b 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorFactoryTest.java @@ -40,7 +40,7 @@ import java.util.Arrays; import java.util.Collection; -public class QueryableIndexStorageAdapterTest +public class QueryableIndexCursorFactoryTest { @RunWith(Parameterized.class) public static class DimensionDictionarySelectorTest extends InitializedNullHandlingTest @@ -68,9 +68,9 @@ public DimensionDictionarySelectorTest(boolean vectorize) public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); if (vectorize) { - final VectorCursor cursor = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asVectorCursor(); + final VectorCursor cursor = closer.register(cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asVectorCursor(); final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -81,7 +81,7 @@ public void setUp() partialNullSelector = columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column")); } else { - final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); + final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); final Cursor cursor = cursorHolder.asCursor(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -225,8 +225,8 @@ public static class ManySelectorsOneColumnTest extends InitializedNullHandlingTe public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); cursor = cursorHolder.asCursor(); columnSelectorFactory = cursor.getColumnSelectorFactory(); } diff --git a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java index b1f65cfc8d1d..774af2c30c26 100644 --- a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java @@ -43,7 +43,7 @@ public class ReferenceCountingSegmentTest private final SegmentId segmentId = SegmentId.dummy("test_segment"); private final Interval dataInterval = new Interval(DateTimes.nowUtc().minus(Days.days(1)), DateTimes.nowUtc()); private QueryableIndex index; - private StorageAdapter adapter; + private CursorFactory cursorFactory; private IndexedTable indexedTable; private int underlyingSegmentClosedCount; @@ -52,7 +52,7 @@ public void setUp() { underlyingSegmentClosedCount = 0; index = EasyMock.createNiceMock(QueryableIndex.class); - adapter = EasyMock.createNiceMock(StorageAdapter.class); + cursorFactory = EasyMock.createNiceMock(CursorFactory.class); indexedTable = EasyMock.createNiceMock(IndexedTable.class); segment = ReferenceCountingSegment.wrapRootGenerationSegment( @@ -77,9 +77,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return adapter; + return cursorFactory; } @Override @@ -87,8 +87,8 @@ public T as(Class clazz) { if (clazz.equals(QueryableIndex.class)) { return (T) asQueryableIndex(); - } else if (clazz.equals(StorageAdapter.class)) { - return (T) asStorageAdapter(); + } else if (clazz.equals(CursorFactory.class)) { + return (T) asCursorFactory(); } else if (clazz.equals(IndexedTable.class)) { return (T) indexedTable; } @@ -172,14 +172,14 @@ public void testExposesWrappedSegment() Assert.assertEquals(segmentId, segment.getId()); Assert.assertEquals(dataInterval, segment.getDataInterval()); Assert.assertEquals(index, segment.asQueryableIndex()); - Assert.assertEquals(adapter, segment.asStorageAdapter()); + Assert.assertEquals(cursorFactory, segment.asCursorFactory()); } @Test public void testSegmentAs() { Assert.assertSame(index, segment.as(QueryableIndex.class)); - Assert.assertSame(adapter, segment.as(StorageAdapter.class)); + Assert.assertSame(cursorFactory, segment.as(CursorFactory.class)); Assert.assertSame(indexedTable, segment.as(IndexedTable.class)); Assert.assertNull(segment.as(String.class)); } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java similarity index 82% rename from processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java index 6c46050c6971..aab86309eaa5 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java @@ -51,7 +51,6 @@ import java.math.RoundingMode; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.concurrent.atomic.AtomicLong; @@ -61,9 +60,7 @@ import java.util.function.ToLongFunction; import java.util.stream.Collectors; -import static org.junit.Assert.assertThrows; - -public class RowBasedStorageAdapterTest +public class RowBasedCursorFactoryTest { private static final String UNKNOWN_TYPE_NAME = "unknownType"; @@ -246,14 +243,14 @@ public Function columnFunction(String columnName) public final AtomicLong numCloses = new AtomicLong(); - private RowBasedStorageAdapter createIntAdapter(final int... ints) + private RowBasedCursorFactory createIntAdapter(final int... ints) { return createIntAdapter(ROW_ADAPTER, ints); } - private RowBasedStorageAdapter createIntAdapter(RowAdapter adapter, final int... ints) + private RowBasedCursorFactory createIntAdapter(RowAdapter adapter, final int... ints) { - return new RowBasedStorageAdapter<>( + return new RowBasedCursorFactory<>( Sequences.simple(Arrays.stream(ints).boxed().collect(Collectors.toList())) .withBaggage(numCloses::incrementAndGet), adapter, @@ -261,113 +258,17 @@ private RowBasedStorageAdapter createIntAdapter(RowAdapter ada ); } - @Test - public void test_getInterval() - { - final RowBasedStorageAdapter adapter = createIntAdapter(); - Assert.assertEquals(Intervals.ETERNITY, adapter.getInterval()); - } - - @Test - public void test_getAvailableDimensions() - { - final RowBasedStorageAdapter adapter = createIntAdapter(); - - // Sort them for comparison purposes. - Assert.assertEquals( - ROW_SIGNATURE.getColumnNames().stream().sorted().collect(Collectors.toList()), - Lists.newArrayList(adapter.getAvailableDimensions()).stream().sorted().collect(Collectors.toList()) - ); - } - - @Test - public void test_getAvailableMetrics() - { - final RowBasedStorageAdapter adapter = createIntAdapter(); - - Assert.assertEquals( - Collections.emptyList(), - Lists.newArrayList(adapter.getAvailableMetrics()) - ); - } - @Test public void test_getRowSignature() { - final RowBasedStorageAdapter adapter = createIntAdapter(); + final RowBasedCursorFactory adapter = createIntAdapter(); Assert.assertEquals(ROW_SIGNATURE, adapter.getRowSignature()); } - @Test - public void test_getDimensionCardinality_knownColumns() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursor is called). - for (String column : ROW_SIGNATURE.getColumnNames()) { - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality(column)); - } - } - - @Test - public void test_getDimensionCardinality_unknownColumn() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality("unknown")); - } - - @Test - public void test_getDimensionCardinality_timeColumn() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality("__time")); - } - - @Test - public void test_getMinValue() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know min/max values, so they always return null. - // Test both known and unknown columns. - final List columns = - ImmutableList.builder().addAll(ROW_SIGNATURE.getColumnNames()).add("unknown", "__time").build(); - - for (String column : columns) { - Assert.assertNull(column, adapter.getMinValue(column)); - } - } - - @Test - public void test_getMaxValue() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know min/max values, so they always return null. - // Test both known and unknown columns. - final List columns = - ImmutableList.builder().addAll(ROW_SIGNATURE.getColumnNames()).add("unknown", "__time").build(); - - for (String column : columns) { - Assert.assertNull(column, adapter.getMaxValue(column)); - } - } - - @Test - public void test_getCapabilities() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursor is called). - for (String column : ROW_SIGNATURE.getColumnNames()) { - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality(column)); - } - } - @Test public void test_getColumnCapabilities_float() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.FLOAT.name()); Assert.assertEquals(ValueType.FLOAT, capabilities.getType()); @@ -377,7 +278,7 @@ public void test_getColumnCapabilities_float() @Test public void test_getColumnCapabilities_double() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.DOUBLE.name()); Assert.assertEquals(ValueType.DOUBLE, capabilities.getType()); @@ -387,7 +288,7 @@ public void test_getColumnCapabilities_double() @Test public void test_getColumnCapabilities_long() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.LONG.name()); Assert.assertEquals(ValueType.LONG, capabilities.getType()); @@ -397,7 +298,7 @@ public void test_getColumnCapabilities_long() @Test public void test_getColumnCapabilities_string() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.STRING.name()); Assert.assertEquals(ValueType.STRING, capabilities.getType()); @@ -411,7 +312,7 @@ public void test_getColumnCapabilities_string() @Test public void test_getColumnCapabilities_complex() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.COMPLEX.name()); @@ -425,7 +326,7 @@ public void test_getColumnCapabilities_complex() @Test public void test_getColumnCapabilities_unknownType() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(UNKNOWN_TYPE_NAME); Assert.assertNull(capabilities); @@ -434,14 +335,14 @@ public void test_getColumnCapabilities_unknownType() @Test public void test_getColumnCapabilities_nonexistent() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); Assert.assertNull(adapter.getColumnCapabilities("nonexistent")); } @Test public void test_getColumnTypeString() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); for (String columnName : ROW_SIGNATURE.getColumnNames()) { if (UNKNOWN_TYPE_NAME.equals(columnName)) { @@ -456,24 +357,10 @@ public void test_getColumnTypeString() } } - @Test - public void test_getNumRows() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata()); - } - - @Test - public void test_getMetadata() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata()); - } - @Test public void test_makeCursor() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .build(); @@ -495,7 +382,7 @@ public void test_makeCursor() @Test public void test_makeCursor_filterOnLong() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) @@ -518,7 +405,7 @@ public void test_makeCursor_filterOnLong() @Test public void test_makeCursor_filterOnNonexistentColumnEqualsNull() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) @@ -541,7 +428,7 @@ public void test_makeCursor_filterOnNonexistentColumnEqualsNull() @Test public void test_makeCursor_filterOnVirtualColumn() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) @@ -576,7 +463,7 @@ public void test_makeCursor_filterOnVirtualColumn() @Test public void test_makeCursor_descending() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setPreferredOrdering(Cursors.descendingTimeOrder()) @@ -599,7 +486,7 @@ public void test_makeCursor_descending() @Test public void test_makeCursor_intervalDoesNotMatch() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("2000/P1D")) @@ -618,7 +505,7 @@ public void test_makeCursor_intervalDoesNotMatch() @Test public void test_makeCursor_intervalPartiallyMatches() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT1H")) @@ -639,7 +526,7 @@ public void test_makeCursor_intervalPartiallyMatches() @Test public void test_makeCursor_hourGranularity() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 1, 2, 3); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970/1971")) @@ -663,7 +550,7 @@ public void test_makeCursor_hourGranularity() @Test public void test_makeCursor_hourGranularityWithInterval() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 1, 2, 3); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT2H")) @@ -686,7 +573,7 @@ public void test_makeCursor_hourGranularityWithInterval() @Test public void test_makeCursor_hourGranularityWithIntervalDescending() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 1, 2, 3); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT2H")) @@ -710,7 +597,7 @@ public void test_makeCursor_hourGranularityWithIntervalDescending() @Test public void test_makeCursor_allProcessors() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); @@ -815,7 +702,7 @@ public void test_makeCursor_allProcessors() @Test public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) diff --git a/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java b/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java index 2a910d0d42a2..59bde904e3c9 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java +++ b/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java @@ -61,9 +61,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return as(StorageAdapter.class); + return as(CursorFactory.class); } @SuppressWarnings("unchecked") diff --git a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java deleted file mode 100644 index 93266894c695..000000000000 --- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment; - -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.data.Indexed; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; - - -public class TombstoneSegmentStorageAdapterTest -{ - @Test - public void testTombstoneDefaultInterface() - { - StorageAdapter sa = new StorageAdapter() - { - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new CursorHolder() - { - @Nullable - @Override - public Cursor asCursor() - { - return null; - } - }; - } - - @Override - public Interval getInterval() - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Override - public int getDimensionCardinality(String column) - { - return 0; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Nullable - @Override - public Metadata getMetadata() - { - return null; - } - }; - - Assert.assertFalse(sa.isFromTombstone()); - } - -} diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java similarity index 82% rename from processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java index 96a06e3523b1..32770f91f2e5 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java @@ -29,6 +29,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; @@ -44,8 +45,8 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.join.PostJoinCursor; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -74,18 +75,18 @@ import static org.apache.druid.segment.filter.Filters.and; import static org.apache.druid.segment.filter.Filters.or; -public class UnnestStorageAdapterTest extends InitializedNullHandlingTest +public class UnnestCursorFactoryTest extends InitializedNullHandlingTest { @ClassRule public static TemporaryFolder tmp = new TemporaryFolder(); private static Closer CLOSER; private static IncrementalIndex INCREMENTAL_INDEX; - private static IncrementalIndexStorageAdapter INCREMENTAL_INDEX_STORAGE_ADAPTER; + private static IncrementalIndexCursorFactory INCREMENTAL_INDEX_CURSOR_FACTORY; private static QueryableIndex QUERYABLE_INDEX; - private static UnnestStorageAdapter UNNEST_STORAGE_ADAPTER; - private static UnnestStorageAdapter UNNEST_STORAGE_ADAPTER1; - private static UnnestStorageAdapter UNNEST_ARRAYS; - private static List ADAPTERS; + private static UnnestCursorFactory UNNEST_CURSOR_FACTORY; + private static UnnestCursorFactory UNNEST_CURSOR_FACTORY1; + private static UnnestCursorFactory UNNEST_ARRAYS; + private static List CURSOR_FACTORIES; private static String INPUT_COLUMN_NAME = "multi-string1"; private static String OUTPUT_COLUMN_NAME = "unnested-multi-string1"; private static String OUTPUT_COLUMN_NAME1 = "unnested-multi-string1-again"; @@ -110,21 +111,21 @@ public static void setup() throws IOException INCREMENTAL_INDEX = CLOSER.register( segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, numRows) ); - INCREMENTAL_INDEX_STORAGE_ADAPTER = new IncrementalIndexStorageAdapter(INCREMENTAL_INDEX); - UNNEST_STORAGE_ADAPTER = new UnnestStorageAdapter( - INCREMENTAL_INDEX_STORAGE_ADAPTER, + INCREMENTAL_INDEX_CURSOR_FACTORY = new IncrementalIndexCursorFactory(INCREMENTAL_INDEX); + UNNEST_CURSOR_FACTORY = new UnnestCursorFactory( + INCREMENTAL_INDEX_CURSOR_FACTORY, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - UNNEST_STORAGE_ADAPTER1 = new UnnestStorageAdapter( - UNNEST_STORAGE_ADAPTER, + UNNEST_CURSOR_FACTORY1 = new UnnestCursorFactory( + UNNEST_CURSOR_FACTORY, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME1, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); final InputSource inputSource = ResourceInputSource.of( - UnnestStorageAdapterTest.class.getClassLoader(), + UnnestCursorFactoryTest.class.getClassLoader(), NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE ); IndexBuilder bob = IndexBuilder.create() @@ -144,15 +145,15 @@ public static void setup() throws IOException .transform(TransformSpec.NONE) .inputTmpDir(tmp.newFolder()); QUERYABLE_INDEX = CLOSER.register(bob.buildMMappedIndex()); - UNNEST_ARRAYS = new UnnestStorageAdapter( - new QueryableIndexStorageAdapter(QUERYABLE_INDEX), + UNNEST_ARRAYS = new UnnestCursorFactory( + new QueryableIndexCursorFactory(QUERYABLE_INDEX), new ExpressionVirtualColumn("u", "\"arrayLongNulls\"", ColumnType.LONG, ExprMacroTable.nil()), null ); - ADAPTERS = ImmutableList.of( - UNNEST_STORAGE_ADAPTER, - UNNEST_STORAGE_ADAPTER1 + CURSOR_FACTORIES = ImmutableList.of( + UNNEST_CURSOR_FACTORY, + UNNEST_CURSOR_FACTORY1 ); } @@ -164,24 +165,22 @@ public static void teardown() } @Test - public void test_group_of_unnest_adapters_methods() + public void test_capabilities() { String colName = "multi-string1"; - for (StorageAdapter adapter : ADAPTERS) { - adapter.getColumnCapabilities(colName); - Assert.assertEquals(adapter.getNumRows(), 0); - Assert.assertNotNull(adapter.getMetadata()); + for (UnnestCursorFactory cursorFactory : CURSOR_FACTORIES) { + cursorFactory.getColumnCapabilities(colName); Assert.assertEquals( - adapter.getColumnCapabilities(colName).toColumnType(), - INCREMENTAL_INDEX_STORAGE_ADAPTER.getColumnCapabilities(colName).toColumnType() + cursorFactory.getColumnCapabilities(colName).toColumnType(), + INCREMENTAL_INDEX_CURSOR_FACTORY.getColumnCapabilities(colName).toColumnType() ); - assertColumnReadsIdentifier(((UnnestStorageAdapter) adapter).getUnnestColumn(), colName); + assertColumnReadsIdentifier(cursorFactory.getUnnestColumn(), colName); } } @Test - public void test_unnest_adapter_column_capabilities() + public void test_unnest_factory_column_capabilities() { String colName = "multi-string1"; List columnsInTable = Arrays.asList( @@ -200,27 +199,27 @@ public void test_unnest_adapter_column_capabilities() ValueType.STRING, ValueType.STRING ); - UnnestStorageAdapter adapter = UNNEST_STORAGE_ADAPTER; + UnnestCursorFactory cursorFactory = UNNEST_CURSOR_FACTORY; for (int i = 0; i < columnsInTable.size(); i++) { - ColumnCapabilities capabilities = adapter.getColumnCapabilities(columnsInTable.get(i)); + ColumnCapabilities capabilities = cursorFactory.getColumnCapabilities(columnsInTable.get(i)); Assert.assertEquals(capabilities.getType(), valueTypes.get(i)); } - assertColumnReadsIdentifier(adapter.getUnnestColumn(), colName); + assertColumnReadsIdentifier(cursorFactory.getUnnestColumn(), colName); Assert.assertEquals( - adapter.getColumnCapabilities(OUTPUT_COLUMN_NAME).isDictionaryEncoded(), + cursorFactory.getColumnCapabilities(OUTPUT_COLUMN_NAME).isDictionaryEncoded(), ColumnCapabilities.Capable.TRUE // passed through from dict-encoded input ); Assert.assertEquals( - adapter.getColumnCapabilities(OUTPUT_COLUMN_NAME).hasMultipleValues(), + cursorFactory.getColumnCapabilities(OUTPUT_COLUMN_NAME).hasMultipleValues(), ColumnCapabilities.Capable.FALSE ); } @Test - public void test_unnest_adapters_basic() + public void test_unnest_factory_basic() { - try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = UNNEST_CURSOR_FACTORY.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -254,7 +253,7 @@ public void test_unnest_adapters_basic() } @Test - public void test_unnest_adapters_basic_array_column() + public void test_unnest_factory_basic_array_column() { try (final CursorHolder cursorHolder = UNNEST_ARRAYS.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); @@ -286,10 +285,10 @@ public void test_unnest_adapters_basic_array_column() } @Test - public void test_unnest_adapters_basic_row_based_array_column() + public void test_unnest_factory_basic_row_based_array_column() { - StorageAdapter adapter = new UnnestStorageAdapter( - new RowBasedStorageAdapter<>( + UnnestCursorFactory cursorFactory = new UnnestCursorFactory( + new RowBasedCursorFactory<>( Sequences.simple( Arrays.asList( new Object[]{1L, new Object[]{1L, 2L}}, @@ -321,7 +320,7 @@ public Function columnFunction(String columnName) new ExpressionVirtualColumn("u", "\"a\"", ColumnType.LONG, ExprMacroTable.nil()), null ); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -351,9 +350,9 @@ public Function columnFunction(String columnName) } @Test - public void test_two_levels_of_unnest_adapters() + public void test_two_levels_of_unnest() { - try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = UNNEST_CURSOR_FACTORY1.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -384,17 +383,18 @@ public void test_two_levels_of_unnest_adapters() } @Test - public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest_adapters() + public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final UnnestCursorFactory cursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = cursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = cursorFactory.getUnnestInputIfDirectAccess(vc); final OrFilter baseFilter = new OrFilter(ImmutableList.of( selector(OUTPUT_COLUMN_NAME, "1"), @@ -409,10 +409,9 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(baseFilter) .build(); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); @@ -423,17 +422,18 @@ public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest } @Test - public void test_nested_filters_unnested_and_original_dimension_with_unnest_adapters() + public void test_nested_filters_unnested_and_original_dimension_with_unnest() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final UnnestCursorFactory cursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = cursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = cursorFactory.getUnnestInputIfDirectAccess(vc); final OrFilter baseFilter = new OrFilter(ImmutableList.of( selector(OUTPUT_COLUMN_NAME, "1"), @@ -454,10 +454,9 @@ public void test_nested_filters_unnested_and_original_dimension_with_unnest_adap final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(baseFilter) .build(); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); @@ -714,30 +713,31 @@ public void testArrayCannotPushdown2() } @Test - public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() + public void test_pushdown_filters_unnested_dimension_with_unnest() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final DimFilter filter = new SelectorDimFilter(OUTPUT_COLUMN_NAME, "1", null); + final UnnestCursorFactory unnestCursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), - new SelectorDimFilter(OUTPUT_COLUMN_NAME, "1", null) + filter ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = unnestCursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = unnestCursorFactory.getUnnestInputIfDirectAccess(vc); final Filter expectedPushDownFilter = selector(inputColumn, "1"); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = unnestCursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); - Assert.assertEquals(unnestStorageAdapter.getUnnestFilter(), postFilter); + Assert.assertEquals(filter.toFilter(), postFilter); int count = 0; while (!cursor.isDone()) { @@ -752,15 +752,16 @@ public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() @Test public void test_pushdown_filters_unnested_dimension_outside() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final UnnestCursorFactory unnestCursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = unnestCursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = unnestCursorFactory.getUnnestInputIfDirectAccess(vc); final Filter expectedPushDownFilter = selector(inputColumn, "1"); @@ -770,10 +771,9 @@ public void test_pushdown_filters_unnested_dimension_outside() .setFilter(queryFilter) .build(); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = unnestCursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); @@ -807,14 +807,14 @@ public void testUnnestValueMatcherValueDoesntExist() IncrementalIndex index = CLOSER.register( segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, 100) ); - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - UnnestStorageAdapter withNullsStorageAdapter = new UnnestStorageAdapter( - adapter, + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + UnnestCursorFactory unnestCursorFactory = new UnnestCursorFactory( + cursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), null ); - try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = unnestCursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -843,7 +843,7 @@ public void testComputeBaseAndPostUnnestFilters( ) { testComputeBaseAndPostUnnestFilters( - UNNEST_STORAGE_ADAPTER, + UNNEST_CURSOR_FACTORY, testQueryFilter, expectedBasePushDown, expectedPostUnnest @@ -851,20 +851,20 @@ public void testComputeBaseAndPostUnnestFilters( } public void testComputeBaseAndPostUnnestFilters( - UnnestStorageAdapter adapter, + UnnestCursorFactory cursorFactory, Filter testQueryFilter, String expectedBasePushDown, String expectedPostUnnest ) { - final String inputColumn = adapter.getUnnestInputIfDirectAccess(adapter.getUnnestColumn()); - final VirtualColumn vc = adapter.getUnnestColumn(); - Pair filterPair = adapter.computeBaseAndPostUnnestFilters( + final String inputColumn = cursorFactory.getUnnestInputIfDirectAccess(cursorFactory.getUnnestColumn()); + final VirtualColumn vc = cursorFactory.getUnnestColumn(); + Pair filterPair = cursorFactory.computeBaseAndPostUnnestFilters( testQueryFilter, null, VirtualColumns.EMPTY, inputColumn, - vc.capabilities(adapter, inputColumn) + vc.capabilities(cursorFactory, inputColumn) ); Filter actualPushDownFilter = filterPair.lhs; Filter actualPostUnnestFilter = filterPair.rhs; @@ -889,16 +889,16 @@ private static void assertColumnReadsIdentifier(final VirtualColumn column, fina /** * Class to test the flow of pushing down filters into the base cursor - * while using the UnnestStorageAdapter. This class keeps a reference of the filter + * while using the {@link UnnestCursorFactory}. This class keeps a reference of the filter * which is pushed down to the cursor which serves as a checkpoint to validate * if the right filter is being pushed down */ -class TestStorageAdapter extends IncrementalIndexStorageAdapter +class TestCursorFactory extends IncrementalIndexCursorFactory { private Filter pushDownFilter; - public TestStorageAdapter(IncrementalIndex index) + public TestCursorFactory(IncrementalIndex index) { super(index); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java index 2481a6e8aad4..35185967f66d 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java @@ -34,8 +34,8 @@ import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -69,7 +69,7 @@ public class AndFilterTest extends BaseFilterTest public AndFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java index f45885790dc3..6830040bab88 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java @@ -34,8 +34,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -56,7 +56,7 @@ public static class ArrayContainsElementFilterTest extends BaseFilterTest public ArrayContainsElementFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) 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 1491fbd9d388..2e3ae0b633ff 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 @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.SettableSupplier; @@ -40,7 +39,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.segment.FrameSegment; -import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -69,16 +69,16 @@ 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.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; -import org.apache.druid.segment.RowBasedStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -91,8 +91,8 @@ import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -404,7 +404,7 @@ public static InputRow makeSchemaRow( private final List rows; protected final IndexBuilder indexBuilder; - protected final Function> finisher; + protected final Function> finisher; protected final boolean cnf; protected final boolean optimize; protected final String testName; @@ -414,22 +414,22 @@ public static InputRow makeSchemaRow( // In other words, numeric null values will be treated as nulls instead of the default value protected final boolean canTestNumericNullsAsDefaultValues; - protected StorageAdapter adapter; + protected CursorFactory cursorFactory; protected VirtualColumns virtualColumns; // JUnit creates a new test instance for every test method call. // For filter tests, the test setup creates a segment. - // Creating a new segment for every test method call is pretty slow, so cache the StorageAdapters. + // Creating a new segment for every test method call is pretty slow, so cache the CursorFactory. // Each thread gets its own map. - private static ThreadLocal>> adapterCache = + private static ThreadLocal>> adapterCache = ThreadLocal.withInitial(HashMap::new); public BaseFilterTest( String testName, List rows, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) @@ -449,18 +449,18 @@ public void setUp() throws Exception { BuiltInTypesModule.registerHandlersAndSerde(); String className = getClass().getName(); - Map adaptersForClass = adapterCache.get().get(className); + Map adaptersForClass = adapterCache.get().get(className); if (adaptersForClass == null) { adaptersForClass = new HashMap<>(); adapterCache.get().put(className, adaptersForClass); } - AdapterStuff adapterStuff = adaptersForClass.get(testName); - if (adapterStuff == null) { - Pair pair = finisher.apply( + CursorStuff cursorStuff = adaptersForClass.get(testName); + if (cursorStuff == null) { + Pair pair = finisher.apply( indexBuilder.tmpDir(temporaryFolder.newFolder()).rows(rows) ); - adapterStuff = new AdapterStuff( + cursorStuff = new CursorStuff( pair.lhs, VirtualColumns.create( Arrays.stream(VIRTUAL_COLUMNS.getVirtualColumns()) @@ -469,19 +469,19 @@ public void setUp() throws Exception ), pair.rhs ); - adaptersForClass.put(testName, adapterStuff); + adaptersForClass.put(testName, cursorStuff); } - this.adapter = adapterStuff.adapter; - this.virtualColumns = adapterStuff.virtualColumns; + this.cursorFactory = cursorStuff.cursorFactory; + this.virtualColumns = cursorStuff.virtualColumns; } public static void tearDown(String className) throws Exception { - Map adaptersForClass = adapterCache.get().get(className); + Map adaptersForClass = adapterCache.get().get(className); if (adaptersForClass != null) { - for (Map.Entry entry : adaptersForClass.entrySet()) { + for (Map.Entry entry : adaptersForClass.entrySet()) { entry.getValue().closeable.close(); } adapterCache.get().put(className, null); @@ -508,13 +508,13 @@ public static Collection makeConstructors() "off-heap memory segment write-out medium", OffHeapMemorySegmentWriteOutMediumFactory.instance() ); - final Map>> finishers = - ImmutableMap.>>builder() + final Map>> finishers = + ImmutableMap.>>builder() .put( "incremental", input -> { final IncrementalIndex index = input.buildIncrementalIndex(); - return Pair.of(new IncrementalIndexStorageAdapter(index), index); + return Pair.of(new IncrementalIndexCursorFactory(index), index); } ) .put( @@ -542,7 +542,7 @@ public static Collection makeConstructors() ) ); final IncrementalIndex index = input.buildIncrementalIndex(); - return Pair.of(new IncrementalIndexStorageAdapter(index), index); + return Pair.of(new IncrementalIndexCursorFactory(index), index); } ) .put( @@ -570,7 +570,7 @@ public static Collection makeConstructors() ) ); final QueryableIndex index = input.buildMMappedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( @@ -605,21 +605,21 @@ public static Collection makeConstructors() .intermediaryPersistSize(3) .buildMMappedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( "mmapped", input -> { final QueryableIndex index = input.buildMMappedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( "mmappedMerged", input -> { final QueryableIndex index = input.buildMMappedMergedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( @@ -640,7 +640,7 @@ public static Collection makeConstructors() Assert.assertTrue(NullHandling.replaceWithDefault()); try { final QueryableIndex index = input.getIndexIO().loadIndex(file); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } catch (IOException e) { throw new RuntimeException(e); @@ -649,11 +649,11 @@ public static Collection makeConstructors() ) .put( "rowBasedWithoutTypeSignature", - input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asStorageAdapter(), () -> {}) + input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asCursorFactory(), () -> {}) ) .put( "rowBasedWithTypeSignature", - input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> {}) + input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asCursorFactory(), () -> {}) ) .put("frame (row-based)", input -> { // remove variant type columns from row frames since they aren't currently supported @@ -676,7 +676,7 @@ public static Collection makeConstructors() ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED); - return Pair.of(segment.asStorageAdapter(), segment); + return Pair.of(segment.asCursorFactory(), segment); }) .put("frame (columnar)", input -> { // remove array type columns from columnar frames since they aren't currently supported @@ -699,7 +699,7 @@ public static Collection makeConstructors() ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR); - return Pair.of(segment.asStorageAdapter(), segment); + return Pair.of(segment.asCursorFactory(), segment); }) .build(); @@ -711,7 +711,7 @@ public static Collection makeConstructors() for (Map.Entry bitmapSerdeFactoryEntry : bitmapSerdeFactories.entrySet()) { for (Map.Entry segmentWriteOutMediumFactoryEntry : segmentWriteOutMediumFactories.entrySet()) { - for (Map.Entry>> finisherEntry : + for (Map.Entry>> finisherEntry : finishers.entrySet()) { for (boolean cnf : ImmutableList.of(false, true)) { for (boolean optimize : ImmutableList.of(false, true)) { @@ -815,7 +815,7 @@ private CursorBuildSpec makeVectorCursorBuildSpec(@Nullable Filter filter) private VectorCursor makeVectorCursor(final Filter filter) { final CursorBuildSpec buildSpec = makeVectorCursorBuildSpec(filter); - return adapter.makeCursorHolder(buildSpec).asVectorCursor(); + return cursorFactory.makeCursorHolder(buildSpec).asVectorCursor(); } /** @@ -823,7 +823,7 @@ private VectorCursor makeVectorCursor(final Filter filter) */ private List selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn) { - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(makeFilter(filter)))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(makeFilter(filter)))) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -843,7 +843,7 @@ private List selectColumnValuesMatchingFilter(final DimFilter filter, fi private long selectCountUsingFilteredAggregator(final DimFilter filter) { - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(null))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(null))) { final Cursor cursor = cursorHolder.asCursor(); Aggregator agg = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), @@ -861,13 +861,13 @@ private long selectCountUsingFilteredAggregator(final DimFilter filter) private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFilter) { Preconditions.checkState( - makeFilter(dimFilter).canVectorizeMatcher(adapter), + makeFilter(dimFilter).canVectorizeMatcher(cursorFactory), "Cannot vectorize filter: %s", dimFilter ); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(null))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(null))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), @@ -934,7 +934,7 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(postFilteringFilter))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(postFilteringFilter))) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -994,7 +994,7 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) } }; - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(postFilteringFilter))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(postFilteringFilter))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() @@ -1019,7 +1019,7 @@ private List selectColumnValuesMatchingFilterUsingVectorCursor( final String selectColumn ) { - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() @@ -1046,7 +1046,7 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur ) { final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); @@ -1102,10 +1102,6 @@ private List selectColumnValuesMatchingFilterUsingRowBasedColumnSelector ) { // Generate rowSignature - final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - for (String columnName : Iterables.concat(adapter.getAvailableDimensions(), adapter.getAvailableMetrics())) { - rowSignatureBuilder.add(columnName, adapter.getColumnCapabilities(columnName).toColumnType()); - } // Perform test final SettableSupplier rowSupplier = new SettableSupplier<>(); @@ -1114,7 +1110,7 @@ private List selectColumnValuesMatchingFilterUsingRowBasedColumnSelector RowBasedColumnSelectorFactory.create( RowAdapters.standardRow(), rowSupplier::get, - rowSignatureBuilder.build(), + cursorFactory.getRowSignature(), false, false ) @@ -1136,12 +1132,13 @@ protected void assertFilterMatches( ) { // IncrementalIndex, RowBasedSegment cannot vectorize. - // Columnar FrameStorageAdapter *can* vectorize, but the tests won't pass, because the vectorizable cases - // differ from QueryableIndexStorageAdapter due to frames not having indexes. So, skip these too. + // ColumnarFrameCursorFactory *can* vectorize, but the tests won't pass, because the vectorizable cases + // differ from QueryableIndexCursorFactory due to frames not having indexes. So, skip these too. final boolean testVectorized = - !(adapter instanceof IncrementalIndexStorageAdapter) - && !(adapter instanceof RowBasedStorageAdapter) - && !(adapter instanceof FrameStorageAdapter); + !(cursorFactory instanceof IncrementalIndexCursorFactory) + && !(cursorFactory instanceof RowBasedCursorFactory) + && !(cursorFactory instanceof RowFrameCursorFactory) + && !(cursorFactory instanceof ColumnarFrameCursorFactory); assertFilterMatches(filter, expectedRows, testVectorized); // test double inverted @@ -1156,12 +1153,13 @@ protected void assertFilterMatchesSkipArrays( ) { // IncrementalIndex, RowBasedSegment cannot vectorize. - // Columnar FrameStorageAdapter *can* vectorize, but the tests won't pass, because the vectorizable cases - // differ from QueryableIndexStorageAdapter due to frames not having indexes. So, skip these too. + // ColumnarFrameCursorHolderFactory *can* vectorize, but the tests won't pass, because the vectorizable cases + // differ from QueryableIndexCursorFactory due to frames not having indexes. So, skip these too. final boolean testVectorized = - !(adapter instanceof IncrementalIndexStorageAdapter) - && !(adapter instanceof RowBasedStorageAdapter) - && !(adapter instanceof FrameStorageAdapter); + !(cursorFactory instanceof IncrementalIndexCursorFactory) + && !(cursorFactory instanceof RowBasedCursorFactory) + && !(cursorFactory instanceof RowFrameCursorFactory) + && !(cursorFactory instanceof ColumnarFrameCursorFactory); if (isAutoSchema()) { Throwable t = Assert.assertThrows( @@ -1246,19 +1244,19 @@ private void assertFilterMatches( } } - private static class AdapterStuff + private static class CursorStuff { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; private final VirtualColumns virtualColumns; private final Closeable closeable; - private AdapterStuff( - StorageAdapter adapter, + private CursorStuff( + CursorFactory cursorFactory, VirtualColumns virtualColumns, Closeable closeable ) { - this.adapter = adapter; + this.cursorFactory = cursorFactory; this.virtualColumns = virtualColumns; this.closeable = closeable; } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java index 036abc13cd66..69d80b3f3a2c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -62,7 +62,7 @@ public class BoundFilterTest extends BaseFilterTest public BoundFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java index 8a46d313d2d0..3525e120c3e3 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java @@ -38,8 +38,8 @@ import org.apache.druid.query.filter.ColumnComparisonDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -78,7 +78,7 @@ public class ColumnComparisonFilterTest extends BaseFilterTest public ColumnComparisonFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index 97be448e61ab..0259a2c34327 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -40,8 +40,8 @@ import org.apache.druid.query.filter.IsFalseDimFilter; import org.apache.druid.query.filter.IsTrueDimFilter; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -62,7 +62,7 @@ public static class EqualityFilterTest extends BaseFilterTest public EqualityFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java index e9fea68c0dfe..8b60360b6c16 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java @@ -25,8 +25,8 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -40,7 +40,7 @@ public class ExpressionFilterNonStrictBooleansTest extends ExpressionFilterTest public ExpressionFilterNonStrictBooleansTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index 1bade62c5c80..aa2d21ef9d33 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -41,8 +41,8 @@ import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -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; @@ -108,7 +108,7 @@ public class ExpressionFilterTest extends BaseFilterTest public ExpressionFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index f95bd2ea8e99..d0d2941277a6 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -39,8 +39,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.index.BitmapColumnIndex; import org.junit.AfterClass; @@ -160,7 +160,7 @@ public DruidDoublePredicate makeDoublePredicate() public FilterPartitionTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java index c9717f186284..b1112e061fdf 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java @@ -45,8 +45,8 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; import org.junit.Test; @@ -93,7 +93,7 @@ public class FloatAndDoubleFilteringTest extends BaseFilterTest public FloatAndDoubleFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java index 6f5c4b72eb1a..fb0ebbfb031f 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java @@ -47,9 +47,9 @@ import org.apache.druid.query.filter.TypedInFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -87,7 +87,7 @@ public static class InFilterTest extends BaseFilterTest public InFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java index 54267591697b..36702eedb1f0 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java @@ -35,8 +35,8 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; import org.junit.Test; @@ -79,7 +79,7 @@ public class InvalidFilteringTest extends BaseFilterTest public InvalidFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java index 98b9579a9079..9e337fe6ece5 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java @@ -32,8 +32,8 @@ import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -51,7 +51,7 @@ public class JavaScriptFilterTest extends BaseFilterTest public JavaScriptFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java index 5cc37bf4fa0d..d3cc658d6ae1 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java @@ -39,8 +39,8 @@ import org.apache.druid.query.filter.LikeDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.junit.AfterClass; @@ -85,7 +85,7 @@ public class LikeFilterTest extends BaseFilterTest public LikeFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java index 3fd1c55ebab8..5011f3379093 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java @@ -43,8 +43,8 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; import org.junit.Test; @@ -88,7 +88,7 @@ public class LongFilteringTest extends BaseFilterTest public LongFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java index ed1be8acde3e..ddc21c6d374e 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java @@ -32,8 +32,8 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -67,7 +67,7 @@ public class NotFilterEvaluateTest extends BaseFilterTest public NotFilterEvaluateTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java index 767f450fc2e2..26d0ca547f48 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java @@ -32,8 +32,8 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; @@ -52,7 +52,7 @@ public static class NullFilterTest extends BaseFilterTest public NullFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java index 691339fe4980..54689d30d9de 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java @@ -38,8 +38,8 @@ import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.TrueDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -73,7 +73,7 @@ public class OrFilterTest extends BaseFilterTest public OrFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java index e2c0be5909c7..d55d71b792d8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java @@ -41,8 +41,8 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.RangeFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -117,7 +117,7 @@ public static class RangeFilterTest extends BaseFilterTest public RangeFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java index a6ec01681771..744232ad6fcd 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.extraction.JavaScriptExtractionFn; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.RegexDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -49,7 +49,7 @@ public class RegexFilterTest extends BaseFilterTest public RegexFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java index 8aef0ef27ab9..4c620eec8d88 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.filter.SearchQueryDimFilter; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.query.search.SearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -51,7 +51,7 @@ public class SearchQueryFilterTest extends BaseFilterTest public SearchQueryFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java index c51b94fc59a5..ed082c4598e0 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; @@ -55,7 +55,7 @@ public class SelectorFilterTest extends BaseFilterTest public SelectorFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java index da4abbe587a6..48c02ad71d75 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java @@ -47,8 +47,8 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; import org.junit.AfterClass; import org.junit.Test; @@ -88,7 +88,7 @@ public class TimeFilteringTest extends BaseFilterTest public TimeFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) 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 bab827130e4f..87c60cb74207 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 @@ -64,17 +64,19 @@ 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.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -182,18 +184,17 @@ public void testSanity() throws Exception .addOrderByColumn("billy") .build(); final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); try ( CloseableStupidPool pool = new CloseableStupidPool<>( "GroupByQueryEngine-bufferPool", () -> ByteBuffer.allocate(50000) ); ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec) + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) ) { final Sequence rows = GroupByQueryEngine.process( query, - adapter, new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, @@ -258,7 +259,7 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception ) .addOrderByColumn("billy") .build(); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); try ( CloseableStupidPool pool = new CloseableStupidPool<>( @@ -266,11 +267,10 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception () -> ByteBuffer.allocate(50000) ); ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec) + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) ) { final Sequence rows = GroupByQueryEngine.process( query, - adapter, new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, @@ -320,7 +320,7 @@ public void testResetSanity() throws IOException ) ); - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); for (boolean descending : Arrays.asList(false, true)) { final CursorBuildSpec buildSpec = CursorBuildSpec @@ -330,7 +330,7 @@ public void testResetSanity() throws IOException .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : Cursors.ascendingTimeOrder()) .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector; @@ -390,8 +390,7 @@ public void testSingleValueTopN() throws IOException .threshold(10) .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) .build(), - new IncrementalIndexStorageAdapter(index), - new IncrementalIndexTimeBoundaryInspector(index), + new IncrementalIndexSegment(index, SegmentId.dummy("test")), null ) .toList(); @@ -430,7 +429,7 @@ public void testFilterByNull() throws Exception .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); try ( @@ -439,11 +438,10 @@ public void testFilterByNull() throws Exception () -> ByteBuffer.allocate(50000) ); ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec) + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) ) { final Sequence rows = GroupByQueryEngine.process( query, - adapter, new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, @@ -478,12 +476,12 @@ public void testCursoringAndIndexUpdationInterleaving() throws Exception ); } - final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .build(); - try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector = cursor .getColumnSelectorFactory() @@ -534,13 +532,13 @@ public void testCursorDictionaryRaceConditionFix() throws Exception ); } - final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new DictionaryRaceTestFilter(index, timestamp)) .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .build(); - try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector = cursor .getColumnSelectorFactory() @@ -579,12 +577,12 @@ public void testCursoringAndSnapshot() throws Exception index.add(rows.get(i)); } - final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .build(); - try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector1A = cursor diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentCursorFactoryTest.java similarity index 95% rename from processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentCursorFactoryTest.java index c3a59659afc6..105b2ae7e3c1 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentCursorFactoryTest.java @@ -28,6 +28,7 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -54,7 +55,7 @@ import java.util.Collections; import java.util.List; -public class BaseHashJoinSegmentStorageAdapterTest extends InitializedNullHandlingTest +public class BaseHashJoinSegmentCursorFactoryTest extends InitializedNullHandlingTest { public static JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( true, @@ -223,17 +224,18 @@ protected JoinableClause regionToCountry(final JoinType joinType) /** * Creates a fact-to-country join segment without a {@link JoinFilterPreAnalysis}. This means it cannot - * have {@link org.apache.druid.segment.StorageAdapter#makeCursorHolder(CursorBuildSpec)} called on it. + * have {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)} called on it. */ - protected HashJoinSegmentStorageAdapter makeFactToCountrySegment() + protected HashJoinSegment makeFactToCountrySegment() { return makeFactToCountrySegment(JoinType.LEFT); } - protected HashJoinSegmentStorageAdapter makeFactToCountrySegment(JoinType joinType) + protected HashJoinSegment makeFactToCountrySegment(JoinType joinType) { - return new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + return new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, ImmutableList.of(factToCountryOnIsoCode(joinType)), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java similarity index 86% rename from processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java index 7ae04d9fcb12..1aedc1cd3c37 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java @@ -32,6 +32,8 @@ import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.TopNOptimizationInspector; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; @@ -47,14 +49,14 @@ import java.util.Collections; import java.util.List; -public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorageAdapterTest +public class HashJoinSegmentCursorFactoryTest extends BaseHashJoinSegmentCursorFactoryTest { @Test public void test_getInterval_factToCountry() { Assert.assertEquals( Intervals.of("2015-09-12/2015-09-12T05:21:00.060Z"), - makeFactToCountrySegment().getInterval() + makeFactToCountrySegment().getDataInterval() ); } @@ -79,133 +81,14 @@ public void test_getRowSignature_factToCountry() "c1.countryIsoCode", "c1.countryName" ), - Lists.newArrayList(makeFactToCountrySegment().getRowSignature().getColumnNames()) + Lists.newArrayList(makeFactToCountrySegment().asCursorFactory().getRowSignature().getColumnNames()) ); } - @Test - public void test_getAvailableDimensions_factToCountry() - { - Assert.assertEquals( - ImmutableList.of( - "channel", - "regionIsoCode", - "countryNumber", - "countryIsoCode", - "user", - "isRobot", - "isAnonymous", - "namespace", - "page", - "delta", - "c1.countryNumber", - "c1.countryIsoCode", - "c1.countryName" - ), - Lists.newArrayList(makeFactToCountrySegment().getAvailableDimensions().iterator()) - ); - } - - @Test - public void test_getAvailableMetrics_factToCountry() - { - Assert.assertEquals( - ImmutableList.of("channel_uniques"), - Lists.newArrayList(makeFactToCountrySegment().getAvailableMetrics().iterator()) - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryFactColumn() - { - Assert.assertEquals( - 18, - makeFactToCountrySegment().getDimensionCardinality("countryIsoCode") - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryJoinColumn() - { - Assert.assertEquals( - 19, - makeFactToCountrySegment().getDimensionCardinality(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryNonexistentFactColumn() - { - Assert.assertEquals( - 1, - makeFactToCountrySegment().getDimensionCardinality("nonexistent") - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryNonexistentJoinColumn() - { - Assert.assertEquals( - 1, - makeFactToCountrySegment().getDimensionCardinality(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent") - ); - } - - @Test - public void test_getMinValue_factToCountryFactColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue("countryIsoCode")); - } - - @Test - public void test_getMinValue_factToCountryJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode")); - } - - @Test - public void test_getMinValue_factToCountryNonexistentFactColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue("nonexistent")); - } - - @Test - public void test_getMinValue_factToCountryNonexistentJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent")); - } - - @Test - public void test_getMaxValue_factToCountryFactColumn() - { - Assert.assertEquals( - "US", - makeFactToCountrySegment().getMaxValue("countryIsoCode") - ); - } - - @Test - public void test_getMaxValue_factToCountryJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName")); - } - - @Test - public void test_getMaxValue_factToCountryNonexistentFactColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMaxValue("nonexistent")); - } - - @Test - public void test_getMaxValue_factToCountryNonexistentJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent")); - } - @Test public void test_getColumnCapabilities_factToCountryFactColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment().getColumnCapabilities("countryIsoCode"); + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("countryIsoCode"); Assert.assertEquals(ValueType.STRING, capabilities.getType()); Assert.assertTrue(capabilities.hasBitmapIndexes()); @@ -217,7 +100,7 @@ public void test_getColumnCapabilities_factToCountryFactColumn() @Test public void test_getColumnCapabilities_factToCountryJoinColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment().getColumnCapabilities( + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory().getColumnCapabilities( FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode" ); @@ -231,7 +114,7 @@ public void test_getColumnCapabilities_factToCountryJoinColumn() @Test public void test_getColumnCapabilities_factToCountryNonexistentFactColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment() + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory() .getColumnCapabilities("nonexistent"); Assert.assertNull(capabilities); @@ -240,7 +123,7 @@ public void test_getColumnCapabilities_factToCountryNonexistentFactColumn() @Test public void test_getColumnCapabilities_factToCountryNonexistentJoinColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment() + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory() .getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent"); Assert.assertNull(capabilities); @@ -251,7 +134,7 @@ public void test_getColumnCapabilities_complexTypeName_factToCountryFactColumn() { Assert.assertEquals( "hyperUnique", - makeFactToCountrySegment().getColumnCapabilities("channel_uniques").getComplexTypeName() + makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("channel_uniques").getComplexTypeName() ); } @@ -260,7 +143,7 @@ public void test_getColumnCapabilities_typeString_factToCountryFactColumn() { Assert.assertEquals( "COMPLEX", - makeFactToCountrySegment().getColumnCapabilities("channel_uniques").asTypeString() + makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("channel_uniques").asTypeString() ); } @@ -269,29 +152,12 @@ public void test_getColumnCapabilities_typeString_factToCountryJoinColumn() { Assert.assertEquals( "STRING", - makeFactToCountrySegment().getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") + makeFactToCountrySegment().asCursorFactory() + .getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") .asTypeString() ); } - @Test - public void test_getNumRows_factToCountry() - { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("Cannot retrieve number of rows from join segment"); - - makeFactToCountrySegment().getNumRows(); - } - - @Test - public void test_getMetadata_factToCountry() - { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("Cannot retrieve metadata from join segment"); - - makeFactToCountrySegment().getMetadata(); - } - @Test public void test_makeCursor_factToCountryLeft() { @@ -303,8 +169,9 @@ public void test_makeCursor_factToCountryLeft() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -360,8 +227,9 @@ public void test_makeCursor_factToCountryLeftUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -415,8 +283,9 @@ public void test_makeCursor_factToCountryInner() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -465,8 +334,9 @@ public void test_makeCursor_factToCountryInnerUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -517,8 +387,9 @@ public void test_makeCursor_factToCountryInnerUsingCountryNumber() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -577,8 +448,9 @@ public void test_makeCursor_factToCountryInnerUsingCountryNumberUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -633,8 +505,9 @@ public void test_makeCursor_factToCountryLeftWithFilterOnFacts() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -664,8 +537,9 @@ public void test_makeCursor_factToCountryLeftWithFilterOnFactsUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -693,8 +567,9 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -708,7 +583,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNull() ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, @@ -727,8 +602,9 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup( joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -741,7 +617,7 @@ public void test_makeCursor_factToCountryRightWithFilterOnLeftIsNullUsingLookup( ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, @@ -760,8 +636,9 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -775,7 +652,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNull() ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, @@ -794,8 +671,9 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -808,7 +686,7 @@ public void test_makeCursor_factToCountryFullWithFilterOnLeftIsNullUsingLookup() ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, @@ -833,8 +711,9 @@ public void test_makeCursor_factToCountryRightWithFilterOnJoinable() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -870,8 +749,9 @@ public void test_makeCursor_factToCountryRightWithFilterOnJoinableUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -907,8 +787,9 @@ public void test_makeCursor_factToCountryLeftWithFilterOnJoinable() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -944,8 +825,9 @@ public void test_makeCursor_factToCountryLeftWithFilterOnJoinableUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -994,8 +876,9 @@ public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditi VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1065,8 +948,9 @@ public void test_makeCursor_factToCountryInnerWithFilterInsteadOfRealJoinConditi VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1119,8 +1003,9 @@ public void test_makeCursor_factToRegionToCountryLeft() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1176,8 +1061,9 @@ public void test_makeCursor_factToRegionToCountryInnerIncludeNull() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1241,8 +1127,9 @@ public void test_makeCursor_factToCountryAlwaysTrue() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1252,7 +1139,7 @@ public void test_makeCursor_factToCountryAlwaysTrue() ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, @@ -1302,8 +1189,9 @@ public void test_makeCursor_factToCountryAlwaysFalse() ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1342,8 +1230,9 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1353,7 +1242,7 @@ public void test_makeCursor_factToCountryAlwaysTrueUsingLookup() ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, @@ -1403,8 +1292,9 @@ public void test_makeCursor_factToCountryAlwaysFalseUsingLookup() ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1448,8 +1338,9 @@ public void test_makeCursor_factToCountryUsingVirtualColumn() virtualColumns ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1499,8 +1390,9 @@ public void test_makeCursor_factToCountryUsingVirtualColumnUsingLookup() virtualColumns ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1547,8 +1439,9 @@ public void test_makeCursor_factToCountryUsingExpression() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1592,8 +1485,9 @@ public void test_makeCursor_factToCountryUsingExpressionUsingLookup() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1638,8 +1532,9 @@ public void test_makeCursor_factToRegionTheWrongWay() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1652,7 +1547,7 @@ public void test_makeCursor_factToRegionTheWrongWay() ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{"Giusy Ferreri discography", "VA", "IT", "Provincia di Varese", "IT"}, @@ -1691,8 +1586,9 @@ public void test_makeCursor_errorOnNonEquiJoin() ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1726,8 +1622,9 @@ public void test_makeCursor_errorOnNonEquiJoinUsingLookup() ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1761,8 +1658,9 @@ public void test_makeCursor_errorOnNonKeyBasedJoin() ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1796,8 +1694,9 @@ public void test_makeCursor_errorOnNonKeyBasedJoinUsingLookup() ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1817,8 +1716,9 @@ public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRows() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1846,8 +1746,9 @@ public void test_makeCursor_factToCountryLeft_filterExcludesAllLeftRowsUsingLook VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1874,8 +1775,9 @@ public void test_makeCursor_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE VirtualColumns.EMPTY ); - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1899,8 +1801,8 @@ public void test_makeCursor_factToCountryLeftWithBaseFilter() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -1936,8 +1838,8 @@ public void test_makeCursor_factToCountryInnerWithBaseFilter() VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -1971,8 +1873,8 @@ public void test_makeCursor_factToCountryRightWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -1985,7 +1887,7 @@ public void test_makeCursor_factToCountryRightWithBaseFilter() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), + cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN), columns, ImmutableList.of( new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, @@ -2026,8 +1928,8 @@ public void test_makeCursor_factToCountryFullWithBaseFilter() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -2040,7 +1942,7 @@ public void test_makeCursor_factToCountryFullWithBaseFilter() FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), + cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN), columns, ImmutableList.of( new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, @@ -2071,111 +1973,100 @@ public void test_makeCursor_factToCountryFullWithBaseFilter() @Test public void test_hasBuiltInFiltersForSingleJoinableClauseWithVariousJoinTypes() { - Assert.assertTrue(makeFactToCountrySegment(JoinType.INNER).hasBuiltInFilters()); - Assert.assertFalse(makeFactToCountrySegment(JoinType.LEFT).hasBuiltInFilters()); - Assert.assertFalse(makeFactToCountrySegment(JoinType.RIGHT).hasBuiltInFilters()); - Assert.assertFalse(makeFactToCountrySegment(JoinType.FULL).hasBuiltInFilters()); + Assert.assertFalse(makeFactToCountrySegment(JoinType.INNER).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + Assert.assertTrue(makeFactToCountrySegment(JoinType.LEFT).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + Assert.assertFalse(makeFactToCountrySegment(JoinType.RIGHT).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + Assert.assertTrue(makeFactToCountrySegment(JoinType.FULL).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); // cross join - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( + HashJoinSegment segment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + "'true'", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - "'true'", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - null - ).hasBuiltInFilters() - ); - } - - @Test - public void test_hasBuiltInFiltersForEmptyJoinableClause() - { - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(), - null - ).hasBuiltInFilters() + ) + ), + null ); + TopNOptimizationInspector inspector = segment.as(TopNOptimizationInspector.class); + Assert.assertTrue(inspector.areAllDictionaryIdsPresent()); } @Test public void test_hasBuiltInFiltersForConvertedJoin() { - Assert.assertTrue( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - new InDimFilter("dim", ImmutableSet.of("foo", "bar")), - ImmutableList.of(), - null - ).hasBuiltInFilters() + final HashJoinSegment segment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + new InDimFilter("dim", ImmutableSet.of("foo", "bar")), + ImmutableList.of(), + null ); + final TopNOptimizationInspector inspector = segment.as(TopNOptimizationInspector.class); + Assert.assertFalse(inspector.areAllDictionaryIdsPresent()); } @Test public void test_hasBuiltInFiltersForMultipleJoinableClausesWithVariousJoinTypes() { - Assert.assertTrue( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.INNER), - regionToCountry(JoinType.LEFT) - ), - null - ).hasBuiltInFilters() - ); - - Assert.assertTrue( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.RIGHT), - regionToCountry(JoinType.INNER), - factToCountryOnNumber(JoinType.FULL) - ), - null - ).hasBuiltInFilters() - ); - - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ), - null - ).hasBuiltInFilters() - ); - - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - new JoinableClause( + final HashJoinSegment segment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.INNER), + regionToCountry(JoinType.LEFT) + ), + null + ); + Assert.assertFalse(segment.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + + final HashJoinSegment segment2 = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.RIGHT), + regionToCountry(JoinType.INNER), + factToCountryOnNumber(JoinType.FULL) + ), + null + ); + Assert.assertFalse(segment2.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + + final HashJoinSegment segment3 = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ), + null + ); + Assert.assertTrue(segment3.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + + final HashJoinSegment segment4 = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.LEFT), + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + "'true'", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - "'true'", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - null - ).hasBuiltInFilters() + ) + ), + null ); + Assert.assertTrue(segment4.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java index 9f9905e47d47..55e245bdb1c3 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java @@ -23,12 +23,12 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.MaxIngestedEventTimeInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -163,9 +163,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return referencedSegment.asStorageAdapter(); + return referencedSegment.asCursorFactory(); } @Nullable @@ -234,14 +234,15 @@ public void test_getDataInterval() public void test_asQueryableIndex() { Assert.assertNull(hashJoinSegment.asQueryableIndex()); + Assert.assertNull(hashJoinSegment.as(QueryableIndex.class)); } @Test public void test_asStorageAdapter() { Assert.assertThat( - hashJoinSegment.asStorageAdapter(), - CoreMatchers.instanceOf(HashJoinSegmentStorageAdapter.class) + hashJoinSegment.asCursorFactory(), + CoreMatchers.instanceOf(HashJoinSegmentCursorFactory.class) ); } 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 656cbdb2ba24..e043e40fa48d 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 @@ -58,7 +58,7 @@ import java.util.List; import java.util.Set; -public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTest +public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentCursorFactoryTest { @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() @@ -75,14 +75,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -147,14 +148,15 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -198,14 +200,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -260,14 +263,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -326,14 +330,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -391,14 +396,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC virtualColumns ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) @@ -463,14 +469,15 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu joinableClauses.getJoinableClauses(), virtualColumns ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses.getJoinableClauses(), joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) @@ -565,14 +572,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -687,14 +695,15 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -777,13 +786,14 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan expectedException.expectMessage( "Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -821,14 +831,15 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -918,14 +929,15 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1003,14 +1015,15 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1072,14 +1085,15 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1125,14 +1139,15 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1177,14 +1192,15 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1228,14 +1244,15 @@ public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNo joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1280,14 +1297,15 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1330,8 +1348,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1339,7 +1358,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1389,8 +1408,9 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1398,7 +1418,7 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1447,14 +1467,15 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1497,14 +1518,15 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1546,14 +1568,15 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1599,14 +1622,15 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1651,14 +1675,15 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1702,14 +1727,15 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1764,14 +1790,15 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1830,14 +1857,15 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1898,8 +1926,9 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1913,7 +1942,7 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1962,14 +1991,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses.getJoinableClauses(), joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2037,14 +2067,15 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses.getJoinableClauses(), joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2206,8 +2237,9 @@ public boolean supportsRequiredColumnRewrite() joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -2242,7 +2274,7 @@ public boolean supportsRequiredColumnRewrite() } JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2364,14 +2396,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2439,14 +2472,15 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( 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 1a5e6dc63de4..ba194ace2c61 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 @@ -27,12 +27,15 @@ 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.CursorHolder; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.ReferenceCountingSegment; 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.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; import org.junit.Test; @@ -47,7 +50,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest +public class PostJoinCursorTest extends BaseHashJoinSegmentCursorFactoryTest { public QueryableIndexSegment infiniteFactSegment; @@ -58,45 +61,58 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest */ private static class TestInfiniteQueryableIndexSegment extends QueryableIndexSegment { - - private static class InfiniteQueryableIndexStorageAdapter extends QueryableIndexStorageAdapter + private static class InfiniteCursorFactory implements CursorFactory { + final CursorFactory delegate; CountDownLatch countDownLatch; - public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch countDownLatch) + public InfiniteCursorFactory(CursorFactory delegate, CountDownLatch countDownLatch) { - super(index); + this.delegate = delegate; this.countDownLatch = countDownLatch; } @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - final CursorHolder delegate = super.makeCursorHolder(spec); + final CursorHolder holder = delegate.makeCursorHolder(spec); return new CursorHolder() { @Nullable @Override public Cursor asCursor() { - return new CursorNoAdvance(delegate.asCursor(), countDownLatch); + return new CursorNoAdvance(holder.asCursor(), countDownLatch); } @Nullable @Override public List getOrdering() { - return delegate.getOrdering(); + return holder.getOrdering(); } @Override public void close() { - delegate.close(); + holder.close(); } }; } + @Override + public RowSignature getRowSignature() + { + return delegate.getRowSignature(); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(column); + } + private static class CursorNoAdvance implements Cursor { Cursor cursor; @@ -149,18 +165,18 @@ public void reset() } } - private final StorageAdapter testStorageAdaptor; + private final CursorFactory cursorFactory; public TestInfiniteQueryableIndexSegment(QueryableIndex index, SegmentId segmentId, CountDownLatch countDownLatch) { super(index, segmentId); - testStorageAdaptor = new InfiniteQueryableIndexStorageAdapter(index, countDownLatch); + cursorFactory = new InfiniteCursorFactory(new QueryableIndexCursorFactory(index), countDownLatch); } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return testStorageAdaptor; + return cursorFactory; } } @@ -231,13 +247,14 @@ public void makeCursorAndAdvance() VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter = new HashJoinSegmentStorageAdapter( - infiniteFactSegment.asStorageAdapter(), + HashJoinSegment hashJoinSegment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(infiniteFactSegment), + null, joinableClauses, joinFilterPreAnalysis ); - try (final CursorHolder cursorHolder = hashJoinSegmentStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java index 9b8f7f4d9164..8df2bffb2fd6 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java @@ -42,8 +42,10 @@ import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestIndex; @@ -142,8 +144,8 @@ public void setup() throws IOException, SegmentLoadingException segment.getTotalSpace() ); backingSegment = (QueryableIndexSegment) factory.factorize(dataSegment, segment, false, SegmentLazyLoadFailCallback.NOOP); - columnNames = - new QueryableIndexStorageAdapter(backingSegment.asQueryableIndex()).getRowSignature().getColumnNames(); + columnNames = new QueryableIndexCursorFactory(backingSegment.as(QueryableIndex.class)).getRowSignature() + .getColumnNames(); broadcastTable = new BroadcastSegmentIndexedTable(backingSegment, keyColumns, dataSegment.getVersion()); } @@ -287,11 +289,11 @@ private void checkNonIndexedReader(String columnName) checkColumnSelectorFactory(columnName); try (final Closer closer = Closer.create()) { final int columnIndex = columnNames.indexOf(columnName); - final int numRows = backingSegment.asStorageAdapter().getNumRows(); + final int numRows = backingSegment.as(PhysicalSegmentInspector.class).getNumRows(); final IndexedTable.Reader reader = broadcastTable.columnReader(columnIndex); closer.register(reader); final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows); - final BaseColumn theColumn = backingSegment.asQueryableIndex() + final BaseColumn theColumn = backingSegment.as(QueryableIndex.class) .getColumnHolder(columnName) .getColumn(); closer.register(theColumn); @@ -317,10 +319,10 @@ private void checkNonIndexedReader(String columnName) private void checkColumnSelectorFactory(String columnName) { try (final Closer closer = Closer.create()) { - final int numRows = backingSegment.asStorageAdapter().getNumRows(); + final int numRows = backingSegment.as(PhysicalSegmentInspector.class).getNumRows(); final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows); - final BaseColumn theColumn = backingSegment.asQueryableIndex() + final BaseColumn theColumn = backingSegment.as(QueryableIndex.class) .getColumnHolder(columnName) .getColumn(); closer.register(theColumn); diff --git a/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java index 557bf4b8dfc4..8a877d6f8ed7 100644 --- a/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java @@ -21,20 +21,27 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; + import static org.junit.Assert.assertThrows; public class TombstoneSegmentizerFactoryTest { - @Test public void testSegmentCreation() { @@ -50,11 +57,11 @@ public void testSegmentCreation() .build(); Segment segment = factory.factorize(tombstone, null, true, null); - Assert.assertNotNull(segment.asStorageAdapter()); + Assert.assertNotNull(segment.asCursorFactory()); Assert.assertEquals("foo_2021-01-01T00:00:00.000Z_2022-01-01T00:00:00.000Z_1", segment.getId().toString()); Assert.assertEquals(expectedInterval, segment.getDataInterval()); - QueryableIndex queryableIndex = segment.asQueryableIndex(); + QueryableIndex queryableIndex = segment.as(QueryableIndex.class); Assert.assertNotNull(queryableIndex); assertThrows(UnsupportedOperationException.class, queryableIndex::getNumRows); assertThrows(UnsupportedOperationException.class, queryableIndex::getAvailableDimensions); @@ -63,10 +70,77 @@ public void testSegmentCreation() assertThrows(UnsupportedOperationException.class, queryableIndex::getDimensionHandlers); assertThrows(UnsupportedOperationException.class, queryableIndex::getColumnNames); assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder(null)); + Assert.assertTrue(segment.isTombstone()); + + Assert.assertNotNull(segment.asCursorFactory()); + + } + + @Test + public void testDefaultSegmentIsTombstoneFalse() + { + Segment segment = new Segment() + { + @Override + public SegmentId getId() + { + return null; + } + + @Override + public Interval getDataInterval() + { + return null; + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + return null; + } + + @Override + public CursorFactory asCursorFactory() + { + return new CursorFactory() + { + @Override + public CursorHolder makeCursorHolder(CursorBuildSpec spec) + { + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + return null; + } + }; + } + + @Override + public RowSignature getRowSignature() + { + return RowSignature.empty(); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return null; + } + }; + } + + @Override + public void close() + { - StorageAdapter storageAdapter = segment.asStorageAdapter(); - Assert.assertNotNull(storageAdapter); - Assert.assertTrue(storageAdapter.isFromTombstone()); + } + }; + Assert.assertFalse(segment.isTombstone()); } } 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 fd89e083441a..377db0f4f54a 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 @@ -37,7 +37,6 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.TransformSpec; @@ -344,11 +343,11 @@ private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns vir IndexSpec.DEFAULT ); Assert.assertEquals(1, segments.size()); - StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); + Segment segment = segments.get(0); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); + final CursorHolder cursorHolder = closer.register(segment.asCursorFactory().makeCursorHolder(buildSpec)); final Cursor cursor = cursorHolder.asCursor(); return cursor.getColumnSelectorFactory(); } @@ -369,9 +368,9 @@ private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumn IndexSpec.DEFAULT ); Assert.assertEquals(1, segments.size()); - StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); + Segment segment = segments.get(0); final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build(); - VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor(); + VectorCursor cursor = closer.register(segment.asCursorFactory().makeCursorHolder(buildSpec)).asVectorCursor(); return cursor.getColumnSelectorFactory(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java index 2f4c93e862ed..704ef9bf11aa 100644 --- a/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java @@ -21,7 +21,7 @@ import org.apache.druid.collections.bitmap.WrappedRoaringBitmap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.query.QueryContexts; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -35,7 +35,7 @@ public class NilVectorSelectorTest extends InitializedNullHandlingTest @Test public void testDefaultSizedVector() { - testVectorSize(QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE); + testVectorSize(QueryContexts.DEFAULT_VECTOR_SIZE); } @Test 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 13f82e2be8ed..632a848830d8 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -44,11 +44,11 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestObjectColumnSelector; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -58,8 +58,8 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -79,10 +79,10 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest { private static Closer CLOSER; private static QueryableIndex QUERYABLE_INDEX; - private static QueryableIndexStorageAdapter QUERYABLE_INDEX_STORAGE_ADAPTER; + private static QueryableIndexCursorFactory QUERYABLE_INDEX_CURSOR_FACTORY; private static IncrementalIndex INCREMENTAL_INDEX; - private static IncrementalIndexStorageAdapter INCREMENTAL_INDEX_STORAGE_ADAPTER; - private static List ADAPTERS; + private static IncrementalIndexCursorFactory INCREMENTAL_INDEX_CURSOR_FACTORY; + private static List CURSOR_FACTORIES; private static final ColumnCapabilities SINGLE_VALUE = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true) @@ -116,14 +116,17 @@ public static void setup() INCREMENTAL_INDEX = CLOSER.register( segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, numRows) ); - INCREMENTAL_INDEX_STORAGE_ADAPTER = new IncrementalIndexStorageAdapter(INCREMENTAL_INDEX); + INCREMENTAL_INDEX_CURSOR_FACTORY = new IncrementalIndexCursorFactory(INCREMENTAL_INDEX); QUERYABLE_INDEX = CLOSER.register( segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, numRows) ); - QUERYABLE_INDEX_STORAGE_ADAPTER = new QueryableIndexStorageAdapter(QUERYABLE_INDEX); + QUERYABLE_INDEX_CURSOR_FACTORY = new QueryableIndexCursorFactory(QUERYABLE_INDEX); - ADAPTERS = ImmutableList.of(INCREMENTAL_INDEX_STORAGE_ADAPTER, QUERYABLE_INDEX_STORAGE_ADAPTER); + CURSOR_FACTORIES = ImmutableList.of( + INCREMENTAL_INDEX_CURSOR_FACTORY, + QUERYABLE_INDEX_CURSOR_FACTORY + ); } @AfterClass @@ -137,7 +140,7 @@ public static void teardown() public void test_single_value_string_bindings() { final String columnName = "string3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); @@ -202,7 +205,7 @@ public void test_single_value_string_bindings() public void test_multi_value_string_bindings() { final String columnName = "multi-string3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -279,7 +282,7 @@ public void test_multi_value_string_bindings() public void test_long_bindings() { final String columnName = "long3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -324,7 +327,7 @@ public void test_long_bindings() public void test_double_bindings() { final String columnName = "double3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -643,8 +646,8 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept ) ); - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector xExprSelector = ExpressionSelectors.makeDimensionSelector( cursor.getColumnSelectorFactory(), diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java index 6fbee67734f6..e103413ae9cd 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java @@ -26,7 +26,7 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -116,11 +116,11 @@ public static void testCast( ) ) ); - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor(); + VectorCursor cursor = closer.register(cursorFactory.makeCursorHolder(buildSpec)).asVectorCursor(); ColumnCapabilities capabilities = INDEX.getColumnCapabilities(column); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java index 5d284a741297..142230593db6 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 @@ -40,7 +40,7 @@ import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.StringEncodingStrategy; @@ -234,15 +234,15 @@ public static void sanityTestVectorizedExpressionSelectors( ) ) ); - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final VectorCursor cursor = cursorHolder.asVectorCursor(); Assert.assertNotNull(cursor); - ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); + ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(cursorFactory, "v"); int rowCount = 0; if (capabilities.isDictionaryEncoded().isTrue()) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java index b051fa72724d..e9f6e3a2ade7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java @@ -20,14 +20,15 @@ package org.apache.druid.segment.realtime; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.IncrementalIndexSegment; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -76,10 +77,16 @@ public SegmentId getSegmentId() public int getSegmentNumDimensionColumns() { - final Segment segment = adapter.get().getBaseSegment(); - if (segment != null) { - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - return storageAdapter.getAvailableDimensions().size(); + if (hasSwapped()) { + final Segment segment = adapter.get().getBaseSegment(); + if (segment != null) { + QueryableIndex queryableIndex = segment.as(QueryableIndex.class); + if (queryableIndex != null) { + return queryableIndex.getAvailableDimensions().size(); + } + } + } else { + return index.getDimensions().size(); } return 0; } @@ -88,8 +95,9 @@ public int getSegmentNumMetricColumns() { final Segment segment = adapter.get().getBaseSegment(); if (segment != null) { - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - return Iterables.size(storageAdapter.getAvailableMetrics()); + final PhysicalSegmentInspector segmentInspector = segment.as(PhysicalSegmentInspector.class); + final Metadata metadata = segmentInspector == null ? null : segmentInspector.getMetadata(); + return metadata != null && metadata.getAggregators() != null ? metadata.getAggregators().length : 0; } return 0; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java b/server/src/main/java/org/apache/druid/segment/realtime/WindowedCursorFactory.java similarity index 74% rename from server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java rename to server/src/main/java/org/apache/druid/segment/realtime/WindowedCursorFactory.java index 9b4bd1e4ea7f..63eb32a02533 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/WindowedCursorFactory.java @@ -19,23 +19,23 @@ package org.apache.druid.segment.realtime; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.joda.time.Interval; -public class WindowedStorageAdapter +public class WindowedCursorFactory { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; private final Interval interval; - public WindowedStorageAdapter(StorageAdapter adapter, Interval interval) + public WindowedCursorFactory(CursorFactory cursorFactory, Interval interval) { - this.adapter = adapter; + this.cursorFactory = cursorFactory; this.interval = interval; } - public StorageAdapter getAdapter() + public CursorFactory getCursorFactory() { - return adapter; + return cursorFactory; } public Interval getInterval() @@ -46,8 +46,8 @@ public Interval getInterval() @Override public String toString() { - return "WindowedStorageAdapter{" + - "adapter=" + adapter + + return "WindowedCursorFactory{" + + "cursorFactory=" + cursorFactory + ", interval=" + interval + '}'; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 979d8cb92b3d..bb5acec7b8f4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -816,7 +816,7 @@ private DataSegmentWithMetadata mergeAndPush( try { for (FireHydrant fireHydrant : sink) { Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); - final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); + final QueryableIndex queryableIndex = segmentAndCloseable.lhs.as(QueryableIndex.class); if (queryableIndex != null) { rowsinMergedSegment += queryableIndex.getNumRows(); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index a25d6b7acd77..0d7d01253c84 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -934,7 +934,7 @@ private DataSegmentWithMetadata mergeAndPush( try { for (FireHydrant fireHydrant : sink) { Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); - final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); + final QueryableIndex queryableIndex = segmentAndCloseable.lhs.as(QueryableIndex.class); log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); indexes.add(queryableIndex); closer.register(segmentAndCloseable.rhs); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java index 8897390a7464..980f9bf4d8e7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java @@ -22,10 +22,9 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SchemaPayloadPlus; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import java.io.File; @@ -41,10 +40,9 @@ public class TaskSegmentSchemaUtil public static SchemaPayloadPlus getSegmentSchema(File segmentFile, IndexIO indexIO) throws IOException { final QueryableIndex queryableIndex = indexIO.loadIndex(segmentFile); - final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex); - final RowSignature rowSignature = storageAdapter.getRowSignature(); - final long numRows = storageAdapter.getNumRows(); - final AggregatorFactory[] aggregatorFactories = storageAdapter.getMetadata().getAggregators(); + final RowSignature rowSignature = new QueryableIndexCursorFactory(queryableIndex).getRowSignature(); + final long numRows = queryableIndex.getNumRows(); + final AggregatorFactory[] aggregatorFactories = queryableIndex.getMetadata().getAggregators(); Map aggregatorFactoryMap = new HashMap<>(); if (null != aggregatorFactories) { for (AggregatorFactory aggregatorFactory : aggregatorFactories) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java index 5e59c0e9b15f..98c5e11fa0b2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.realtime.sink; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; @@ -33,10 +34,9 @@ import org.apache.druid.query.Query; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -44,7 +44,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.FireHydrant; @@ -157,8 +156,8 @@ public Sink( maxCount = hydrant.getCount(); ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); try { - QueryableIndex index = segment.asQueryableIndex(); - overwriteIndexDimensions(new QueryableIndexStorageAdapter(index)); + overwriteIndexDimensions(segment); + QueryableIndex index = segment.as(QueryableIndex.class); numRowsExcludingCurrIndex.addAndGet(index.getNumRows()); } finally { @@ -287,7 +286,7 @@ public int getNumRowsInMemory() return 0; } - return currHydrant.getIndex().size(); + return index.size(); } } @@ -299,7 +298,7 @@ public long getBytesInMemory() return 0; } - return currHydrant.getIndex().getBytesInMemory().get(); + return index.getBytesInMemory().get(); } } @@ -354,11 +353,11 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) if (lastHydrant.hasSwapped()) { oldFormat = new HashMap<>(); - ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); + final ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); try { - QueryableIndex oldIndex = segment.asQueryableIndex(); - overwriteIndexDimensions(new QueryableIndexStorageAdapter(oldIndex)); + overwriteIndexDimensions(segment); if (variableDimensions) { + final QueryableIndex oldIndex = Preconditions.checkNotNull(segment.as(QueryableIndex.class)); for (String dim : oldIndex.getAvailableDimensions()) { dimOrder.add(dim); oldFormat.put(dim, oldIndex.getColumnHolder(dim).getColumnFormat()); @@ -369,9 +368,9 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) segment.decrement(); } } else { - IncrementalIndex oldIndex = lastHydrant.getIndex(); - overwriteIndexDimensions(new IncrementalIndexStorageAdapter(oldIndex)); + overwriteIndexDimensions(lastHydrant.getHydrantSegment()); if (variableDimensions) { + IncrementalIndex oldIndex = lastHydrant.getIndex(); dimOrder.addAll(oldIndex.getDimensionOrder()); oldFormat = oldIndex.getColumnFormats(); } @@ -399,9 +398,9 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) * Merge the column from the index with the existing columns. */ @GuardedBy("hydrantLock") - private void overwriteIndexDimensions(StorageAdapter storageAdapter) + private void overwriteIndexDimensions(Segment segment) { - RowSignature rowSignature = storageAdapter.getRowSignature(); + RowSignature rowSignature = segment.asCursorFactory().getRowSignature(); for (String dim : rowSignature.getColumnNames()) { columnsExcludingCurrIndex.add(dim); rowSignature.getColumnType(dim).ifPresent(type -> columnTypeExcludingCurrIndex.put(dim, type)); @@ -422,9 +421,7 @@ public RowSignature getSignature() } // Add columns from the currHydrant that do not yet exist in columnsExcludingCurrIndex. - IncrementalIndexStorageAdapter currStorageAdapter = - new IncrementalIndexStorageAdapter(currHydrant.getIndex()); - RowSignature currSignature = currStorageAdapter.getRowSignature(); + RowSignature currSignature = currHydrant.getHydrantSegment().asCursorFactory().getRowSignature(); for (String dim : currSignature.getColumnNames()) { if (!columnsExcludingCurrIndex.contains(dim)) { diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index 672e4c23aa8e..baf1c298348d 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -29,9 +29,9 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.ReferenceCountingIndexedTable; import org.apache.druid.segment.loading.SegmentCacheManager; @@ -343,8 +343,13 @@ private void loadSegment( dataSegment.getVersion(), dataSegment.getShardSpec().createChunk(segment) ); - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + final PhysicalSegmentInspector countInspector = segment.as(PhysicalSegmentInspector.class); + final long numOfRows; + if (dataSegment.isTombstone() || countInspector == null) { + numOfRows = 0; + } else { + numOfRows = countInspector.getNumRows(); + } dataSourceState.addSegment(dataSegment, numOfRows); pageCacheLoadFunction.accept(dataSegment); @@ -387,8 +392,13 @@ public void dropSegment(final DataSegment segment) if (oldQueryable != null) { try (final Closer closer = Closer.create()) { - StorageAdapter storageAdapter = oldQueryable.asStorageAdapter(); - long numOfRows = (segment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + final PhysicalSegmentInspector countInspector = oldQueryable.as(PhysicalSegmentInspector.class); + final long numOfRows; + if (segment.isTombstone() || countInspector == null) { + numOfRows = 0; + } else { + numOfRows = countInspector.getNumRows(); + } dataSourceState.removeSegment(segment, numOfRows); closer.register(oldQueryable); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index f3264dad44cf..4bee18091f24 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -58,7 +58,6 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.ResourceIdPopulatingQueryRunner; @@ -278,8 +277,11 @@ private QueryRunner buildAndDecorateQueryRunner( final AtomicLong cpuTimeAccumulator ) { - - + // Short-circuit when the index comes from a tombstone (it has no data by definition), + // check for null also since no all segments (higher level ones) will have QueryableIndex... + if (segment.isTombstone()) { + return new NoopQueryRunner<>(); + } final SpecificSegmentSpec segmentSpec = new SpecificSegmentSpec(segmentDescriptor); final SegmentId segmentId = segment.getId(); @@ -291,12 +293,6 @@ private QueryRunner buildAndDecorateQueryRunner( return new ReportTimelineMissingSegmentQueryRunner<>(segmentDescriptor); } - StorageAdapter storageAdapter = segment.asStorageAdapter(); - // Short-circuit when the index comes from a tombstone (it has no data by definition), - // check for null also since no all segments (higher level ones) will have QueryableIndex... - if (storageAdapter.isFromTombstone()) { - return new NoopQueryRunner<>(); - } String segmentIdString = segmentId.toString(); MetricsEmittingQueryRunner metricsEmittingQueryRunnerInner = new MetricsEmittingQueryRunner<>( diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java index 323ec7978904..1ca73d4b934e 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java @@ -29,10 +29,10 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.TestSegmentUtils; @@ -1251,11 +1251,11 @@ public void testGetTombstoneSegment() throws SegmentLoadingException Assert.assertEquals(tombstone.getId(), segment.getId()); Assert.assertEquals(interval, segment.getDataInterval()); - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - Assert.assertNotNull(storageAdapter); - Assert.assertTrue(storageAdapter.isFromTombstone()); + final CursorFactory cursorFactory = segment.asCursorFactory(); + Assert.assertNotNull(cursorFactory); + Assert.assertTrue(segment.isTombstone()); - final QueryableIndex queryableIndex = segment.asQueryableIndex(); + final QueryableIndex queryableIndex = segment.as(QueryableIndex.class); Assert.assertNotNull(queryableIndex); Assert.assertEquals(interval, queryableIndex.getDataInterval()); Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getMetadata); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java index 772a79ae0ad1..283c1687a972 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java @@ -56,8 +56,10 @@ import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; +import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SchemaPayloadPlus; import org.apache.druid.segment.SegmentMetadata; @@ -1540,8 +1542,8 @@ public void testSchemaBackfilling() throws InterruptedException config ); - QueryableIndexStorageAdapter index1StorageAdaptor = new QueryableIndexStorageAdapter(index1); - QueryableIndexStorageAdapter index2StorageAdaptor = new QueryableIndexStorageAdapter(index2); + QueryableIndexCursorFactory index1CursorFactory = new QueryableIndexCursorFactory(index1); + QueryableIndexCursorFactory index2CursorFactory = new QueryableIndexCursorFactory(index2); MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); @@ -1558,27 +1560,27 @@ public void testSchemaBackfilling() throws InterruptedException pluses.add(new SegmentSchemaManager.SegmentSchemaMetadataPlus( segment1.getId(), fingerprintGenerator.generateFingerprint( - new SchemaPayload(index1StorageAdaptor.getRowSignature()), + new SchemaPayload(index1CursorFactory.getRowSignature()), segment1.getDataSource(), CentralizedDatasourceSchemaConfig.SCHEMA_VERSION ), new SchemaPayloadPlus( new SchemaPayload( - index1StorageAdaptor.getRowSignature()), - (long) index1StorageAdaptor.getNumRows() + index1CursorFactory.getRowSignature()), + (long) index1.getNumRows() ) )); pluses.add(new SegmentSchemaManager.SegmentSchemaMetadataPlus( segment2.getId(), fingerprintGenerator.generateFingerprint( - new SchemaPayload(index2StorageAdaptor.getRowSignature()), + new SchemaPayload(index2CursorFactory.getRowSignature()), segment1.getDataSource(), CentralizedDatasourceSchemaConfig.SCHEMA_VERSION ), new SchemaPayloadPlus( new SchemaPayload( - index2StorageAdaptor.getRowSignature()), - (long) index2StorageAdaptor.getNumRows() + index2CursorFactory.getRowSignature()), + (long) index2.getNumRows() ) )); @@ -1684,9 +1686,9 @@ public void refresh(Set segmentsToRefresh, Set dataSourcesToR try { SchemaPayload schemaPayload = mapper.readValue(r.getBytes(1), SchemaPayload.class); long numRows = r.getLong(2); - QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index2); - Assert.assertEquals(adapter.getRowSignature(), schemaPayload.getRowSignature()); - Assert.assertEquals(adapter.getNumRows(), numRows); + QueryableIndexCursorFactory cursorFa = new QueryableIndexCursorFactory(index2); + Assert.assertEquals(cursorFa.getRowSignature(), schemaPayload.getRowSignature()); + Assert.assertEquals(index2.getNumRows(), numRows); } catch (IOException e) { throw new RuntimeException(e); @@ -1711,12 +1713,14 @@ public void testSameSegmentAddedOnMultipleServer() throws InterruptedException, config.setDisableSegmentMetadataQueries(true); CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(config); - QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index2); + QueryableIndexSegment queryableIndexSegment = new QueryableIndexSegment(index2, SegmentId.dummy("test")); + PhysicalSegmentInspector rowCountInspector = queryableIndexSegment.as(PhysicalSegmentInspector.class); + QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index2); ImmutableMap.Builder segmentStatsMap = new ImmutableMap.Builder<>(); - segmentStatsMap.put(segment3.getId(), new SegmentMetadata((long) adapter.getNumRows(), "fp")); + segmentStatsMap.put(segment3.getId(), new SegmentMetadata((long) rowCountInspector.getNumRows(), "fp")); ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); - schemaPayloadMap.put("fp", new SchemaPayload(adapter.getRowSignature())); + schemaPayloadMap.put("fp", new SchemaPayload(cursorFactory.getRowSignature())); segmentSchemaCache.updateFinalizedSegmentSchema( new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) ); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java index 38c3fda1e7e2..79f31c7c814e 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java @@ -21,12 +21,12 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -197,9 +197,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return incrementalIndexSegment.asStorageAdapter(); + return incrementalIndexSegment.asCursorFactory(); } @Override diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index cc0dc1fad1bc..cd990e76f892 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.realtime.appenderator; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -29,6 +30,8 @@ import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.guice.BuiltInTypesModule; +import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -36,6 +39,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.ForwardingQueryProcessingPool; @@ -43,6 +47,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryConfig; import org.apache.druid.query.scan.ScanQueryEngine; @@ -110,6 +115,13 @@ public StreamAppenderatorTester( { objectMapper = new DefaultObjectMapper(); objectMapper.registerSubtypes(LinearShardSpec.class); + objectMapper.registerModules(new AggregatorsModule()); + objectMapper.registerModules(new BuiltInTypesModule().getJacksonModules()); + objectMapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), objectMapper) + ); final Map parserMap = objectMapper.convertValue( new MapInputRowParser( diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java index 07ed8b17e6b1..55a882c32188 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -31,10 +31,11 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; @@ -246,7 +247,7 @@ public Segment factorize(DataSegment segment, File parentDir, boolean lazy, Segm { return new Segment() { - StorageAdapter storageAdapter = Mockito.mock(StorageAdapter.class); + PhysicalSegmentInspector rowCountInspector = Mockito.mock(PhysicalSegmentInspector.class); @Override public SegmentId getId() @@ -268,15 +269,18 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - Mockito.when(storageAdapter.getNumRows()).thenReturn(1); - return storageAdapter; + throw new UnsupportedOperationException(); } @Override public T as(Class clazz) { + if (PhysicalSegmentInspector.class.equals(clazz)) { + Mockito.when(rowCountInspector.getNumRows()).thenReturn(1); + return (T) rowCountInspector; + } return null; } 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 866381dd70cf..7cc61d940945 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -28,18 +28,15 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.OrderBy; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.loading.LoadSpec; @@ -242,9 +239,9 @@ public QueryableIndex asQueryableIndex() } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return makeFakeStorageAdapter(interval, 0); + return new QueryableIndexCursorFactory(index); } @Override @@ -254,85 +251,6 @@ public void close() closed = true; } } - - private StorageAdapter makeFakeStorageAdapter(Interval interval, int cardinality) - { - StorageAdapter adapter = new StorageAdapter() - { - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public int getDimensionCardinality(String column) - { - return cardinality; - } - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public Metadata getMetadata() - { - return null; - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new CursorHolder() - { - @Nullable - @Override - public Cursor asCursor() - { - return null; - } - }; - } - }; - - return adapter; - } } public static DataSegment makeSegment(String dataSource, String version, Interval interval) 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 3f8cfcb94a45..5c31f02b6039 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -75,8 +75,8 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnHolder; @@ -295,14 +295,14 @@ public static void runDump( throws IOException { final ObjectMapper objectMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); final DimFilter filter = filterJson != null ? objectMapper.readValue(filterJson, DimFilter.class) : null; final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(filter)) .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return; diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index f17d51291433..fc7e9d333238 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -133,7 +133,7 @@ public void testDumpRows() throws Exception Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); - QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); File outputFile = tempFolder.newFile(); @@ -147,11 +147,11 @@ public void testDumpRows() throws Exception ); final byte[] fileBytes = Files.readAllBytes(outputFile.toPath()); final String output = StringUtils.fromUtf8(fileBytes); - final String expected = "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":200,\"y\":2.2}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":400,\"y\":1.1,\"z\":\"a\"}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":200,\"z\":\"b\"}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":100,\"y\":1.1,\"z\":\"a\"}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"y\":3.3,\"z\":\"b\"}}\n"; + final String expected = "{\"__time\":1609459200000,\"nest\":{\"x\":200,\"y\":2.2},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"x\":400,\"y\":1.1,\"z\":\"a\"},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"x\":200,\"z\":\"b\"},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"x\":100,\"y\":1.1,\"z\":\"a\"},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"y\":3.3,\"z\":\"b\"},\"count\":1}\n"; Assert.assertEquals(expected, output); } @@ -207,7 +207,7 @@ public void testDumpNestedColumn() throws Exception Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); - QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); File outputFile = tempFolder.newFile(); DumpSegment.runDumpNestedColumn( @@ -247,7 +247,7 @@ public void testDumpNestedColumnPath() throws Exception Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); - QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); File outputFile = tempFolder.newFile(); DumpSegment.runDumpNestedColumnPath( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java index ee58446b5ab3..e5c403f1e1de 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java @@ -463,7 +463,7 @@ private static RowSignature getCorrelateRowSignature( ) { // Compute signature of the correlation operation. It's like a join: the left and right sides are concatenated. - // On the native query side, this is what is ultimately emitted by the UnnestStorageAdapter. + // On the native query side, this is what is ultimately emitted by the UnnestSegment. // // Ignore prefix (lhs) from computeJoinRowSignature; we don't need this since we will declare the name of the // single output column directly. (And we know it's the last column in the signature.) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 51cf9e923edb..3ce33e722452 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -85,7 +85,6 @@ import org.apache.druid.query.topn.NumericTopNMetricSpec; import org.apache.druid.query.topn.TopNMetricSpec; import org.apache.druid.query.topn.TopNQuery; -import org.apache.druid.segment.RowBasedStorageAdapter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -916,12 +915,12 @@ private static boolean canUseQueryGranularity( ) { if (Granularities.ALL.equals(queryGranularity)) { - // Always OK: no storage adapter has problem with ALL. + // Always OK: no cursor factory has problem with ALL. return true; } if (dataSource.getAnalysis().isConcreteAndTableBased()) { - // Always OK: queries on concrete tables (regular Druid datasources) use segment-based storage adapters + // Always OK: queries on concrete tables (regular Druid datasources) use segment-based cursors // (IncrementalIndex or QueryableIndex). These clip query interval to data interval, making wide query // intervals safer. They do not have special checks for granularity and interval safety. return true; @@ -931,9 +930,9 @@ private static boolean canUseQueryGranularity( // count on interval-clipping to save us. for (final Interval filtrationInterval : filtration.getIntervals()) { - // Query may be using RowBasedStorageAdapter. We don't know for sure, so check - // RowBasedStorageAdapter#isQueryGranularityAllowed to be safe. - if (!RowBasedStorageAdapter.isQueryGranularityAllowed(filtrationInterval, queryGranularity)) { + // Query may be using RowBasedCursorFactory. We don't know for sure, so check if the interval is too big + + if (Intervals.ETERNITY.equals(filtrationInterval)) { return false; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 04113067cb1f..aabc90f587d7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -3508,7 +3508,7 @@ public void testJoinOnConstantShouldFail(Map queryContext) @Test public void testLeftJoinRightTableCanBeEmpty() { - // HashJoinSegmentStorageAdapter is not vectorizable + // HashJoinSegmentCursorFactory is not vectorizable cannotVectorize(); final DataSource rightTable; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 58d29a738c09..a8dcc35ea7ad 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -15550,6 +15550,7 @@ public void testFilterWithNvlishCase2AndNotIn() ) ); } + @Test public void testGroupByDateTrunc() { @@ -16085,11 +16086,9 @@ public void testScanAndSortOnJoin() .run(); } - @NotYetSupported(Modes.CANNOT_RETRIEVE_ROWS) @Test public void testWindowingOverJoin() { - msqIncompatible(); testBuilder() .sql("with " + "main as " diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index e5442a2bda24..da1431f433d4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -89,7 +89,6 @@ enum Modes RESULT_MISMATCH(AssertionError.class, "(assertResulEquals|AssertionError: column content mismatch)"), LONG_CASTING(AssertionError.class, "expected: java.lang.Long"), UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"), - CANNOT_RETRIEVE_ROWS(UnsupportedOperationException.class, "Cannot retrieve number of rows from join segment"), UNION_WITH_COMPLEX_OPERAND(DruidException.class, "Only Table and Values are supported as inputs for Union"), UNION_MORE_STRICT_ROWTYPE_CHECK(DruidException.class, "Row signature mismatch in Union inputs"), JOIN_CONDITION_NOT_PUSHED_CONDITION(DruidException.class, "SQL requires a join with '.*' condition"), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java index 10504ca49d55..d9b24ed011dc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java @@ -53,7 +53,7 @@ import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -220,10 +220,10 @@ public void refresh( @Test public void testCoordinatorReturnsAllDSSchema() throws InterruptedException { - final RowSignature dataSource1RowSignature = new QueryableIndexStorageAdapter(index1).getRowSignature(); - final RowSignature dataSource2RowSignature = new QueryableIndexStorageAdapter(index2).getRowSignature(); - final RowSignature someDataSourceRowSignature = new QueryableIndexStorageAdapter(indexAuto1).getRowSignature(); - final RowSignature foo3RowSignature = new QueryableIndexStorageAdapter(indexAuto2).getRowSignature(); + final RowSignature dataSource1RowSignature = new QueryableIndexCursorFactory(index1).getRowSignature(); + final RowSignature dataSource2RowSignature = new QueryableIndexCursorFactory(index2).getRowSignature(); + final RowSignature someDataSourceRowSignature = new QueryableIndexCursorFactory(indexAuto1).getRowSignature(); + final RowSignature foo3RowSignature = new QueryableIndexCursorFactory(indexAuto2).getRowSignature(); NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { @Override @@ -272,9 +272,9 @@ public ListenableFuture> fetchDataSourceInformation( @Test public void testCoordinatorReturnsFewDSSchema() throws InterruptedException { - final RowSignature dataSource1RowSignature = new QueryableIndexStorageAdapter(index1).getRowSignature(); - final RowSignature dataSource2RowSignature = new QueryableIndexStorageAdapter(index2).getRowSignature(); - final RowSignature someDataSourceRowSignature = new QueryableIndexStorageAdapter(indexAuto1).getRowSignature(); + final RowSignature dataSource1RowSignature = new QueryableIndexCursorFactory(index1).getRowSignature(); + final RowSignature dataSource2RowSignature = new QueryableIndexCursorFactory(index2).getRowSignature(); + final RowSignature someDataSourceRowSignature = new QueryableIndexCursorFactory(indexAuto1).getRowSignature(); NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { @Override