transition away from StorageAdapter (#16985)

* 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
This commit is contained in:
Clint Wylie 2024-09-09 14:55:29 -07:00 committed by GitHub
parent f4261c0e4d
commit f57cd6f7af
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
242 changed files with 3390 additions and 3819 deletions

View File

@ -36,7 +36,7 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.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 class ExpressionAggregationBenchmark
private double compute(final Function<ColumnSelectorFactory, BufferAggregator> 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());

View File

@ -35,7 +35,7 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.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 class ExpressionFilterBenchmark
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 class ExpressionFilterBenchmark
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()) {

View File

@ -42,9 +42,8 @@ import org.apache.druid.segment.CursorHolder;
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 class ExpressionSelectorBenchmark
)
)
.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 class ExpressionSelectorBenchmark
@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 class ExpressionSelectorBenchmark
@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<Long> results =
Sequences.simple(granularizer.getBucketIterable())
@ -241,7 +242,8 @@ public class ExpressionSelectorBenchmark
)
.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 class ExpressionSelectorBenchmark
@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 class ExpressionSelectorBenchmark
)
.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 class ExpressionSelectorBenchmark
)
.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 class ExpressionSelectorBenchmark
@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 class ExpressionSelectorBenchmark
)
.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 class ExpressionSelectorBenchmark
)
)
.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 class ExpressionSelectorBenchmark
)
)
.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 class ExpressionSelectorBenchmark
)
)
.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 class ExpressionSelectorBenchmark
)
.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 class ExpressionSelectorBenchmark
)
.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 class ExpressionSelectorBenchmark
)
.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);

View File

@ -35,7 +35,7 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.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 class ExpressionVectorSelectorBenchmark
.setVirtualColumns(virtualColumns)
.build();
final CursorHolder cursorHolder = closer.register(
new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)
new QueryableIndexCursorFactory(index).makeCursorHolder(buildSpec)
);
if (vectorize) {
VectorCursor cursor = cursorHolder.asVectorCursor();

View File

@ -46,14 +46,14 @@ import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 @@ public class FilterPartitionBenchmark
@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 class FilterPartitionBenchmark
@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 class FilterPartitionBenchmark
@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 class FilterPartitionBenchmark
@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 class FilterPartitionBenchmark
@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 class FilterPartitionBenchmark
{
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 class FilterPartitionBenchmark
{
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 class FilterPartitionBenchmark
{
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 class FilterPartitionBenchmark
{
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 class FilterPartitionBenchmark
)
);
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 class FilterPartitionBenchmark
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 class FilterPartitionBenchmark
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 class FilterPartitionBenchmark
))
);
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 class FilterPartitionBenchmark
))
);
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())

View File

@ -238,7 +238,7 @@ public class IndexedTableJoinCursorBenchmark
private CursorHolder makeCursorHolder()
{
return hashJoinSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN);
return hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN);
}

View File

@ -369,7 +369,7 @@ public class JoinAndLookupBenchmark
@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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
@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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
@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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
@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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
@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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
.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 class JoinAndLookupBenchmark
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 class JoinAndLookupBenchmark
.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));
}

View File

@ -23,7 +23,7 @@ import com.google.common.base.Supplier;
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 class LongCompressionBenchmark
@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++) {

View File

@ -284,7 +284,7 @@ public class FrameChannelMergerBenchmark
signature
);
final Sequence<Frame> frameSequence =
FrameSequenceBuilder.fromAdapter(segment.asStorageAdapter())
FrameSequenceBuilder.fromCursorFactory(segment.asCursorFactory())
.allocator(ArenaMemoryAllocator.createOnHeap(10_000_000))
.frameType(FrameType.ROW_BASED)
.frames();

View File

@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.search.ContainsSearchQuerySpec;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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.generator.GeneratorSchemaInfo;
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 @@ public class IncrementalIndexReadBenchmark
@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<DimensionSelector> selectors = new ArrayList<>();
@ -183,8 +184,8 @@ public class IncrementalIndexReadBenchmark
)
);
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<DimensionSelector> selectors = new ArrayList<>();
@ -204,14 +205,14 @@ public class IncrementalIndexReadBenchmark
}
}
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)

View File

@ -57,8 +57,8 @@ import org.apache.druid.query.lookup.LookupExtractor;
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 @@ public class SqlBenchmark
} 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 @@ public class SqlBenchmark
} else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) {
walker.add(
descriptor,
FrameTestUtil.adapterToFrameSegment(
new QueryableIndexStorageAdapter(index),
FrameTestUtil.cursorFactoryToFrameSegment(
new QueryableIndexCursorFactory(index),
FrameType.COLUMNAR,
descriptor.getId()
)

View File

@ -55,8 +55,8 @@ import org.apache.druid.query.lookup.LookupExtractor;
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 @@ public class SqlWindowFunctionsBenchmark
} 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 @@ public class SqlWindowFunctionsBenchmark
} else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) {
walker.add(
descriptor,
FrameTestUtil.adapterToFrameSegment(
new QueryableIndexStorageAdapter(index),
FrameTestUtil.cursorFactoryToFrameSegment(
new QueryableIndexCursorFactory(index),
FrameType.COLUMNAR,
descriptor.getId()
)

View File

@ -35,8 +35,8 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue;
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 class DistinctCountTimeseriesQueryTest extends InitializedNullHandlingTes
final Iterable<Result<TimeseriesResultValue>> results =
engine.process(
query,
new IncrementalIndexStorageAdapter(index),
new IncrementalIndexCursorFactory(index),
new IncrementalIndexTimeBoundaryInspector(index),
new DefaultTimeseriesQueryMetrics()
).toList();

View File

@ -33,13 +33,13 @@ import org.apache.druid.query.topn.TopNQuery;
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 class DistinctCountTopNQueryTest extends InitializedNullHandlingTest
final Iterable<Result<TopNResultValue>> results =
engine.query(
query,
new IncrementalIndexStorageAdapter(index),
new IncrementalIndexTimeBoundaryInspector(index),
new IncrementalIndexSegment(index, SegmentId.dummy(QueryRunnerTestHelper.DATA_SOURCE)),
null
).toList();

View File

@ -26,10 +26,10 @@ import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 @@ import java.util.function.Supplier;
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 @@ public class ToObjectVectorColumnProcessorFactoryTest extends InitializedNullHan
)
)
.build();
return adapter.makeCursorHolder(buildSpec);
return cursorFactory.makeCursorHolder(buildSpec);
}
private List<Object> readColumn(final String column, final int limit)

