From c462e103b6e837954633b5d12ca49ea917dd2835 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 9 Sep 2024 21:43:41 -0700 Subject: [PATCH] transition away from StorageAdapter (#16985) (#17024) * transition away from StorageAdapter changes: * CursorHolderFactory has been renamed to CursorFactory and moved off of StorageAdapter, instead fetched directly from the segment via 'asCursorFactory'. The previous deprecated CursorFactory interface has been merged into StorageAdapter * StorageAdapter is no longer used by any engines or tests and has been marked as deprecated with default implementations of all methods that throw exceptions indicating the new methods to call instead * StorageAdapter methods not covered by CursorFactory (CursorHolderFactory prior to this change) have been moved into interfaces which are retrieved by Segment.as, the primary classes are the previously existing Metadata, as well as new interfaces PhysicalSegmentInspector and TopNOptimizationInspector * added UnnestSegment and FilteredSegment that extend WrappedSegmentReference since their StorageAdapter implementations were previously provided by WrappedSegmentReference * added PhysicalSegmentInspector which covers some of the previous StorageAdapter functionality which was primarily used for segment metadata queries and other metadata uses, and is implemented for QueryableIndexSegment and IncrementalIndexSegment * added TopNOptimizationInspector to cover the oddly specific StorageAdapter.hasBuiltInFilters implementation, which is implemented for HashJoinSegment, UnnestSegment, and FilteredSegment * Updated all engines and tests to no longer use StorageAdapter --- .../ExpressionAggregationBenchmark.java | 7 +- .../benchmark/ExpressionFilterBenchmark.java | 10 +- .../ExpressionSelectorBenchmark.java | 51 +- .../ExpressionVectorSelectorBenchmark.java | 4 +- .../benchmark/FilterPartitionBenchmark.java | 64 +-- .../IndexedTableJoinCursorBenchmark.java | 2 +- .../benchmark/JoinAndLookupBenchmark.java | 28 +- .../compression/LongCompressionBenchmark.java | 4 +- .../frame/FrameChannelMergerBenchmark.java | 2 +- .../IncrementalIndexReadBenchmark.java | 15 +- .../druid/benchmark/query/SqlBenchmark.java | 10 +- .../query/SqlWindowFunctionsBenchmark.java | 10 +- .../DistinctCountTimeseriesQueryTest.java | 4 +- .../DistinctCountTopNQueryTest.java | 7 +- ...bjectVectorColumnProcessorFactoryTest.java | 10 +- .../query/filter/BloomDimFilterTest.java | 4 +- .../SegmentGeneratorFrameProcessor.java | 8 +- .../GroupByPreShuffleFrameProcessor.java | 4 +- .../results/ExportResultsFrameProcessor.java | 10 +- .../scan/ScanQueryFrameProcessor.java | 20 +- ...roadcastJoinSegmentMapFnProcessorTest.java | 20 +- .../msq/querykit/FrameProcessorTestBase.java | 12 +- ...WindowOperatorQueryFrameProcessorTest.java | 10 +- .../SortMergeJoinFrameProcessorTest.java | 12 +- .../QueryResultsFrameProcessorTest.java | 10 +- .../scan/ScanQueryFrameProcessorTest.java | 13 +- .../output/ChannelStageOutputReaderTest.java | 8 +- .../druid/msq/test/CalciteMSQTestsHelper.java | 8 +- .../apache/druid/msq/test/MSQTestBase.java | 17 +- .../hadoop/DatasourceRecordReader.java | 27 +- .../indexer/BatchDeltaIngestionTest.java | 8 +- ...tasourceRecordReaderSegmentReaderTest.java | 34 +- .../indexing/input/DruidSegmentReader.java | 23 +- .../common/task/CompactionTaskRunTest.java | 28 +- .../indexing/common/task/IndexTaskTest.java | 16 +- .../apache/druid/frame/field/RowReader.java | 4 +- .../frame/processor/FrameProcessors.java | 5 +- .../apache/druid/frame/read/FrameReader.java | 14 +- .../druid/frame/segment/FrameCursor.java | 8 +- .../druid/frame/segment/FrameSegment.java | 6 +- .../frame/segment/FrameStorageAdapter.java | 130 ----- ...y.java => ColumnarFrameCursorFactory.java} | 34 +- .../segment/columnar/FrameQueryableIndex.java | 2 +- ...actory.java => RowFrameCursorFactory.java} | 27 +- .../druid/query/FilteredDataSource.java | 10 +- .../query/FrameBasedInlineDataSource.java | 9 +- .../org/apache/druid/query/QueryContext.java | 3 +- .../org/apache/druid/query/QueryContexts.java | 1 + .../apache/druid/query/UnnestDataSource.java | 10 +- .../DataSourceMetadataQueryRunnerFactory.java | 2 +- .../apache/druid/query/filter/DimFilter.java | 2 +- .../query/filter/vector/VectorMatch.java | 4 +- .../groupby/GroupByQueryRunnerFactory.java | 8 +- .../druid/query/groupby/GroupingEngine.java | 30 +- .../epinephelinae/GroupByQueryEngine.java | 19 +- .../epinephelinae/GroupingSelector.java} | 16 +- .../column/GroupByColumnSelectorPlus.java | 16 +- .../vector/GroupByVectorColumnSelector.java | 3 +- .../NilGroupByVectorColumnSelector.java | 6 + ...alueStringGroupByVectorColumnSelector.java | 6 + .../vector/VectorGroupByEngine.java | 17 +- .../druid/query/metadata/SegmentAnalyzer.java | 46 +- .../SegmentMetadataQueryRunnerFactory.java | 20 +- .../query/planning/DataSourceAnalysis.java | 2 +- .../rowsandcols/ArrayListRowsAndColumns.java | 8 +- ....java => CursorFactoryRowsAndColumns.java} | 42 +- .../LazilyDecoratedRowsAndColumns.java | 10 +- .../ColumnBasedFrameRowsAndColumns.java | 8 +- .../QueryableIndexRowsAndColumns.java | 8 +- .../concrete/RowBasedFrameRowsAndColumns.java | 8 +- .../druid/query/scan/ScanQueryEngine.java | 20 +- .../query/search/CursorOnlyStrategy.java | 19 +- .../druid/query/search/SearchStrategy.java | 40 +- .../query/search/UseIndexesStrategy.java | 20 +- .../TimeBoundaryQueryRunnerFactory.java | 25 +- .../timeseries/TimeseriesQueryEngine.java | 10 +- .../TimeseriesQueryRunnerFactory.java | 13 +- .../AggregateTopNMetricFirstAlgorithm.java | 11 +- .../druid/query/topn/BaseTopNAlgorithm.java | 29 +- .../query/topn/HeapBasedTopNAlgorithm.java | 9 +- .../druid/query/topn/PooledTopNAlgorithm.java | 11 +- .../topn/TimeExtractionTopNAlgorithm.java | 5 +- .../druid/query/topn/TopNCursorInspector.java | 70 +++ .../apache/druid/query/topn/TopNMapFn.java | 20 +- .../druid/query/topn/TopNQueryEngine.java | 98 +++- .../query/topn/TopNQueryRunnerFactory.java | 4 +- ...eNumericTopNColumnAggregatesProcessor.java | 4 +- .../StringTopNColumnAggregatesProcessor.java | 8 +- .../types/TopNColumnAggregatesProcessor.java | 12 +- .../druid/segment/ArrayListSegment.java | 51 +- .../java/org/apache/druid/segment/Cursor.java | 2 +- .../apache/druid/segment/CursorFactory.java | 102 +--- ...dapter.java => FilteredCursorFactory.java} | 85 +-- .../apache/druid/segment/FilteredSegment.java | 56 ++ .../segment/IncrementalIndexSegment.java | 13 +- .../segment/PhysicalSegmentInspector.java | 59 ++ .../apache/druid/segment/QueryableIndex.java | 7 +- .../segment/QueryableIndexCursorFactory.java | 79 +++ .../QueryableIndexIndexableAdapter.java | 4 +- ...eryableIndexPhysicalSegmentInspector.java} | 136 +---- .../druid/segment/QueryableIndexSegment.java | 14 +- .../segment/ReferenceCountingSegment.java | 12 +- .../apache/druid/segment/RowBasedCursor.java | 2 +- ...dapter.java => RowBasedCursorFactory.java} | 128 +---- .../apache/druid/segment/RowBasedSegment.java | 26 +- .../org/apache/druid/segment/RowWalker.java | 2 +- .../org/apache/druid/segment/Segment.java | 32 +- .../apache/druid/segment/SegmentWrangler.java | 2 +- .../druid/segment/SimpleQueryableIndex.java | 12 +- ...a => SimpleTopNOptimizationInspector.java} | 19 +- .../apache/druid/segment/StorageAdapter.java | 320 ++++++----- .../segment/TopNOptimizationInspector.java | 37 ++ .../UnnestColumnValueSelectorCursor.java | 2 +- ...eAdapter.java => UnnestCursorFactory.java} | 385 +++++-------- .../druid/segment/UnnestDimensionCursor.java | 2 +- .../apache/druid/segment/UnnestSegment.java | 59 ++ .../segment/VectorColumnProcessorFactory.java | 4 +- .../segment/WrappedSegmentReference.java | 34 +- .../segment/incremental/IncrementalIndex.java | 4 +- ...IncrementalIndexColumnSelectorFactory.java | 21 +- ...ava => IncrementalIndexCursorFactory.java} | 147 +---- .../IncrementalIndexCursorHolder.java | 7 +- ...rementalIndexPhysicalSegmentInspector.java | 139 +++++ .../IncrementalIndexRowIterator.java | 2 +- .../druid/segment/join/HashJoinSegment.java | 49 +- ...java => HashJoinSegmentCursorFactory.java} | 245 ++------ .../druid/segment/join/PostJoinCursor.java | 2 +- .../table/BroadcastSegmentIndexedTable.java | 24 +- .../join/table/FrameBasedIndexedTable.java | 13 +- .../loading/TombstoneSegmentizerFactory.java | 59 +- .../segment/vector/NilVectorSelector.java | 16 +- .../druid/segment/vector/VectorCursor.java | 4 +- ...onDeferredGroupByVectorColumnSelector.java | 10 + .../org/apache/druid/frame/FrameTest.java | 20 +- ...apter.java => TestArrayCursorFactory.java} | 14 +- .../ReadableByteChunksFrameChannelTest.java | 34 +- .../ReadableConcatFrameChannelTest.java | 10 +- .../ReadableInputStreamFrameChannelTest.java | 30 +- .../druid/frame/field/FieldReaderRACTest.java | 9 +- .../FrameFileHttpResponseHandlerTest.java | 26 +- .../druid/frame/file/FrameFileTest.java | 99 ++-- .../druid/frame/file/FrameFileWriterTest.java | 4 +- .../key/FrameComparisonWidgetImplTest.java | 14 +- .../ComposingOutputChannelFactoryTest.java | 2 +- .../processor/FrameProcessorExecutorTest.java | 33 +- .../processor/OutputChannelFactoryTest.java | 34 +- .../ReadableFileFrameChannelTest.java | 12 +- .../processor/RunAllFullyWidgetTest.java | 4 +- .../frame/processor/SuperSorterTest.java | 12 +- .../test/TestFrameProcessorUtils.java | 12 +- .../druid/frame/read/FrameReaderTest.java | 16 +- ...rTest.java => FrameCursorFactoryTest.java} | 109 +--- .../frame/testutil/FrameSequenceBuilder.java | 18 +- .../druid/frame/testutil/FrameTestUtil.java | 66 ++- .../testutil/RowNumberUpdatingCursor.java | 2 +- .../druid/frame/write/FrameWriterTest.java | 14 +- .../druid/query/CursorGranularizerTest.java | 19 +- .../first/StringFirstTimeseriesQueryTest.java | 8 +- .../last/StringLastTimeseriesQueryTest.java | 8 +- .../query/groupby/GroupByQueryRunnerTest.java | 2 +- .../groupby/NestedDataGroupByQueryTest.java | 2 +- .../VectorGroupByEngineIteratorTest.java | 9 +- .../druid/query/lookup/LookupSegmentTest.java | 61 +- .../query/metadata/SegmentAnalyzerTest.java | 4 - ...a => CursorFactoryRowsAndColumnsTest.java} | 16 +- .../rowsandcols/RowsAndColumnsTestBase.java | 2 +- .../semantic/RowsAndColumnsDecoratorTest.java | 2 +- ...ualColumnEvaluationRowsAndColumnsTest.java | 4 +- .../scan/ScanQueryResultOrderingTest.java | 2 +- .../NestedDataTimeseriesQueryTest.java | 2 +- .../query/topn/PooledTopNAlgorithmTest.java | 5 +- .../topn/TopNMetricSpecOptimizationsTest.java | 125 +--- .../druid/query/topn/TopNQueryRunnerTest.java | 2 +- .../segment/AutoTypeColumnIndexerTest.java | 44 +- .../apache/druid/segment/IndexBuilder.java | 6 +- .../druid/segment/IndexMergerTestBase.java | 50 +- .../org/apache/druid/segment/ListCursor.java | 2 +- .../NestedDataColumnIndexerV4Test.java | 26 +- ...a => QueryableIndexCursorFactoryTest.java} | 12 +- .../segment/ReferenceCountingSegmentTest.java | 16 +- ...st.java => RowBasedCursorFactoryTest.java} | 163 +----- .../druid/segment/TestSegmentForAs.java | 4 +- .../TombstoneSegmentStorageAdapterTest.java | 114 ---- ...Test.java => UnnestCursorFactoryTest.java} | 174 +++--- .../druid/segment/filter/AndFilterTest.java | 4 +- .../ArrayContainsElementFilterTests.java | 4 +- .../druid/segment/filter/BaseFilterTest.java | 126 ++-- .../druid/segment/filter/BoundFilterTest.java | 4 +- .../filter/ColumnComparisonFilterTest.java | 4 +- .../segment/filter/EqualityFilterTests.java | 4 +- ...ExpressionFilterNonStrictBooleansTest.java | 4 +- .../segment/filter/ExpressionFilterTest.java | 4 +- .../segment/filter/FilterPartitionTest.java | 4 +- .../filter/FloatAndDoubleFilteringTest.java | 4 +- .../druid/segment/filter/InFilterTests.java | 4 +- .../segment/filter/InvalidFilteringTest.java | 4 +- .../segment/filter/JavaScriptFilterTest.java | 4 +- .../druid/segment/filter/LikeFilterTest.java | 4 +- .../segment/filter/LongFilteringTest.java | 4 +- .../segment/filter/NotFilterEvaluateTest.java | 4 +- .../druid/segment/filter/NullFilterTests.java | 4 +- .../druid/segment/filter/OrFilterTest.java | 4 +- .../segment/filter/RangeFilterTests.java | 4 +- .../druid/segment/filter/RegexFilterTest.java | 4 +- .../segment/filter/SearchQueryFilterTest.java | 4 +- .../segment/filter/SelectorFilterTest.java | 4 +- .../segment/filter/TimeFilteringTest.java | 4 +- .../IncrementalIndexStorageAdapterTest.java | 38 +- ...BaseHashJoinSegmentCursorFactoryTest.java} | 14 +- ... => HashJoinSegmentCursorFactoryTest.java} | 539 +++++++----------- .../segment/join/HashJoinSegmentTest.java | 11 +- .../segment/join/JoinFilterAnalyzerTest.java | 240 ++++---- .../segment/join/PostJoinCursorTest.java | 53 +- .../BroadcastSegmentIndexedTableTest.java | 16 +- .../TombstoneSegmentizerFactoryTest.java | 88 ++- .../NestedFieldColumnSelectorsTest.java | 9 +- .../segment/vector/NilVectorSelectorTest.java | 4 +- .../virtual/ExpressionSelectorsTest.java | 33 +- .../ExpressionVectorSelectorsCastTest.java | 6 +- .../ExpressionVectorSelectorsTest.java | 8 +- .../druid/segment/realtime/FireHydrant.java | 24 +- ...dapter.java => WindowedCursorFactory.java} | 18 +- .../appenderator/BatchAppenderator.java | 2 +- .../appenderator/StreamAppenderator.java | 2 +- .../appenderator/TaskSegmentSchemaUtil.java | 10 +- .../druid/segment/realtime/sink/Sink.java | 31 +- .../apache/druid/server/SegmentManager.java | 20 +- .../server/coordination/ServerManager.java | 14 +- .../loading/SegmentLocalCacheManagerTest.java | 10 +- .../CoordinatorSegmentMetadataCacheTest.java | 34 +- .../segment/realtime/FireHydrantTest.java | 6 +- .../StreamAppenderatorTester.java | 12 + .../SegmentManagerThreadSafetyTest.java | 14 +- .../apache/druid/server/TestSegmentUtils.java | 90 +-- .../org/apache/druid/cli/DumpSegment.java | 6 +- .../org/apache/druid/cli/DumpSegmentTest.java | 16 +- .../calcite/rel/DruidCorrelateUnnestRel.java | 2 +- .../druid/sql/calcite/rel/DruidQuery.java | 11 +- .../sql/calcite/CalciteJoinQueryTest.java | 2 +- .../druid/sql/calcite/CalciteQueryTest.java | 3 +- .../druid/sql/calcite/NotYetSupported.java | 1 - .../BrokerSegmentMetadataCacheTest.java | 16 +- 242 files changed, 3293 insertions(+), 3722 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java rename processing/src/main/java/org/apache/druid/frame/segment/columnar/{ColumnarFrameCursorHolderFactory.java => ColumnarFrameCursorFactory.java} (86%) rename processing/src/main/java/org/apache/druid/frame/segment/row/{RowFrameCursorHolderFactory.java => RowFrameCursorFactory.java} (85%) rename processing/src/main/java/org/apache/druid/{segment/CursorHolderFactory.java => query/groupby/epinephelinae/GroupingSelector.java} (59%) rename processing/src/main/java/org/apache/druid/query/rowsandcols/{StorageAdapterRowsAndColumns.java => CursorFactoryRowsAndColumns.java} (73%) create mode 100644 processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.java rename processing/src/main/java/org/apache/druid/segment/{FilteredStorageAdapter.java => FilteredCursorFactory.java} (51%) create mode 100644 processing/src/main/java/org/apache/druid/segment/FilteredSegment.java create mode 100644 processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.java create mode 100644 processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java rename processing/src/main/java/org/apache/druid/segment/{QueryableIndexStorageAdapter.java => QueryableIndexPhysicalSegmentInspector.java} (55%) rename processing/src/main/java/org/apache/druid/segment/{RowBasedStorageAdapter.java => RowBasedCursorFactory.java} (59%) rename processing/src/main/java/org/apache/druid/segment/{AbstractSegment.java => SimpleTopNOptimizationInspector.java} (69%) create mode 100644 processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java rename processing/src/main/java/org/apache/druid/segment/{UnnestStorageAdapter.java => UnnestCursorFactory.java} (83%) create mode 100644 processing/src/main/java/org/apache/druid/segment/UnnestSegment.java rename processing/src/main/java/org/apache/druid/segment/incremental/{IncrementalIndexStorageAdapter.java => IncrementalIndexCursorFactory.java} (52%) create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java rename processing/src/main/java/org/apache/druid/segment/join/{HashJoinSegmentStorageAdapter.java => HashJoinSegmentCursorFactory.java} (56%) rename processing/src/test/java/org/apache/druid/frame/{TestArrayStorageAdapter.java => TestArrayCursorFactory.java} (92%) rename processing/src/test/java/org/apache/druid/frame/segment/{FrameStorageAdapterTest.java => FrameCursorFactoryTest.java} (77%) rename processing/src/test/java/org/apache/druid/query/rowsandcols/{StorageAdapterRowsAndColumnsTest.java => CursorFactoryRowsAndColumnsTest.java} (69%) rename processing/src/test/java/org/apache/druid/segment/{QueryableIndexStorageAdapterTest.java => QueryableIndexCursorFactoryTest.java} (94%) rename processing/src/test/java/org/apache/druid/segment/{RowBasedStorageAdapterTest.java => RowBasedCursorFactoryTest.java} (82%) delete mode 100644 processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java rename processing/src/test/java/org/apache/druid/segment/{UnnestStorageAdapterTest.java => UnnestCursorFactoryTest.java} (82%) rename processing/src/test/java/org/apache/druid/segment/join/{BaseHashJoinSegmentStorageAdapterTest.java => BaseHashJoinSegmentCursorFactoryTest.java} (95%) rename processing/src/test/java/org/apache/druid/segment/join/{HashJoinSegmentStorageAdapterTest.java => HashJoinSegmentCursorFactoryTest.java} (86%) rename server/src/main/java/org/apache/druid/segment/realtime/{WindowedStorageAdapter.java => WindowedCursorFactory.java} (74%) 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