View File

@ -37,8 +37,8 @@ import org.apache.druid.query.extraction.MapLookupExtractor;
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<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
Function<IndexBuilder, Pair<CursorFactory, Closeable>> finisher,
boolean cnf,
boolean optimize
)

View File

@ -35,9 +35,7 @@ import org.apache.druid.frame.processor.FrameProcessor;
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.BaseObjectColumnValueSelector;
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 @@ public class SegmentGeneratorFrameProcessor implements FrameProcessor<DataSegmen
// 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;

View File

@ -153,7 +153,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
final Sequence<ResultRow> rowSequence =
groupingEngine.process(
query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())),
mappedSegment.asStorageAdapter(),
mappedSegment.asCursorFactory(),
mappedSegment.as(TimeBoundaryInspector.class),
null
);
@ -187,7 +187,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
final Sequence<ResultRow> rowSequence =
groupingEngine.process(
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)),
mappedSegment.asStorageAdapter(),
mappedSegment.asCursorFactory(),
mappedSegment.as(TimeBoundaryInspector.class),
null
);

View File

@ -31,8 +31,7 @@ import org.apache.druid.frame.processor.FrameProcessor;
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.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.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 class ExportResultsFrameProcessor implements FrameProcessor<Object>
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();

View File

@ -65,12 +65,12 @@ import org.apache.druid.query.scan.ScanQueryEngine;
import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
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 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor
if (cursor == null) {
final ResourceHolder<Segment> 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 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor
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();

View File

@ -45,8 +45,8 @@ import org.apache.druid.query.InlineDataSource;
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 class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl
// 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 class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl
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 class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl
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 class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl
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()))
);

View File

@ -27,15 +27,13 @@ import org.apache.druid.frame.channel.BlockingQueueFrameChannel;
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 class FrameProcessorTestBase extends InitializedNullHandlingTest
exec.getExecutorService().awaitTermination(10, TimeUnit.MINUTES);
}
protected ReadableInput makeChannelFromAdapter(
final StorageAdapter adapter,
protected ReadableInput makeChannelFromCursorFactory(
final CursorFactory cursorFactory,
final List<KeyColumn> 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 @@ public class FrameProcessorTestBase extends InitializedNullHandlingTest
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);

View File

@ -47,8 +47,8 @@ import org.apache.druid.query.operator.window.WindowOperatorFactory;
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 @@ public class WindowOperatorQueryFrameProcessorTest extends FrameProcessorTestBas
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<KeyColumn> keyColumns
) throws IOException
{
return makeChannelFromAdapter(adapter, keyColumns, 1000);
return makeChannelFromCursorFactory(cursorFactory, keyColumns, 1000);
}
}

View File

@ -45,8 +45,8 @@ import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault;
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 @@ public class SortMergeJoinFrameProcessorTest extends FrameProcessorTestBase
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<KeyColumn> keyColumns
) throws IOException
{
return makeChannelFromAdapter(adapter, keyColumns, rowsPerInputFrame);
return makeChannelFromCursorFactory(cursorFactory, keyColumns, rowsPerInputFrame);
}
private FrameWriterFactory makeFrameWriterFactory(final RowSignature signature)

View File

@ -36,7 +36,7 @@ import org.apache.druid.msq.kernel.StagePartition;
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 class QueryResultsFrameProcessorTest extends FrameProcessorTestBase
FrameReader.create(signature)
);
FrameTestUtil.assertRowsEqual(
FrameTestUtil.readRowsFromAdapter(adapter, signature, false),
FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false),
rowsFromProcessor
);
Assert.assertEquals(Unit.instance(), retVal.get());

View File

@ -46,9 +46,10 @@ import org.apache.druid.msq.test.LimitedFrameWriterFactory;
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 class ScanQueryFrameProcessorTest extends FrameProcessorTestBase
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 class ScanQueryFrameProcessorTest extends FrameProcessorTestBase
);
FrameTestUtil.assertRowsEqual(
FrameTestUtil.readRowsFromAdapter(adapter, signature, false),
FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false),
rowsFromProcessor
);

View File

@ -32,7 +32,7 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder;
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()

View File

@ -61,13 +61,13 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.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 class CalciteMSQTestsHelper
}
@Override
public StorageAdapter asStorageAdapter()
public CursorFactory asCursorFactory()
{
return new QueryableIndexStorageAdapter(index);
return new QueryableIndexCursorFactory(index);
}
@Override

View File

@ -134,12 +134,12 @@ 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.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 class MSQTestBase extends BaseCalciteQueryTest
}
@Override
public StorageAdapter asStorageAdapter()
public CursorFactory asCursorFactory()
{
return new QueryableIndexStorageAdapter(index);
return new QueryableIndexCursorFactory(index);
}
@Override
@ -1246,12 +1246,11 @@ public class MSQTestBase extends BaseCalciteQueryTest
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 class MSQTestBase extends BaseCalciteQueryTest
queryableIndex.getMetadata().getAggregators()
);
for (List<Object> row : FrameTestUtil.readRowsFromAdapter(storageAdapter, null, false).toList()) {
for (List<Object> row : FrameTestUtil.readRowsFromCursorFactory(cursorFactory).toList()) {
// transforming rows for sketch assertions
List<Object> transformedRow = row.stream()
.map(r -> {

View File

@ -47,11 +47,11 @@ import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.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 class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
spec = DatasourceInputFormat.getIngestionSpec(context.getConfiguration(), dataSource);
logger.info("load schema [%s]", spec);
List<WindowedStorageAdapter> adapters = Lists.transform(
List<WindowedCursorFactory> adapters = Lists.transform(
segments,
new Function<WindowedDataSegment, WindowedStorageAdapter>()
new Function<WindowedDataSegment, WindowedCursorFactory>()
{
@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 class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
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 class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
private Yielder<InputRow> rowYielder;
public SegmentReader(
final List<WindowedStorageAdapter> adapters,
final List<WindowedCursorFactory> cursorFactories,
final TransformSpec transformSpec,
final List<String> dims,
final List<String> metrics,
@ -207,18 +204,18 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
Sequence<InputRow> rows = Sequences.concat(
Iterables.transform(
adapters,
new Function<WindowedStorageAdapter, Sequence<InputRow>>()
cursorFactories,
new Function<WindowedCursorFactory, Sequence<InputRow>>()
{
@Nullable
@Override
public Sequence<InputRow> apply(WindowedStorageAdapter adapter)
public Sequence<InputRow> 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();

View File

@ -44,12 +44,11 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
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 @@ public class BatchDeltaIngestionTest
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,

View File

@ -36,22 +36,22 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
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 org.junit.runners.Parameterized;
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 class DatasourceRecordReaderSegmentReaderTest
.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 class DatasourceRecordReaderSegmentReaderTest
// 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<String> 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"),

View File

@ -49,15 +49,17 @@ import org.apache.druid.segment.ColumnProcessorFactory;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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<Map<String,
protected CloseableIterator<Map<String, Object>> 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 @@ public class DruidSegmentReader extends IntermediateRowParsingReader<Map<String,
// Retain order of columns from the original segments. Useful for preserving dimension order if we're in
// schemaless mode.
final Set<String> columnsToRead = Sets.newLinkedHashSet(
Iterables.filter(
storageAdapter.getAdapter().getRowSignature().getColumnNames(),
columnsFilter::apply
)
Iterables.filter(cursorFactory.getRowSignature().getColumnNames(), columnsFilter::apply)
);
final Sequence<Map<String, Object>> sequence = cursorToSequence(cursor, columnsToRead).withBaggage(cursorHolder);

View File

@ -80,7 +80,7 @@ import org.apache.druid.segment.DataSegmentsWithSchemas;
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.StorageLocationConfig;
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 class CompactionTaskRunTest extends IngestionTestBase
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 class CompactionTaskRunTest extends IngestionTestBase
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 class CompactionTaskRunTest extends IngestionTestBase
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 class CompactionTaskRunTest extends IngestionTestBase
);
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 @@ public class CompactionTaskRunTest extends IngestionTestBase
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();

View File

@ -73,7 +73,7 @@ import org.apache.druid.segment.DataSegmentsWithSchemas;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.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.SegmentLoaderConfig;
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 class IndexTaskTest extends IngestionTestBase
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<Map<String, Object>> transforms = new ArrayList<>();
@ -765,12 +765,12 @@ public class IndexTaskTest extends IngestionTestBase
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<Integer> hashes = new ArrayList<>();
final DimensionSelector selector = cursor.getColumnSelectorFactory()

View File

@ -57,7 +57,7 @@ public class RowReader
/**
* 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 class RowReader
/**
* 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<Object> readRow(final Memory memory, final long rowPosition, final long rowLength)

View File

@ -29,9 +29,7 @@ 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.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 class FrameProcessors
// 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();
}
/**

View File

@ -29,11 +29,11 @@ import org.apache.druid.frame.key.FrameComparisonWidgetImpl;
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 class FrameReader
}
/**
* 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());
}

View File

@ -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.

View File

@ -23,9 +23,9 @@ import org.apache.druid.frame.Frame;
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 class FrameSegment implements Segment
}
@Override
public StorageAdapter asStorageAdapter()
public CursorFactory asCursorFactory()
{
return new FrameStorageAdapter(frame, frameReader, segmentId.getInterval());
return frameReader.makeCursorFactory(frame);
}
@Override

View File

@ -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<String> getAvailableDimensions()
{
return new ListIndexed<>(frameReader.signature().getColumnNames());
}
@Override
public Iterable<String> 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);
}
}

View File

@ -25,7 +25,7 @@ import org.apache.druid.frame.read.columnar.FrameColumnReader;
import org.apache.druid.frame.segment.FrameCursor;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.segment.FrameFilteredOffset;
import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory;
import org.apache.druid.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.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.ColumnCache;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.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.Collections;
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<FrameColumnReader> columnReaders;
public ColumnarFrameCursorHolderFactory(
public ColumnarFrameCursorFactory(
final Frame frame,
final RowSignature signature,
final List<FrameColumnReader> columnReaders
@ -179,4 +180,25 @@ public class ColumnarFrameCursorHolderFactory implements CursorHolderFactory
}
};
}
@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();
}
}
}

View File

@ -43,7 +43,7 @@ import java.util.Map;
* 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.
*/

View File

@ -26,35 +26,37 @@ import org.apache.druid.frame.read.FrameReader;
import org.apache.druid.frame.segment.FrameCursor;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.segment.FrameFilteredOffset;
import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory;
import org.apache.druid.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<FieldReader> fieldReaders;
public RowFrameCursorHolderFactory(
public RowFrameCursorFactory(
final Frame frame,
final FrameReader frameReader,
final List<FieldReader> fieldReaders
@ -115,4 +117,17 @@ public class RowFrameCursorHolderFactory implements CursorHolderFactory
}
};
}
@Override
public RowSignature getRowSignature()
{
return frameReader.signature();
}
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return frameReader.signature().getColumnCapabilities(column);
}
}

View File

@ -25,9 +25,8 @@ import com.google.common.collect.ImmutableList;
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 class FilteredDataSource implements DataSource
);
return JvmUtils.safeAccumulateThreadCpuTime(
cpuTimeAccumulator,
() ->
baseSegment ->
new WrappedSegmentReference(
segmentMapFn.apply(baseSegment),
storageAdapter -> new FilteredStorageAdapter(storageAdapter, filter)
)
() -> baseSegment -> new FilteredSegment(segmentMapFn.apply(baseSegment), filter)
);
}

View File

@ -21,9 +21,7 @@ package org.apache.druid.query;
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 class FrameBasedInlineDataSource implements DataSource
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);
}
);

View File

@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.granularity.Granularity;
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 class QueryContext
public int getVectorSize()
{
return getVectorSize(QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE);
return getVectorSize(QueryContexts.DEFAULT_VECTOR_SIZE);
}
public int getVectorSize(int defaultSize)

View File

@ -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);

View File

@ -26,9 +26,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.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 class UnnestDataSource implements DataSource
);
return JvmUtils.safeAccumulateThreadCpuTime(
cpuTimeAccumulator,
() ->
baseSegment ->
new WrappedSegmentReference(
segmentMapFn.apply(baseSegment),
storageAdapter -> new UnnestStorageAdapter(storageAdapter, virtualColumn, unnestFilter)
)
() -> baseSegment -> new UnnestSegment(segmentMapFn.apply(baseSegment), virtualColumn, unnestFilter)
);
}

View File

@ -85,7 +85,7 @@ public class DataSourceMetadataQueryRunnerFactory
public DataSourceMetadataQueryRunner(Segment segment)
{
this.segmentInterval = segment.asStorageAdapter().getInterval();
this.segmentInterval = segment.getDataInterval();
this.inspector = segment.as(MaxIngestedEventTimeInspector.class);
}

View File

@ -77,7 +77,7 @@ public interface DimFilter extends Cacheable
*
* The Filter returned by this method across multiple calls must be the same object: parts of the query stack
* compare Filters, and returning the same object allows these checks to avoid deep comparisons.
* (see {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter#makeCursorHolder(CursorBuildSpec)}
* (see {@link org.apache.druid.segment.join.HashJoinSegmentCursorFactory#makeCursorHolder(CursorBuildSpec)}
* for an example}
*
* @param mayIncludeUnknown whether the optimized filter may need to operate in "includeUnknown" mode.

View File

@ -20,7 +20,7 @@
package org.apache.druid.query.filter.vector;
import com.google.common.base.Preconditions;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.query.QueryContexts;
import javax.annotation.Nullable;
@ -31,7 +31,7 @@ import javax.annotation.Nullable;
*/
public class VectorMatch implements ReadableVectorMatch
{
private static final int[] DEFAULT_ALL_TRUE_VECTOR = new int[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
private static final int[] DEFAULT_ALL_TRUE_VECTOR = new int[QueryContexts.DEFAULT_VECTOR_SIZE];
private static final VectorMatch ALL_FALSE = new VectorMatch(new int[0], 0);

View File

@ -30,8 +30,8 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
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;
@ -88,14 +88,14 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
private static class GroupByQueryRunner implements QueryRunner<ResultRow>
{
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 class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
return groupingEngine.process(
(GroupByQuery) query,
adapter,
cursorFactory,
timeBoundaryInspector,
(GroupByQueryMetrics) queryPlus.getQueryMetrics()
);

View File

@ -70,6 +70,7 @@ import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner;
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.groupby.orderby.NoopLimitSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 class GroupingEngine
}
/**
* 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<ResultRow> 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 class GroupingEngine
: 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 class GroupingEngine
if (shouldVectorize) {
result = VectorGroupByEngine.process(
query,
storageAdapter,
timeBoundaryInspector,
cursorHolder,
bufferHolder.get(),
@ -529,7 +529,6 @@ public class GroupingEngine
} else {
result = GroupByQueryEngine.process(
query,
storageAdapter,
timeBoundaryInspector,
cursorHolder,
buildSpec,
@ -874,7 +873,8 @@ public class GroupingEngine
public static int getCardinalityForArrayAggregation(
GroupByQueryConfig querySpecificConfig,
GroupByQuery query,
StorageAdapter storageAdapter,
ColumnInspector columnInspector,
List<? extends GroupingSelector> groupingSelectors,
ByteBuffer buffer
)
{
@ -893,7 +893,7 @@ public class GroupingEngine
} 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 class GroupingEngine
}
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;

View File

@ -53,7 +53,6 @@ import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.StorageAdapter;
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 org.joda.time.DateTime;
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.
* <p>
* 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.
* <p>
* Used for non-vectorized processing by {@link GroupingEngine#process}.
*
@ -95,7 +96,6 @@ public class GroupByQueryEngine
public static Sequence<ResultRow> process(
final GroupByQuery query,
final StorageAdapter storageAdapter,
@Nullable final TimeBoundaryInspector timeBoundaryInspector,
final CursorHolder cursorHolder,
final CursorBuildSpec buildSpec,
@ -126,7 +126,7 @@ public class GroupByQueryEngine
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 class GroupByQueryEngine
final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation(
querySpecificConfig,
query,
storageAdapter,
columnSelectorFactory,
Arrays.asList(dims),
processingBuffer
);

View File

@ -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;
}
}

View File

@ -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<GroupByColumnSelectorStrategy>
public class GroupByColumnSelectorPlus extends ColumnSelectorPlus<GroupByColumnSelectorStrategy> implements
GroupingSelector
{
/**
* Indicates the offset of this dimension's value within ResultRows.
@ -58,4 +62,14 @@ public class GroupByColumnSelectorPlus extends ColumnSelectorPlus<GroupByColumnS
{
return resultRowPosition;
}
@Override
public int getValueCardinality()
{
final ColumnValueSelector<?> selector = getSelector();
if (selector instanceof DimensionDictionarySelector) {
return ((DimensionDictionarySelector) selector).getValueCardinality();
}
return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.druid.query.groupby.epinephelinae.vector;
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 @@ import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer;
* @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.

View File

@ -41,6 +41,12 @@ public class NilGroupByVectorColumnSelector implements GroupByVectorColumnSelect
return 0;
}
@Override
public int getValueCardinality()
{
return 1;
}
@Override
public int writeKeys(WritableMemory keySpace, int keySize, int keyOffset, int startRow, int endRow)
{

View File

@ -39,6 +39,12 @@ public class SingleValueStringGroupByVectorColumnSelector implements GroupByVect
return Integer.BYTES;
}
@Override
public int getValueCardinality()
{
return selector.getValueCardinality();
}
@Override
public int writeKeys(
final WritableMemory keySpace,

View File

@ -45,7 +45,6 @@ import org.apache.druid.query.vector.VectorCursorGranularizer;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.TimeBoundaryInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
@ -63,8 +62,9 @@ import java.util.NoSuchElementException;
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.
* <p>
* Used for vectorized processing by {@link GroupingEngine#process}.
*
@ -79,7 +79,6 @@ public class VectorGroupByEngine
public static Sequence<ResultRow> process(
final GroupByQuery query,
final StorageAdapter storageAdapter,
@Nullable TimeBoundaryInspector timeBoundaryInspector,
final CursorHolder cursorHolder,
final ByteBuffer processingBuffer,
@ -145,7 +144,6 @@ public class VectorGroupByEngine
query,
config,
processingConfig,
storageAdapter,
timeBoundaryInspector,
cursor,
cursorHolder.getTimeOrder(),
@ -209,7 +207,6 @@ public class VectorGroupByEngine
private final GroupByQuery query;
private final GroupByQueryConfig querySpecificConfig;
private final DruidProcessingConfig processingConfig;
private final StorageAdapter storageAdapter;
private final VectorCursor cursor;
private final List<GroupByVectorColumnSelector> selectors;
private final ByteBuffer processingBuffer;
@ -241,7 +238,6 @@ public class VectorGroupByEngine
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 @@ public class VectorGroupByEngine
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 class VectorGroupByEngine
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 @@ public class VectorGroupByEngine
grouper = new BufferArrayGrouper(
Suppliers.ofInstance(processingBuffer),
AggregatorAdapters.factorizeVector(
cursor.getColumnSelectorFactory(),
columnSelectorFactory,
query.getAggregatorSpecs()
),
cardinalityForArrayAggregation

View File

@ -29,11 +29,12 @@ import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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.RowSignature;
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 class SegmentAnalyzer
public long numRows(Segment segment)
{
return Preconditions.checkNotNull(segment, "segment").asStorageAdapter().getNumRows();
return Preconditions.checkNotNull(segment.as(PhysicalSegmentInspector.class), "PhysicalSegmentInspector")
.getNumRows();
}
public Map<String, ColumnAnalysis> 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<String, ColumnAnalysis> 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 class SegmentAnalyzer
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 @@ public class SegmentAnalyzer
private ColumnAnalysis analyzeStringColumn(
final ColumnCapabilities capabilities,
final StorageAdapter storageAdapter,
@Nullable final PhysicalSegmentInspector analysisInspector,
final CursorFactory cursorFactory,
final String columnName
)
{
@ -265,12 +265,17 @@ public class SegmentAnalyzer
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 @@ public class SegmentAnalyzer
}
}
if (analyzingMinMax()) {
min = storageAdapter.getMinValue(columnName);
max = storageAdapter.getMaxValue(columnName);
}
return ColumnAnalysis.builder()
.withCapabilities(capabilities)
.withSize(size)

View File

@ -47,9 +47,11 @@ import org.apache.druid.query.metadata.metadata.ColumnIncluderator;
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 class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
final Map<String, AggregatorFactory> 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 class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
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 class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
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 class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
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 class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
{
return toolChest;
}
@Nullable
private Metadata getMetadata(Segment segment)
{
PhysicalSegmentInspector inspector = segment.as(PhysicalSegmentInspector.class);
if (inspector != null) {
return inspector.getMetadata();
}
return null;
}
}

View File

@ -64,7 +64,7 @@ import java.util.Optional;
* 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

View File

@ -41,9 +41,9 @@ import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
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 class ArrayListRowsAndColumns<RowType> implements AppendableRowsAndColumn
@SuppressWarnings("unused")
@SemanticCreator
public StorageAdapter toStorageAdapter()
public CursorFactory toCursorFactory()
{
return new RowBasedStorageAdapter<RowType>(Sequences.simple(rows), rowAdapter, rowSignature);
return new RowBasedCursorFactory<>(Sequences.simple(rows), rowAdapter, rowSignature);
}
private class MyClusteredGroupPartitioner implements ClusteredGroupPartitioner

View File

@ -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.CloseableShapeshifter;
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.Collection;
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<RowsAndColumns> 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> T as(Class<T> clazz)
{
if (StorageAdapter.class == clazz) {
return (T) storageAdapter;
if (CursorFactory.class == clazz) {
return (T) cursorFactory;
}
return null;
}
@ -64,19 +67,19 @@ public class StorageAdapterRowsAndColumns implements CloseableShapeshifter, Rows
@Override
public Collection<String> 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 class StorageAdapterRowsAndColumns implements CloseableShapeshifter, Rows
{
}
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();

View File

@ -46,8 +46,8 @@ import org.apache.druid.query.rowsandcols.semantic.WireTransferable;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
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 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
}
@Nullable
private Pair<byte[], RowSignature> materializeStorageAdapter(StorageAdapter as)
private Pair<byte[], RowSignature> materializeCursorFactory(CursorFactory cursorFactory)
{
final Collection<String> cols;
if (viewableColumns != null) {
@ -228,7 +228,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
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) {

View File

@ -23,13 +23,11 @@ import org.apache.druid.frame.Frame;
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 class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoClose
@Override
public <T> T as(Class<T> 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;
}

View File

@ -26,9 +26,9 @@ import org.apache.druid.java.util.common.io.Closer;
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 class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseab
@SuppressWarnings("unused")
@SemanticCreator
public StorageAdapter toStorageAdapter()
public CursorFactory toCursorFactory()
{
return new QueryableIndexStorageAdapter(index);
return new QueryableIndexCursorFactory(index);
}
@SuppressWarnings("unused")

View File

@ -25,12 +25,10 @@ import org.apache.druid.frame.FrameType;
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 class RowBasedFrameRowsAndColumns implements RowsAndColumns, AutoCloseabl
@Override
public <T> T as(Class<T> 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;
}

View File

@ -38,10 +38,10 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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,18 +75,18 @@ public class ScanQueryEngine
if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(Order.NONE)) {
return Sequences.empty();
}
final boolean hasTimeout = query.context().hasTimeout();
final Long timeoutAt = responseContext.getTimeoutTime();
final StorageAdapter adapter = segment.asStorageAdapter();
if (adapter == null) {
throw new ISE(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
);
if (segment.isTombstone()) {
return Sequences.empty();
}
if (adapter.isFromTombstone()) {
return Sequences.empty();
final boolean hasTimeout = query.context().hasTimeout();
final Long timeoutAt = responseContext.getTimeoutTime();
final CursorFactory cursorFactory = segment.asCursorFactory();
if (cursorFactory == null) {
throw new ISE(
"Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped."
);
}
final List<String> allColumns = new ArrayList<>();
@ -99,7 +99,7 @@ public class ScanQueryEngine
} else {
final Set<String> 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 class ScanQueryEngine
// 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(

View File

@ -26,10 +26,10 @@ import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.search.SearchQueryRunner.SearchColumnSelectorStrategy;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 @@ public class CursorOnlyStrategy extends SearchStrategy
@Override
public List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment)
{
final StorageAdapter adapter = segment.asStorageAdapter();
final List<DimensionSpec> dimensionSpecs = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions());
return ImmutableList.of(new CursorBasedExecutor(
query,
segment,
dimensionSpecs
));
final List<DimensionSpec> dimensionSpecs = getDimsToSearch(segment, query.getDimensions());
return ImmutableList.of(
new CursorBasedExecutor(
query,
segment,
dimensionSpecs
)
);
}
public static class CursorBasedExecutor extends SearchQueryExecutor
@ -74,7 +75,7 @@ public class CursorOnlyStrategy extends SearchStrategy
@Override
public Object2IntRBTreeMap<SearchHit> execute(final int limit)
{
final StorageAdapter adapter = segment.asStorageAdapter();
final CursorFactory adapter = segment.asCursorFactory();
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setInterval(query.getSingleInterval())
.setFilter(

View File

@ -23,14 +23,22 @@ import com.google.common.collect.ImmutableList;
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 @@ public abstract class SearchStrategy
public abstract List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment);
static List<DimensionSpec> getDimsToSearch(Indexed<String> availableDimensions, List<DimensionSpec> dimensions)
static List<DimensionSpec> getDimsToSearch(Segment segment, List<DimensionSpec> dimensions)
{
if (dimensions == null || dimensions.isEmpty()) {
return ImmutableList.copyOf(Iterables.transform(availableDimensions, Druids.DIMENSION_IDENTITY));
final Set<String> 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<String> 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;
}

View File

@ -35,11 +35,11 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
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 @@ public class UseIndexesStrategy extends SearchStrategy
public List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment)
{
final ImmutableList.Builder<SearchQueryExecutor> builder = ImmutableList.builder();
final QueryableIndex index = segment.asQueryableIndex();
final StorageAdapter adapter = segment.asStorageAdapter();
final List<DimensionSpec> searchDims = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions());
final QueryableIndex index = segment.as(QueryableIndex.class);
final CursorFactory cursorFactory = segment.asCursorFactory();
final List<DimensionSpec> searchDims = getDimsToSearch(segment, query.getDimensions());
if (index != null) {
// pair of bitmap dims and non-bitmap dims
final Pair<List<DimensionSpec>, List<DimensionSpec>> pair = partitionDimensionList(
adapter,
segment,
cursorFactory,
query.getVirtualColumns(),
searchDims
);
@ -131,7 +132,8 @@ public class UseIndexesStrategy extends SearchStrategy
* Note that the returned lists are free to modify.
*/
private static Pair<List<DimensionSpec>, List<DimensionSpec>> partitionDimensionList(
StorageAdapter adapter,
Segment segment,
CursorFactory cursorFactory,
VirtualColumns virtualColumns,
List<DimensionSpec> dimensions
)
@ -139,10 +141,10 @@ public class UseIndexesStrategy extends SearchStrategy
final List<DimensionSpec> bitmapDims = new ArrayList<>();
final List<DimensionSpec> nonBitmapDims = new ArrayList<>();
final List<DimensionSpec> 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 class UseIndexesStrategy extends SearchStrategy
@Override
public Object2IntRBTreeMap<SearchHit> 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(

View File

@ -40,11 +40,10 @@ import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 class TimeBoundaryQueryRunnerFactory
private static class TimeBoundaryQueryRunner implements QueryRunner<Result<TimeBoundaryResultValue>>
{
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 class TimeBoundaryQueryRunnerFactory
@Override
public Iterator<Result<TimeBoundaryResultValue>> 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 class TimeBoundaryQueryRunnerFactory
maxTime = timeBoundaryInspector.getMaxTime();
}
} else {
final Pair<DateTime, DateTime> timeBoundary = getTimeBoundary(query, adapter);
final Pair<DateTime, DateTime> 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 class TimeBoundaryQueryRunnerFactory
* 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 class TimeBoundaryQueryRunnerFactory
private static Pair<DateTime, DateTime> getTimeBoundary(
final TimeBoundaryQuery query,
final CursorHolderFactory cursorHolderFactory
final CursorFactory cursorFactory
)
{
DateTime minTime = null, maxTime = null;
@ -222,7 +223,7 @@ public class TimeBoundaryQueryRunnerFactory
.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 @@ public class TimeBoundaryQueryRunnerFactory
.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);

View File

@ -42,10 +42,10 @@ import org.apache.druid.query.vector.VectorCursorGranularizer;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.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 class TimeseriesQueryEngine
*/
public Sequence<Result<TimeseriesResultValue>> 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<TimeseriesResultValue>> result;

View File

@ -32,8 +32,8 @@ import org.apache.druid.query.QueryToolChest;
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 class TimeseriesQueryRunnerFactory
@Override
public QueryRunner<Result<TimeseriesResultValue>> 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 class TimeseriesQueryRunnerFactory
private static class TimeseriesQueryRunner implements QueryRunner<Result<TimeseriesResultValue>>
{
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 class TimeseriesQueryRunnerFactory
return engine.process(
(TimeseriesQuery) input,
adapter,
cursorFactory,
timeBoundaryInspector,
(TimeseriesQueryMetrics) queryPlus.getQueryMetrics()
);
}
}
}

View File

@ -28,7 +28,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
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 @@ import java.util.List;
*/
public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], TopNParams>
{
private final StorageAdapter storageAdapter;
private final TopNQuery query;
private final TopNCursorInspector cursorInspector;
private final NonBlockingPool<ByteBuffer> bufferPool;
public AggregateTopNMetricFirstAlgorithm(
StorageAdapter storageAdapter,
TopNQuery query,
TopNCursorInspector cursorInspector,
NonBlockingPool<ByteBuffer> bufferPool
)
{
this.storageAdapter = storageAdapter;
this.query = query;
this.cursorInspector = cursorInspector;
this.bufferPool = bufferPool;
}
@ -90,7 +89,7 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
.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 class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
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

View File

@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.BufferAggregator;
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 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
return aggregators;
}
protected final StorageAdapter storageAdapter;
protected TopNCursorInspector cursorInspector;
protected BaseTopNAlgorithm(StorageAdapter storageAdapter)
protected BaseTopNAlgorithm(TopNCursorInspector cursorInspector)
{
this.storageAdapter = storageAdapter;
this.cursorInspector = cursorInspector;
}
@Override
@ -212,11 +212,11 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
public AggregatorArrayProvider(
DimensionSelector dimSelector,
TopNQuery query,
int cardinality,
StorageAdapter storageAdapter
TopNCursorInspector cursorInspector,
int cardinality
)
{
super(dimSelector, query, storageAdapter);
super(dimSelector, query, cursorInspector);
this.expansionAggs = new Aggregator[cardinality][];
this.cardinality = cardinality;
@ -244,17 +244,17 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
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 abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
@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 abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
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());
}

View File

@ -24,7 +24,6 @@ import org.apache.druid.query.CursorGranularizer;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
import org.apache.druid.segment.Cursor;
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 @@ public class HeapBasedTopNAlgorithm
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
ColumnSelectorPlus<TopNColumnAggregatesProcessor> selectorPlus = params.getSelectorPlus();
return selectorPlus.getColumnSelectorStrategy().getRowSelector(query, params, storageAdapter);
return selectorPlus.getColumnSelectorStrategy().getRowSelector(query, params, cursorInspector);
}
@Override

View File

@ -35,7 +35,6 @@ import org.apache.druid.query.monomorphicprocessing.StringRuntimeShape;
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 @@ public class PooledTopNAlgorithm
private static final int AGG_UNROLL_COUNT = 8; // Must be able to fit loop below
public PooledTopNAlgorithm(
StorageAdapter storageAdapter,
TopNQuery query,
TopNCursorInspector cursorInspector,
NonBlockingPool<ByteBuffer> bufferPool
)
{
super(storageAdapter);
super(cursorInspector);
this.query = query;
this.bufferPool = bufferPool;
}
@ -232,11 +231,7 @@ public class PooledTopNAlgorithm
throw new UnsupportedOperationException("Cannot operate on a dimension with no dictionary");
}
final TopNMetricSpecBuilder<int[]> arrayProvider = new BaseArrayProvider<int[]>(
dimSelector,
query,
storageAdapter
)
final TopNMetricSpecBuilder<int[]> arrayProvider = new BaseArrayProvider<int[]>(dimSelector, query, cursorInspector)
{
private final int[] positions = new int[cardinality];

View File

@ -25,7 +25,6 @@ import org.apache.druid.query.aggregation.Aggregator;
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<int[], Map<Ob
private final TopNQuery query;
private final Function<Object, Object> 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

View File

@ -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;
}
}

View File

@ -23,9 +23,7 @@ import org.apache.druid.query.ColumnSelectorPlus;
import org.apache.druid.query.CursorGranularizer;
import org.apache.druid.query.Result;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandlerUtils;
import javax.annotation.Nullable;
@ -45,19 +43,13 @@ public class TopNMapFn
@SuppressWarnings("unchecked")
@Nullable
public Result<TopNResultValue> apply(final Cursor cursor, final CursorGranularizer granularizer, final @Nullable TopNQueryMetrics queryMetrics)
public Result<TopNResultValue> apply(
final Cursor cursor,
ColumnSelectorPlus<TopNColumnAggregatesProcessor<?>> selectorPlus,
final CursorGranularizer granularizer,
final @Nullable TopNQueryMetrics queryMetrics
)
{
final ColumnSelectorPlus<TopNColumnAggregatesProcessor<?>> 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);

View File

@ -25,17 +25,25 @@ import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.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 class TopNQueryEngine
}
/**
* 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<Result<TopNResultValue>> 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<TopNColumnAggregatesProcessor<?>> 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 class TopNQueryEngine
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 class TopNQueryEngine
*/
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 @@ public class TopNQueryEngine
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 @@ public class TopNQueryEngine
// 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) {

View File

@ -34,7 +34,6 @@ import org.apache.druid.query.QueryWatcher;
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 class TopNQueryRunnerFactory implements QueryRunnerFactory<Result<TopNRes
TopNQuery query = (TopNQuery) input.getQuery();
return queryEngine.query(
query,
segment.asStorageAdapter(),
segment.as(TimeBoundaryInspector.class),
segment,
(TopNQueryMetrics) input.getQueryMetrics()
);
}

View File

@ -23,12 +23,12 @@ import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.CursorGranularizer;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.topn.BaseTopNAlgorithm;
import org.apache.druid.query.topn.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 abstract class NullableNumericTopNColumnAggregatesProcessor<Selector exte
}
@Override
public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, StorageAdapter storageAdapter)
public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, TopNCursorInspector cursorInspector)
{
return null;
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.query.topn.types;
import org.apache.druid.query.CursorGranularizer;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.topn.BaseTopNAlgorithm;
import org.apache.druid.query.topn.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.Cursor;
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 class StringTopNColumnAggregatesProcessor implements TopNColumnAggregates
}
@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 class StringTopNColumnAggregatesProcessor implements TopNColumnAggregates
final BaseTopNAlgorithm.AggregatorArrayProvider provider = new BaseTopNAlgorithm.AggregatorArrayProvider(
(DimensionSelector) params.getSelectorPlus().getSelector(),
query,
params.getCardinality(),
storageAdapter
cursorInspector,
params.getCardinality()
);
return provider.build();

View File

@ -23,12 +23,12 @@ import org.apache.druid.query.CursorGranularizer;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.dimension.ColumnSelectorStrategy;
import org.apache.druid.query.topn.HeapBasedTopNAlgorithm;
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<ValueSelectorType> 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()}

View File

@ -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<RowType> implements Segment
public class ArrayListSegment<RowType> extends RowBasedSegment<RowType>
{
private final SegmentId segmentId;
private final ArrayList<RowType> rows;
private final RowAdapter<RowType> rowAdapter;
private final RowAdapter<RowType> adapter;
private final RowSignature rowSignature;
/**
* Create a list-based segment.
* <p>
* 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.
* <p>
* 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 class ArrayListSegment<RowType> implements Segment
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 class ArrayListSegment<RowType> implements Segment
if (CloseableShapeshifter.class.equals(clazz)) {
return (T) new MyCloseableShapeshifter();
}
return Segment.super.as(clazz);
return super.as(clazz);
}
@Override
@ -120,7 +87,7 @@ public class ArrayListSegment<RowType> implements Segment
private RowsAndColumns asRowsAndColumns()
{
return new ArrayListRowsAndColumns<>(rows, rowAdapter, rowSignature);
return new ArrayListRowsAndColumns<>(rows, adapter, rowSignature);
}
private class MyCloseableShapeshifter implements CloseableShapeshifter

View File

@ -24,7 +24,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexCursorHolder;
/**
* 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()}.
* <p>
* A typical usage pattern might look something like this:
* <pre>

View File

@ -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<Cursor> 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.
*
* Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval
* of this segment).
* 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.
*
* @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.
* @param column column name
*
* @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);
}

View File

@ -24,22 +24,20 @@ import org.apache.druid.query.filter.DimFilter;
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 class FilteredStorageAdapter implements StorageAdapter
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<String> getAvailableDimensions()
{
return baseStorageAdapter.getAvailableDimensions();
}
@Override
public Iterable<String> 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);
}
}

View File

@ -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> T as(@Nonnull Class<T> clazz)
{
if (TopNOptimizationInspector.class.equals(clazz)) {
return (T) new SimpleTopNOptimizationInspector(filter == null);
}
return super.as(clazz);
}
}

View File

@ -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 class IncrementalIndexSegment implements Segment
}
@Override
public StorageAdapter asStorageAdapter()
public CursorFactory asCursorFactory()
{
return new IncrementalIndexStorageAdapter(index);
return new IncrementalIndexCursorFactory(index);
}
@Nullable
@ -71,6 +72,12 @@ public class IncrementalIndexSegment implements Segment
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);
}

View File

@ -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();
}

View File

@ -34,9 +34,9 @@ import java.util.Map;
/**
* 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<String> getAvailableDimensions();
BitmapFactory getBitmapFactoryForDimensions();
@Nullable Metadata getMetadata();
@Nullable
Metadata getMetadata();
/**
* Map of column name to {@link DimensionHandler}, whose contents and iteration order matches

View File

@ -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<String> 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);
}
}

View File

@ -265,10 +265,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
/**
* 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.

View File

@ -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()
public Metadata getMetadata()
{
return index.getDataInterval();
}
@Override
public RowSignature getRowSignature()
{
final LinkedHashSet<String> 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<String> getAvailableDimensions()
{
return index.getAvailableDimensions();
}
@Override
public Iterable<String> getAvailableMetrics()
{
// Use LinkedHashSet to preserve the original order.
final Set<String> 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()
{
return index.getNumRows();
return index.getMetadata();
}
@Override
@ -163,31 +72,34 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
}
@Override
public int getDimensionCardinality(String column)
{
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 ColumnCapabilities getColumnCapabilities(String column)
{
return index.getColumnCapabilities(column);
}
@Override
public DateTime getMaxIngestedEventTime()
public int getNumRows()
{
// For immutable indexes, maxIngestedEventTime is maxTime.
return getMaxTime();
}
@Override
public CursorHolder makeCursorHolder(CursorBuildSpec spec)
{
return new QueryableIndexCursorHolder(
index,
CursorBuildSpec.builder(spec).build()
);
}
@Override
public Metadata getMetadata()
{
return index.getMetadata();
return index.getNumRows();
}
}

Some files were not shown because too many files have changed in this diff Show More