This is an automated email from the ASF dual-hosted git repository.
gian pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/master by this push:
new 337b5b5d4c7 cleanup segment interface (#17977)
337b5b5d4c7 is described below
commit 337b5b5d4c738b502b1bde6a1b6e59447d0de9d9
Author: Clint Wylie <[email protected]>
AuthorDate: Mon May 5 14:32:45 2025 -0700
cleanup segment interface (#17977)
* cleanup segment interface
changes:
* remove `Segment#asStorageAdapter`, it was deprecated previously and not
used for anything
* transition all callers of `Segment` methods `asCursorFactory()` and
`asQueryableIndex()` to use `as(CursorFactory.class)` and
`as(QueryableIndex.class)`
* remove default implementation of the `as` method on `Segment`, it is now
required
* remove `asCursorFactory` and `asQueryableIndex` from `Segment`
* fixes
---
.../benchmark/IndexedTableJoinCursorBenchmark.java | 3 +-
.../druid/benchmark/JoinAndLookupBenchmark.java | 29 +--
.../frame/FrameChannelMergerBenchmark.java | 3 +-
.../groupby/GroupByPreShuffleFrameProcessor.java | 6 +-
.../results/ExportResultsFrameProcessor.java | 5 +-
.../msq/querykit/scan/ScanQueryFrameProcessor.java | 4 +-
.../WindowOperatorQueryFrameProcessorTest.java | 2 +-
.../common/SortMergeJoinFrameProcessorTest.java | 2 +-
.../org/apache/druid/msq/test/MSQTestBase.java | 15 +-
.../DatasourceRecordReaderSegmentReaderTest.java | 3 +-
.../apache/druid/frame/segment/FrameSegment.java | 18 +-
.../org/apache/druid/query/JoinDataSource.java | 2 +-
.../query/groupby/GroupByQueryRunnerFactory.java | 3 +-
.../druid/query/metadata/SegmentAnalyzer.java | 6 +-
.../apache/druid/query/scan/ScanQueryEngine.java | 2 +-
.../druid/query/search/CursorOnlyStrategy.java | 2 +-
.../apache/druid/query/search/SearchStrategy.java | 4 +-
.../TimeBoundaryQueryRunnerFactory.java | 2 +-
.../timeseries/TimeseriesQueryRunnerFactory.java | 2 +-
.../apache/druid/query/topn/TopNQueryEngine.java | 2 +-
.../org/apache/druid/segment/ArrayListSegment.java | 6 +-
.../druid/segment/BypassRestrictedSegment.java | 18 +-
.../org/apache/druid/segment/FilteredSegment.java | 12 +-
.../druid/segment/IncrementalIndexSegment.java | 19 +-
.../druid/segment/QueryableIndexSegment.java | 20 +-
.../druid/segment/ReferenceCountingSegment.java | 14 --
.../druid/segment/RestrictedCursorFactory.java | 2 +-
.../apache/druid/segment/RestrictedSegment.java | 19 +-
.../org/apache/druid/segment/RowBasedSegment.java | 17 +-
.../java/org/apache/druid/segment/Segment.java | 33 +--
.../org/apache/druid/segment/StorageAdapter.java | 278 ---------------------
.../org/apache/druid/segment/UnnestSegment.java | 14 +-
.../druid/segment/WrappedSegmentReference.java | 20 --
.../apache/druid/segment/join/HashJoinSegment.java | 35 +--
.../join/table/BroadcastSegmentIndexedTable.java | 2 +-
.../loading/TombstoneSegmentizerFactory.java | 64 ++---
.../frame/key/FrameComparisonWidgetImplTest.java | 4 +-
.../frame/segment/FrameCursorFactoryTest.java | 5 +-
.../apache/druid/frame/write/FrameWriterTest.java | 4 +-
.../query/groupby/GroupByQueryRunnerTest.java | 3 +-
.../query/groupby/NestedDataGroupByQueryTest.java | 6 +-
.../druid/query/lookup/LookupSegmentTest.java | 22 +-
.../semantic/RowsAndColumnsDecoratorTest.java | 3 +-
.../timeseries/NestedDataTimeseriesQueryTest.java | 3 +-
.../druid/query/topn/TopNQueryRunnerTest.java | 3 +-
.../druid/segment/CursorHolderPreaggTest.java | 12 +-
.../segment/ReferenceCountingSegmentTest.java | 20 +-
.../org/apache/druid/segment/TestSegmentForAs.java | 13 -
.../org/apache/druid/segment/TestSegmentUtils.java | 25 +-
.../druid/segment/filter/BaseFilterTest.java | 8 +-
.../join/HashJoinSegmentCursorFactoryTest.java | 97 +++----
.../druid/segment/join/HashJoinSegmentTest.java | 18 +-
.../druid/segment/join/JoinFilterAnalyzerTest.java | 69 ++---
.../druid/segment/join/PostJoinCursorTest.java | 13 +-
.../loading/TombstoneSegmentizerFactoryTest.java | 66 +++--
.../nested/NestedFieldColumnSelectorsTest.java | 10 +-
.../apache/druid/segment/realtime/sink/Sink.java | 6 +-
.../loading/SegmentLocalCacheManagerTest.java | 2 +-
.../druid/segment/realtime/FireHydrantTest.java | 13 +-
.../server/SegmentManagerThreadSafetyTest.java | 16 --
60 files changed, 338 insertions(+), 791 deletions(-)
diff --git
a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
index 1c0a5335825..be845b16d9f 100644
---
a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
+++
b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
@@ -36,6 +36,7 @@ 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.DimensionSelector;
import org.apache.druid.segment.QueryableIndex;
@@ -233,7 +234,7 @@ public class IndexedTableJoinCursorBenchmark
private CursorHolder makeCursorHolder()
{
- return
hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN);
+ return
hashJoinSegment.as(CursorFactory.class).makeCursorHolder(CursorBuildSpec.FULL_SCAN);
}
diff --git
a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
index 73919effeaa..5a701141674 100644
---
a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
+++
b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
@@ -33,6 +33,7 @@ import
org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.query.lookup.MapLookupExtractorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.QueryableIndex;
@@ -364,7 +365,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void baseSegment(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder =
baseSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ try (final CursorHolder cursorHolder =
baseSegment.as(CursorFactory.class).makeCursorHolder(CursorBuildSpec.FULL_SCAN))
{
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "countryIsoCode"));
}
@@ -379,7 +380,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder =
baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder =
baseSegment.as(CursorFactory.class).makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "countryIsoCode"));
}
@@ -390,7 +391,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinLookupStringKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -406,7 +407,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -418,7 +419,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinLookupLongKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -434,7 +435,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -446,7 +447,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinIndexedTableLongKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -462,7 +463,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -474,7 +475,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinIndexedTableStringKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -490,7 +491,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -505,7 +506,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setVirtualColumns(lookupVirtualColumns)
.build();
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME));
@@ -522,7 +523,7 @@ public class JoinAndLookupBenchmark
.setFilter(filter)
.setVirtualColumns(lookupVirtualColumns)
.build();
- try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.asCursorFactory()
+ try (final CursorHolder cursorHolder =
hashJoinLookupStringKeySegment.as(CursorFactory.class)
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME));
@@ -537,7 +538,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setVirtualColumns(lookupVirtualColumns)
.build();
- try (final CursorHolder cursorHolder =
baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder =
baseSegment.as(CursorFactory.class).makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME));
}
@@ -553,7 +554,7 @@ public class JoinAndLookupBenchmark
.setVirtualColumns(lookupVirtualColumns)
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder =
baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder =
baseSegment.as(CursorFactory.class).makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME));
}
diff --git
a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
index 8739f03ccea..33db34f5fcd 100644
---
a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
+++
b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
@@ -45,6 +45,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.RowBasedSegment;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
@@ -282,7 +283,7 @@ public class FrameChannelMergerBenchmark
signature
);
final Sequence<Frame> frameSequence =
- FrameSequenceBuilder.fromCursorFactory(segment.asCursorFactory())
+
FrameSequenceBuilder.fromCursorFactory(segment.as(CursorFactory.class))
.allocator(ArenaMemoryAllocator.createOnHeap(10_000_000))
.frameType(FrameType.ROW_BASED)
.frames();
diff --git
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
index a02e6e6988d..13ec8e0fa5f 100644
---
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
+++
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
@@ -55,6 +55,7 @@ import
org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.SpecificSegmentSpec;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.CompleteSegment;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.TimeBoundaryInspector;
import org.apache.druid.segment.column.RowSignature;
@@ -62,6 +63,7 @@ import org.apache.druid.segment.column.RowSignature;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
+import java.util.Objects;
import java.util.function.Function;
/**
@@ -157,7 +159,7 @@ public class GroupByPreShuffleFrameProcessor extends
BaseLeafFrameProcessor
final Sequence<ResultRow> rowSequence =
groupingEngine.process(
query.withQuerySegmentSpec(new
SpecificSegmentSpec(segment.getDescriptor())),
- mappedSegment.asCursorFactory(),
+ Objects.requireNonNull(mappedSegment.as(CursorFactory.class)),
mappedSegment.as(TimeBoundaryInspector.class),
bufferPool,
null
@@ -192,7 +194,7 @@ public class GroupByPreShuffleFrameProcessor extends
BaseLeafFrameProcessor
final Sequence<ResultRow> rowSequence =
groupingEngine.process(
query.withQuerySegmentSpec(new
MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)),
- mappedSegment.asCursorFactory(),
+ Objects.requireNonNull(mappedSegment.as(CursorFactory.class)),
mappedSegment.as(TimeBoundaryInspector.class),
bufferPool,
null
diff --git
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java
index e57134bc3d3..1d712012cbd 100644
---
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java
+++
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java
@@ -38,6 +38,7 @@ 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.Segment;
import org.apache.druid.segment.column.RowSignature;
@@ -51,6 +52,7 @@ import java.io.IOException;
import java.io.OutputStream;
import java.util.Collections;
import java.util.List;
+import java.util.Objects;
import java.util.stream.Collectors;
public class ExportResultsFrameProcessor implements FrameProcessor<Object>
@@ -151,7 +153,8 @@ public class ExportResultsFrameProcessor implements
FrameProcessor<Object>
private void exportFrame(final Frame frame)
{
final Segment segment = new FrameSegment(frame, frameReader);
- try (final CursorHolder cursorHolder =
segment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ try (final CursorHolder cursorHolder =
Objects.requireNonNull(segment.as(CursorFactory.class))
+
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
if (cursor == null) {
exportWriter.writeRowEnd();
diff --git
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
index ef453badab8..7c19e83cb68 100644
---
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
+++
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
@@ -255,7 +255,7 @@ public class ScanQueryFrameProcessor extends
BaseLeafFrameProcessor
final ResourceHolder<CompleteSegment> segmentHolder =
closer.register(segment.getOrLoad());
final Segment mappedSegment =
mapSegment(segmentHolder.get().getSegment());
- final CursorFactory cursorFactory = mappedSegment.asCursorFactory();
+ final CursorFactory cursorFactory =
mappedSegment.as(CursorFactory.class);
if (cursorFactory == null) {
throw new ISE(
"Null cursor factory found. Probably trying to issue a query
against a segment being memory unmapped."
@@ -315,7 +315,7 @@ public class ScanQueryFrameProcessor extends
BaseLeafFrameProcessor
final FrameSegment frameSegment = new FrameSegment(frame,
inputFrameReader);
final Segment mappedSegment = mapSegment(frameSegment);
- final CursorFactory cursorFactory = mappedSegment.asCursorFactory();
+ final CursorFactory cursorFactory =
mappedSegment.as(CursorFactory.class);
if (cursorFactory == null) {
throw new ISE(
"Null cursor factory found. Probably trying to issue a query
against a segment being memory unmapped."
diff --git
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java
index cabb8f25c1a..88f98e042c0 100644
---
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java
+++
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java
@@ -361,7 +361,7 @@ public class WindowOperatorQueryFrameProcessorTest extends
FrameProcessorTestBas
signature
);
- return makeChannelFromCursorFactory(segment.asCursorFactory(), keyColumns);
+ return makeChannelFromCursorFactory(segment.as(CursorFactory.class),
keyColumns);
}
private ReadableInput makeChannelFromCursorFactory(
diff --git
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java
index dcbf52e0d73..fd3c66acf34 100644
---
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java
+++
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java
@@ -1462,7 +1462,7 @@ public class SortMergeJoinFrameProcessorTest extends
FrameProcessorTestBase
signature
)
)) {
- final CursorFactory cursorFactory = segment.asCursorFactory();
+ final CursorFactory cursorFactory = segment.as(CursorFactory.class);
return makeChannelFromCursorFactory(cursorFactory, keyColumns);
}
}
diff --git
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
index 174590bd4b8..52c4990164a 100644
---
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
+++
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
@@ -726,15 +726,14 @@ public class MSQTestBase extends BaseCalciteQueryTest
@Nullable
@Override
- public QueryableIndex asQueryableIndex()
+ public <T> T as(@Nonnull Class<T> clazz)
{
- return index;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new QueryableIndexCursorFactory(index);
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new QueryableIndexCursorFactory(index);
+ } else if (QueryableIndex.class.equals(clazz)) {
+ return (T) index;
+ }
+ return null;
}
@Override
diff --git
a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java
b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java
index 452a658e9d4..047f9884498 100644
---
a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java
+++
b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java
@@ -36,6 +36,7 @@ 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.CursorFactory;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMerger;
@@ -178,7 +179,7 @@ public class DatasourceRecordReaderSegmentReaderTest
// Do a spatial filter
final DatasourceRecordReader.SegmentReader segmentReader2 = new
DatasourceRecordReader.SegmentReader(
- ImmutableList.of(new
WindowedCursorFactory(queryable.asCursorFactory(), Intervals.of("2000/3000"))),
+ ImmutableList.of(new
WindowedCursorFactory(queryable.as(CursorFactory.class),
Intervals.of("2000/3000"))),
TransformSpec.NONE,
ImmutableList.of("host", "spatial"),
ImmutableList.of("visited_sum", "unique_hosts"),
diff --git
a/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java
b/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java
index 4e0c560c0e0..802b22cc494 100644
--- a/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java
+++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java
@@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.Intervals;
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.timeline.SegmentId;
import org.joda.time.Interval;
@@ -61,19 +60,6 @@ public class FrameSegment implements Segment
return Intervals.ETERNITY;
}
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return null;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return frameReader.makeCursorFactory(frame);
- }
-
@Override
public void close()
{
@@ -87,7 +73,9 @@ public class FrameSegment implements Segment
{
if (CloseableShapeshifter.class.equals(clazz)) {
return (T) new ColumnBasedFrameRowsAndColumns(frame,
frameReader.signature());
+ } else if (CursorFactory.class.equals(clazz)) {
+ return (T) frameReader.makeCursorFactory(frame);
}
- return Segment.super.as(clazz);
+ return null;
}
}
diff --git
a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java
b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java
index 96bdc5422ef..aeba6ad5a55 100644
--- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java
+++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java
@@ -81,7 +81,7 @@ import java.util.stream.Collectors;
* the prefix, will be shadowed. It is up to the caller to ensure that no
important columns are shadowed by the
* chosen prefix.
* <p>
- * When analyzed by {@link DataSourceAnalysis}, the right-hand side of this
datasource
+ * When analyzed by {@link JoinDataSourceAnalysis}, the right-hand side of
this datasource
* will become a {@link PreJoinableClause} object.
*/
public class JoinDataSource implements DataSource
diff --git
a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
index 665eb5dd125..53ca8815728 100644
---
a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
+++
b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
@@ -39,6 +39,7 @@ import org.apache.druid.segment.TimeBoundaryInspector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
+import java.util.Objects;
/**
*
@@ -102,7 +103,7 @@ public class GroupByQueryRunnerFactory implements
QueryRunnerFactory<ResultRow,
final NonBlockingPool<ByteBuffer> processingBufferPool
)
{
- this.cursorFactory = segment.asCursorFactory();
+ this.cursorFactory =
Objects.requireNonNull(segment.as(CursorFactory.class));
this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class);
this.groupingEngine = groupingEngine;
this.processingBufferPool = processingBufferPool;
diff --git
a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
index 82d10a5a944..51558806883 100644
---
a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
+++
b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
@@ -55,6 +55,7 @@ import java.io.IOException;
import java.util.EnumSet;
import java.util.LinkedHashMap;
import java.util.Map;
+import java.util.Objects;
public class SegmentAnalyzer
{
@@ -90,13 +91,14 @@ public class SegmentAnalyzer
// index is null for incremental-index-based segments, but
segmentInspector should always be available
final QueryableIndex index = segment.as(QueryableIndex.class);
+ final CursorFactory cursorFactory =
Objects.requireNonNull(segment.as(CursorFactory.class));
final int numRows = segmentInspector != null ?
segmentInspector.getNumRows() : 0;
// Use LinkedHashMap to preserve column order.
final Map<String, ColumnAnalysis> columns = new LinkedHashMap<>();
- final RowSignature rowSignature =
segment.asCursorFactory().getRowSignature();
+ final RowSignature rowSignature = cursorFactory.getRowSignature();
for (String columnName : rowSignature.getColumnNames()) {
final ColumnCapabilities capabilities;
@@ -131,7 +133,7 @@ public class SegmentAnalyzer
if (index != null) {
analysis = analyzeStringColumn(capabilities,
index.getColumnHolder(columnName));
} else {
- analysis = analyzeStringColumn(capabilities, segmentInspector,
segment.asCursorFactory(), columnName);
+ analysis = analyzeStringColumn(capabilities, segmentInspector,
cursorFactory, columnName);
}
break;
case ARRAY:
diff --git
a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
index 301014282ab..5d32c0d07df 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
@@ -80,7 +80,7 @@ public class ScanQueryEngine
final boolean hasTimeout = query.context().hasTimeout();
final Long timeoutAt = responseContext.getTimeoutTime();
- final CursorFactory cursorFactory = segment.asCursorFactory();
+ final CursorFactory cursorFactory = segment.as(CursorFactory.class);
if (cursorFactory == null) {
throw new ISE(
diff --git
a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java
b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java
index a3bd2905994..c6ac6b1a9b6 100644
---
a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java
+++
b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java
@@ -75,7 +75,7 @@ public class CursorOnlyStrategy extends SearchStrategy
@Override
public Object2IntRBTreeMap<SearchHit> execute(final int limit)
{
- final CursorFactory adapter = segment.asCursorFactory();
+ final CursorFactory adapter = segment.as(CursorFactory.class);
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setInterval(query.getSingleInterval())
.setFilter(
diff --git
a/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java
b/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java
index cbc0a91fb5c..5acc9f82098 100644
--- a/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java
@@ -26,6 +26,7 @@ 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.CursorFactory;
import org.apache.druid.segment.Metadata;
import org.apache.druid.segment.PhysicalSegmentInspector;
import org.apache.druid.segment.QueryableIndex;
@@ -38,6 +39,7 @@ import org.joda.time.Interval;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
public abstract class SearchStrategy
@@ -77,7 +79,7 @@ public abstract class SearchStrategy
ignore.add(factory.getName());
}
}
- final RowSignature rowSignature =
segment.asCursorFactory().getRowSignature();
+ final RowSignature rowSignature =
Objects.requireNonNull(segment.as(CursorFactory.class)).getRowSignature();
for (String columnName : rowSignature.getColumnNames()) {
if (!ignore.contains(columnName)) {
dims.add(columnName);
diff --git
a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
index 3f84da545c9..bad533ca06e 100644
---
a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
+++
b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java
@@ -101,7 +101,7 @@ public class TimeBoundaryQueryRunnerFactory
public TimeBoundaryQueryRunner(Segment segment)
{
- this.cursorFactory = segment.asCursorFactory();
+ this.cursorFactory = segment.as(CursorFactory.class);
this.dataInterval = segment.getDataInterval();
this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class);
}
diff --git
a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
index 2839f310962..e810aae30d8 100644
---
a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
+++
b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
@@ -62,7 +62,7 @@ public class TimeseriesQueryRunnerFactory
@Override
public QueryRunner<Result<TimeseriesResultValue>> createRunner(final Segment
segment)
{
- return new TimeseriesQueryRunner(engine, segment.asCursorFactory(),
segment.as(TimeBoundaryInspector.class));
+ return new TimeseriesQueryRunner(engine, segment.as(CursorFactory.class),
segment.as(TimeBoundaryInspector.class));
}
@Override
diff --git
a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
index 2b1e5f35a3e..f8c7762c004 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
@@ -81,7 +81,7 @@ public class TopNQueryEngine
@Nullable final TopNQueryMetrics queryMetrics
)
{
- final CursorFactory cursorFactory = segment.asCursorFactory();
+ final CursorFactory cursorFactory = segment.as(CursorFactory.class);
if (cursorFactory == null) {
throw new SegmentMissingException(
"Null cursor factory found. Probably trying to issue a query against
a segment being memory unmapped."
diff --git
a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java
b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java
index eefccb0d0ff..0ac4d5d4015 100644
--- a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java
@@ -44,9 +44,9 @@ public class ArrayListSegment<RowType> extends
RowBasedSegment<RowType>
* this, and callers will expect it.
* <p>
* The provided "rowSignature" will be used for reporting available columns
and their capabilities to users of
- * {@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".
+ * {@link #as(Class)} for {@link CursorFactory}. 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 rows objects that comprise this segment. Must be
re-iterable if support for {@link Cursor#reset()}
* is required. Otherwise, does not need to be
re-iterable.
diff --git
a/processing/src/main/java/org/apache/druid/segment/BypassRestrictedSegment.java
b/processing/src/main/java/org/apache/druid/segment/BypassRestrictedSegment.java
index 5ca7e631451..e01515f3b5f 100644
---
a/processing/src/main/java/org/apache/druid/segment/BypassRestrictedSegment.java
+++
b/processing/src/main/java/org/apache/druid/segment/BypassRestrictedSegment.java
@@ -26,9 +26,9 @@ import org.apache.druid.query.policy.Policy;
* caller to apply the policy.
* <p>
* This class is useful when a query engine needs direct access to interfaces
that cannot have policies applied
- * transparently. For example, {@link RestrictedSegment} returns null for
{@link #asQueryableIndex} because it cannot
- * apply policies transparently to a {@link QueryableIndex}. To use one, a
query engine needs to obtain a
- * {@link BypassRestrictedSegment} and apply the policies itself.
+ * transparently. For example, {@link RestrictedSegment} returns null for
{@link #as(Class)} when attempting to use
+ * {@link QueryableIndex} because it cannot apply policies transparently to a
{@link QueryableIndex}. To use one, a
+ * query engine needs to obtain a {@link BypassRestrictedSegment} and apply
the policies itself.
*/
class BypassRestrictedSegment extends WrappedSegmentReference
{
@@ -48,18 +48,6 @@ class BypassRestrictedSegment extends WrappedSegmentReference
return policy;
}
- @Override
- public CursorFactory asCursorFactory()
- {
- return delegate.asCursorFactory();
- }
-
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return delegate.asQueryableIndex();
- }
-
@Override
public <T> T as(Class<T> clazz)
{
diff --git
a/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java
b/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java
index 145f41d19e7..3d5e8da3f15 100644
--- a/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java
@@ -38,19 +38,15 @@ public class FilteredSegment extends WrappedSegmentReference
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)) {
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new FilteredCursorFactory(delegate.as(CursorFactory.class),
filter);
+ } else if (TopNOptimizationInspector.class.equals(clazz)) {
return (T) new SimpleTopNOptimizationInspector(filter == null);
}
- return super.as(clazz);
+ return null;
}
}
diff --git
a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
index 735f6d1e159..b1d2fd647ae 100644
---
a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
+++
b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
@@ -52,23 +52,13 @@ public class IncrementalIndexSegment implements Segment
return index.getInterval();
}
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return null;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new IncrementalIndexCursorFactory(index);
- }
-
@Nullable
@Override
public <T> T as(final Class<T> clazz)
{
- if (TimeBoundaryInspector.class.equals(clazz)) {
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new IncrementalIndexCursorFactory(index);
+ } else if (TimeBoundaryInspector.class.equals(clazz)) {
return (T) new IncrementalIndexTimeBoundaryInspector(index);
} else if (MaxIngestedEventTimeInspector.class.equals(clazz)) {
return (T) new IncrementalIndexMaxIngestedEventTimeInspector(index);
@@ -78,9 +68,8 @@ public class IncrementalIndexSegment implements Segment
return (T) new IncrementalIndexPhysicalSegmentInspector(index);
} else if (TopNOptimizationInspector.class.equals(clazz)) {
return (T) new SimpleTopNOptimizationInspector(true);
- } else {
- return Segment.super.as(clazz);
}
+ return null;
}
@Override
diff --git
a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
index 7c39c8ea240..605660505f0 100644
---
a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
+++
b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
@@ -63,18 +63,6 @@ public class QueryableIndexSegment implements Segment
return index.getDataInterval();
}
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return index;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return cursorFactory;
- }
-
@Override
public void close()
{
@@ -95,7 +83,11 @@ public class QueryableIndexSegment implements Segment
}
}
- if (TimeBoundaryInspector.class.equals(clazz)) {
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) cursorFactory;
+ } else if (QueryableIndex.class.equals(clazz)) {
+ return (T) index;
+ } else if (TimeBoundaryInspector.class.equals(clazz)) {
return (T) timeBoundaryInspector;
} else if (Metadata.class.equals(clazz)) {
return (T) index.getMetadata();
@@ -105,7 +97,7 @@ public class QueryableIndexSegment implements Segment
return (T) new SimpleTopNOptimizationInspector(true);
}
- return Segment.super.as(clazz);
+ return null;
}
@SemanticCreator
diff --git
a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
index 1836efd8ae0..767e63c1bde 100644
---
a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
+++
b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
@@ -107,20 +107,6 @@ public class ReferenceCountingSegment extends
ReferenceCountingCloseableObject<S
return !isClosed() ? baseObject.getDataInterval() : null;
}
- @Override
- @Nullable
- public QueryableIndex asQueryableIndex()
- {
- return !isClosed() ? baseObject.asQueryableIndex() : null;
- }
-
- @Nullable
- @Override
- public CursorFactory asCursorFactory()
- {
- return !isClosed() ? baseObject.asCursorFactory() : null;
- }
-
@Override
public boolean overshadows(ReferenceCountingSegment other)
{
diff --git
a/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java
b/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java
index de1d99e573c..e033aac75a9 100644
---
a/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java
+++
b/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java
@@ -27,7 +27,7 @@ import javax.annotation.Nullable;
/**
* A factory class for creating {@code Cursor} instances with strict adherence
to {@link Policy} restrictions. Created
- * by {@link RestrictedSegment#asCursorFactory()}, and applies policies
transparently.
+ * by {@link RestrictedSegment#as(Class)}, and applies policies transparently.
* <p>
* The {@code CursorFactory} simplifies the process of initializing and
retrieving {@code Cursor} objects while ensuring
* that any cursor created complies with the {@link Policy} restrictions.
diff --git
a/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java
b/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java
index f3f5563d7c3..34625a53f44 100644
--- a/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java
@@ -34,8 +34,8 @@ import java.util.Optional;
/**
* A {@link SegmentReference} wrapper with a {@link Policy} restriction that
is automatically enforced.
- * The policy seamlessly governs queries on the wrapped segment, ensuring
compliance. For example,
- * {@link #asCursorFactory()} returns a policy-enforced {@link
RestrictedCursorFactory}.
+ * The policy seamlessly governs queries on the wrapped segment, ensuring
compliance for supported interfaces. For
+ * example, {@link #as(Class)} with {@link CursorFactory} returns a
policy-enforced {@link RestrictedCursorFactory}.
*
* <p>
* Direct access to the policy or the underlying SegmentReference (the
delegate) is not allowed.
@@ -76,25 +76,12 @@ public class RestrictedSegment implements SegmentReference
return delegate.getDataInterval();
}
- @Override
- public CursorFactory asCursorFactory()
- {
- return new RestrictedCursorFactory(delegate.asCursorFactory(), policy);
- }
-
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return null;
- }
-
@Nullable
@Override
public <T> T as(@Nonnull Class<T> clazz)
{
if (CursorFactory.class.equals(clazz)) {
- return (T) asCursorFactory();
+ return (T) new RestrictedCursorFactory(delegate.as(CursorFactory.class),
policy);
} else if (QueryableIndex.class.equals(clazz)) {
return null;
} else if (TimeBoundaryInspector.class.equals(clazz)) {
diff --git
a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
index 57d9676f571..e7933efd3cb 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java
@@ -46,9 +46,9 @@ public class RowBasedSegment<RowType> implements Segment
* this, and callers will expect it.
*
* The provided "rowSignature" will be used for reporting available columns
and their capabilities to users of
- * {@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".
+ * {@link #as(Class)} to get a {@link CursorFactory}. 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 rowSequence objects that comprise this segment. Must be
re-iterable if support for {@link Cursor#reset()}
* is required. Otherwise, does not need to be
re-iterable.
@@ -81,17 +81,14 @@ public class RowBasedSegment<RowType> implements Segment
@Nullable
@Override
- public QueryableIndex asQueryableIndex()
+ public <T> T as(@Nonnull Class<T> clazz)
{
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new RowBasedCursorFactory<>(rowSequence, rowAdapter,
rowSignature);
+ }
return null;
}
- @Override
- public CursorFactory asCursorFactory()
- {
- return new RowBasedCursorFactory<>(rowSequence, rowAdapter, rowSignature);
- }
-
@Override
public void close()
{
diff --git a/processing/src/main/java/org/apache/druid/segment/Segment.java
b/processing/src/main/java/org/apache/druid/segment/Segment.java
index 0c779d09228..8e5c7529a27 100644
--- a/processing/src/main/java/org/apache/druid/segment/Segment.java
+++ b/processing/src/main/java/org/apache/druid/segment/Segment.java
@@ -19,7 +19,6 @@
package org.apache.druid.segment;
-import org.apache.druid.error.DruidException;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.query.datasourcemetadata.DataSourceMetadataResultValue;
import org.apache.druid.segment.join.table.IndexedTable;
@@ -46,29 +45,15 @@ public interface Segment extends Closeable
Interval getDataInterval();
- @Nullable
- QueryableIndex asQueryableIndex();
-
- @Deprecated
- default StorageAdapter asStorageAdapter()
- {
- throw DruidException.defensive(
- "asStorageAdapter is no longer supported, use Segment.asCursorFactory
to build cursors, or Segment.as(..) to get various metadata information instead"
- );
- }
-
- CursorFactory asCursorFactory();
-
/**
* Request an implementation of a particular interface.
* <p>
- * If the passed-in interface is {@link QueryableIndex} or {@link
CursorFactory}, then this method behaves
- * identically to {@link #asQueryableIndex()} or {@link #asCursorFactory()}.
Other interfaces are only
+ * All implementations of this method should be able to provide a {@link
CursorFactory}. Other interfaces are only
* expected to be requested by callers that have specific knowledge of extra
features provided by specific
* segment types. For example, an extension might provide a custom Segment
type that can offer both
- * StorageAdapter and some new interface. That extension can also offer a
Query that uses that new interface.
+ * CursorFactory and some new interface. That extension can also offer a
Query that uses that new interface.
* <p>
- * Implementations which accept classes other than {@link QueryableIndex} or
{@link CursorFactory} are limited
+ * Implementations which accept classes other than known built-in interfaces
such as {@link CursorFactory} are limited
* to using those classes within the extension. This means that one
extension cannot rely on the `Segment.as`
* behavior of another extension.
*
@@ -89,17 +74,7 @@ public interface Segment extends Closeable
*/
@SuppressWarnings({"unused", "unchecked"})
@Nullable
- default <T> T as(@Nonnull Class<T> clazz)
- {
- if (clazz.equals(CursorFactory.class)) {
- return (T) asCursorFactory();
- } else if (clazz.equals(QueryableIndex.class)) {
- return (T) asQueryableIndex();
- } else if (clazz.equals(StorageAdapter.class)) {
- return (T) asStorageAdapter();
- }
- return null;
- }
+ <T> T as(@Nonnull Class<T> clazz);
default boolean isTombstone()
{
diff --git
a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
deleted file mode 100644
index 5fa7286b7ae..00000000000
--- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.segment;
-
-import org.apache.druid.error.DruidException;
-import org.apache.druid.guice.annotations.PublicApi;
-import org.apache.druid.java.util.common.granularity.Granularity;
-import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.query.QueryMetrics;
-import org.apache.druid.query.filter.Filter;
-import org.apache.druid.segment.column.ColumnCapabilities;
-import org.apache.druid.segment.column.RowSignature;
-import org.apache.druid.segment.data.Indexed;
-import org.apache.druid.segment.vector.VectorCursor;
-import org.joda.time.DateTime;
-import org.joda.time.Interval;
-
-import javax.annotation.Nullable;
-
-/**
- *
- */
-@Deprecated
-@PublicApi
-public interface StorageAdapter extends ColumnInspector
-{
- /**
- * @deprecated Use {@link Segment#asCursorFactory()} and then {@link
CursorFactory#makeCursorHolder(CursorBuildSpec)}
- * and call {@link CursorHolder#canVectorize()} instead.
- */
- @Deprecated
- default boolean canVectorize(
- @Nullable Filter filter,
- VirtualColumns virtualColumns,
- boolean descending
- )
- {
- throw DruidException.defensive(
- "canVectorize is no longer supported, use
Segment.asCursorFactory().makeCursorHolder(..).canVectorize() instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#asCursorFactory()} and then {@link
CursorFactory#makeCursorHolder(CursorBuildSpec)}
- * and call {@link CursorHolder#asCursor()} instead.
- */
- @Deprecated
- default Sequence<Cursor> makeCursors(
- @Nullable Filter filter,
- Interval interval,
- VirtualColumns virtualColumns,
- Granularity gran,
- boolean descending,
- @Nullable QueryMetrics<?> queryMetrics
- )
- {
- throw DruidException.defensive(
- "makeCursors is no longer supported, use
Segment.asCursorFactory().makeCursorHolder(..).asCursor() instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#asCursorFactory()} and then {@link
CursorFactory#makeCursorHolder(CursorBuildSpec)}
- * and call {@link CursorHolder#asVectorCursor()} instead.
- */
- @Deprecated
- @Nullable
- default VectorCursor makeVectorCursor(
- @Nullable Filter filter,
- Interval interval,
- VirtualColumns virtualColumns,
- boolean descending,
- int vectorSize,
- @Nullable QueryMetrics<?> queryMetrics
- )
- {
- throw DruidException.defensive(
- "makeVectorCursor is no longer supported, use
Segment.asCursorFactory().makeCursorHolder(..).asVectorCursor() instead"
- );
- }
-
- /**
- * @deprecated Callers should use {@link Segment#getDataInterval()} instead.
- */
- @Deprecated
- default Interval getInterval()
- {
- throw DruidException.defensive(
- "getInterval is no longer supported, use Segment.getDataInterval()
instead."
- );
- }
-
- /**
- * @deprecated Callers should use {@link Segment#as(Class)} to construct a
{@link Metadata} instead.
- */
- @Deprecated
- default Indexed<String> getAvailableDimensions()
- {
- throw DruidException.defensive(
- "getAvailableDimensions is no longer supported, use
Segment.getRowSignature() and or Segment.as(PhysicalSegmentInspector.class)
instead"
- );
- }
-
- /**
- * @deprecated Callers should use {@link Segment#as(Class)} to construct a
{@link Metadata} if available and check
- * {@link Metadata#getAggregators()} instead.
- */
- @Deprecated
- default Iterable<String> getAvailableMetrics()
- {
- throw DruidException.defensive(
- "getAvailableMetrics is no longer supported, use
Segment.as(PhysicalSegmentInspector.class) instead"
- );
- }
-
- /**
- * @deprecated use {@link Segment#asCursorFactory()} and {@link
CursorFactory#getRowSignature()} instead.
- */
- @Deprecated
- default RowSignature getRowSignature()
- {
- throw DruidException.defensive(
- "getRowSignature is no longer supported, use
Segment.asCursorFactory().getRowSignature() instead"
- );
- }
-
- /**
- * @deprecated Callers should use {@link Segment#as(Class)} to construct a
{@link PhysicalSegmentInspector} if
- * available and call {@link
PhysicalSegmentInspector#getDimensionCardinality(String)} instead.
- */
- @Deprecated
- default int getDimensionCardinality(String column)
- {
- throw DruidException.defensive(
- "getDimensionCardinality is no longer supported, use
Segment.as(SegmentAnalysisInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
TimeBoundaryInspector} if available and call
- * {@link TimeBoundaryInspector#getMinTime()} instead.
- */
- @Deprecated
- default DateTime getMinTime()
- {
- throw DruidException.defensive(
- "getMinTime is no longer supported, use
Segment.as(TimeBoundaryInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
TimeBoundaryInspector} if available and call
- * {@link TimeBoundaryInspector#getMaxTime()} instead.
- */
- @Deprecated
- default DateTime getMaxTime()
- {
- throw DruidException.defensive(
- "getMaxTime is no longer supported, use
Segment.as(TimeBoundaryInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
PhysicalSegmentInspector} if available and call
- * {@link PhysicalSegmentInspector#getMinValue(String)}
- */
- @Deprecated
- @Nullable
- default Comparable getMinValue(String column)
- {
- throw DruidException.defensive(
- "getMinValue is no longer supported, use
Segment.as(SegmentAnalysisInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
PhysicalSegmentInspector} if available and call
- * {@link PhysicalSegmentInspector#getMaxValue(String)}
- */
- @Deprecated
- @Nullable
- default Comparable getMaxValue(String column)
- {
- throw DruidException.defensive(
- "getMaxValue is no longer supported, use
Segment.as(SegmentAnalysisInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#asCursorFactory()} and then {@link
CursorFactory#getColumnCapabilities(String)}
- * instead.
- */
- @Deprecated
- @Override
- @Nullable
- default ColumnCapabilities getColumnCapabilities(String column)
- {
- throw DruidException.defensive(
- "getColumnCapabilities is no longer supported, use
Segment.asCursorFactory().getColumnCapabilities(..) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
PhysicalSegmentInspector} if available then call
- * {@link PhysicalSegmentInspector#getNumRows()} instead.
- */
- @Deprecated
- default int getNumRows()
- {
- throw DruidException.defensive(
- "getNumRows is no longer supported, use
Segment.as(PhysicalSegmentInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
MaxIngestedEventTimeInspector} if available and call
- * {@link MaxIngestedEventTimeInspector#getMaxIngestedEventTime()} instead.
- */
- @Deprecated
- default DateTime getMaxIngestedEventTime()
- {
- throw DruidException.defensive(
- "getMaxIngestedEventTime is no longer supported, use
Segment.as(MaxIngestedEventTimeInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to fetch a {@link Metadata} if
available
- */
- @Deprecated
- @Nullable
- default Metadata getMetadata()
- {
- throw DruidException.defensive(
- "getMetadata is no longer supported, use
Segment.as(PhysicalSegmentInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#as(Class)} to get a {@link
TopNOptimizationInspector} if available and call
- * {@link TopNOptimizationInspector#areAllDictionaryIdsPresent()} instead.
- */
- @Deprecated
- default boolean hasBuiltInFilters()
- {
- throw DruidException.defensive(
- "hasBuiltInFilters is no longer supported, use
Segment.as(FilteredSegmentInspector.class) instead"
- );
- }
-
- /**
- * @deprecated Use {@link Segment#isTombstone()}
- */
- @Deprecated
- default boolean isFromTombstone()
- {
- throw DruidException.defensive(
- "isFromTombstone is no longer supported, use Segment.isTombstone
instead"
- );
- }
-}
diff --git
a/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java
b/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java
index 22b63655f52..5c1db4a433e 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java
@@ -40,20 +40,16 @@ public class UnnestSegment extends WrappedSegmentReference
this.unnestColumn = unnestColumn;
this.filter = filter;
}
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new UnnestCursorFactory(delegate.asCursorFactory(), unnestColumn,
filter);
- }
-
+
@Nullable
@Override
public <T> T as(@Nonnull Class<T> clazz)
{
- if (TopNOptimizationInspector.class.equals(clazz)) {
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new UnnestCursorFactory(delegate.as(CursorFactory.class),
unnestColumn, filter);
+ } else if (TopNOptimizationInspector.class.equals(clazz)) {
return (T) new SimpleTopNOptimizationInspector(filter == null);
}
- return super.as(clazz);
+ return null;
}
}
diff --git
a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
index b9c77b07cad..b430d0cb54d 100644
---
a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
+++
b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java
@@ -23,8 +23,6 @@ import org.apache.druid.query.policy.PolicyEnforcer;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.IOException;
import java.util.Optional;
@@ -62,24 +60,6 @@ public abstract class WrappedSegmentReference implements
SegmentReference
return delegate.getDataInterval();
}
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return delegate.asQueryableIndex();
- }
-
- @Nullable
- @Override
- public <T> T as(@Nonnull Class<T> clazz)
- {
- if (TimeBoundaryInspector.class.equals(clazz)) {
- return (T)
WrappedTimeBoundaryInspector.create(delegate.as(TimeBoundaryInspector.class));
- } else {
- return SegmentReference.super.as(clazz);
- }
- }
-
@Override
public void validateOrElseThrow(PolicyEnforcer policyEnforcer)
{
diff --git
a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
index 3f90e7dddf1..a5e77dd4635 100644
---
a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
+++
b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java
@@ -27,7 +27,6 @@ import org.apache.druid.query.policy.PolicyEnforcer;
import org.apache.druid.query.rowsandcols.CursorFactoryRowsAndColumns;
import org.apache.druid.segment.CloseableShapeshifter;
import org.apache.druid.segment.CursorFactory;
-import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.SimpleTopNOptimizationInspector;
import org.apache.druid.segment.TimeBoundaryInspector;
@@ -96,32 +95,19 @@ public class HashJoinSegment implements SegmentReference
return baseSegment.getDataInterval();
}
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- // Even if baseSegment is a QueryableIndex, we don't want to expose it,
since we've modified its behavior
- // too much while wrapping it.
- return null;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new HashJoinSegmentCursorFactory(
- baseSegment.asCursorFactory(),
- baseFilter,
- clauses,
- joinFilterPreAnalysis
- );
- }
-
@SuppressWarnings("unchecked")
@Override
public <T> T as(Class<T> clazz)
{
- if (CloseableShapeshifter.class.equals(clazz)) {
- return (T) new CursorFactoryRowsAndColumns(asCursorFactory());
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new HashJoinSegmentCursorFactory(
+ baseSegment.as(CursorFactory.class),
+ baseFilter,
+ clauses,
+ joinFilterPreAnalysis
+ );
+ } else if (CloseableShapeshifter.class.equals(clazz)) {
+ return (T) new CursorFactoryRowsAndColumns(as(CursorFactory.class));
} else if (TimeBoundaryInspector.class.equals(clazz)) {
return (T)
WrappedTimeBoundaryInspector.create(baseSegment.as(TimeBoundaryInspector.class));
} else if (TopNOptimizationInspector.class.equals(clazz)) {
@@ -132,9 +118,8 @@ public class HashJoinSegment implements SegmentReference
clause -> clause.getJoinType().isLefty() ||
clause.getCondition().isAlwaysTrue()
)
);
- } else {
- return SegmentReference.super.as(clazz);
}
+ return null;
}
@Override
diff --git
a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
index 9a56ec430cb..a92034bd4ac 100644
---
a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
+++
b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java
@@ -77,7 +77,7 @@ public class BroadcastSegmentIndexedTable implements
IndexedTable
this.version = version;
this.segment = Preconditions.checkNotNull(theSegment, "Segment must not be
null");
this.cursorFactory = Preconditions.checkNotNull(
- segment.asCursorFactory(),
+ segment.as(CursorFactory.class),
"Segment[%s] must have a cursor factory",
segment.getId()
);
diff --git
a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
index fef7a69f238..94654f58c85 100644
---
a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
+++
b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java
@@ -35,6 +35,7 @@ import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
+import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.File;
@@ -84,43 +85,42 @@ public class TombstoneSegmentizerFactory implements
SegmentizerFactory
@Nullable
@Override
- public QueryableIndex asQueryableIndex()
+ public <T> T as(@Nonnull Class<T> clazz)
{
- return queryableIndex;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new CursorFactory()
- {
- @Override
- public CursorHolder makeCursorHolder(CursorBuildSpec spec)
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new CursorFactory()
{
- return new CursorHolder()
+ @Override
+ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
{
- @Nullable
- @Override
- public Cursor asCursor()
+ return new CursorHolder()
{
- return null;
- }
- };
- }
-
- @Override
- public RowSignature getRowSignature()
- {
- return RowSignature.empty();
- }
+ @Nullable
+ @Override
+ public Cursor asCursor()
+ {
+ return null;
+ }
+ };
+ }
+
+ @Override
+ public RowSignature getRowSignature()
+ {
+ return RowSignature.empty();
+ }
- @Override
- @Nullable
- public ColumnCapabilities getColumnCapabilities(String column)
- {
- return null;
- }
- };
+ @Override
+ @Nullable
+ public ColumnCapabilities getColumnCapabilities(String column)
+ {
+ return null;
+ }
+ };
+ } else if (QueryableIndex.class.equals(clazz)) {
+ return (T) queryableIndex;
+ }
+ return null;
}
@Override
diff --git
a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java
b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java
index a3aa832fa2b..8a1cfbee5ff 100644
---
a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java
+++
b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java
@@ -60,7 +60,7 @@ public class FrameComparisonWidgetImplTest extends
InitializedNullHandlingTest
}
},
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
- ).asCursorFactory();
+ ).as(CursorFactory.class);
frameWithoutComplexColumns = Iterables.getOnlyElement(
FrameSequenceBuilder.fromCursorFactory(rowBasedAdapterWithoutComplexColumn)
@@ -80,7 +80,7 @@ public class FrameComparisonWidgetImplTest extends
InitializedNullHandlingTest
}
},
ByteRowKeyComparatorTest.SIGNATURE
- ).asCursorFactory();
+ ).as(CursorFactory.class);
frameWithComplexColumns = Iterables.getOnlyElement(
FrameSequenceBuilder.fromCursorFactory(rowBasedAdapterWithComplexColumn)
diff --git
a/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java
b/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java
index 08978e20b0c..707689b4f96 100644
---
a/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java
+++
b/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java
@@ -67,6 +67,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.Objects;
import java.util.function.Function;
public class FrameCursorFactoryTest
@@ -106,7 +107,7 @@ public class FrameCursorFactoryTest
queryableCursorFactory = new
QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex());
frameSegment =
FrameTestUtil.cursorFactoryToFrameSegment(queryableCursorFactory, frameType);
- frameCursorFactory = frameSegment.asCursorFactory();
+ frameCursorFactory =
Objects.requireNonNull(frameSegment.as(CursorFactory.class));
}
@After
@@ -283,7 +284,7 @@ public class FrameCursorFactoryTest
{
queryableCursorFactory = new
QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex());
frameSegment =
FrameTestUtil.cursorFactoryToFrameSegment(queryableCursorFactory, frameType);
- frameCursorFactory = frameSegment.asCursorFactory();
+ frameCursorFactory =
Objects.requireNonNull(frameSegment.as(CursorFactory.class));
}
@After
diff --git
a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
index f2c9db01257..cf6665cad58 100644
--- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
@@ -75,6 +75,7 @@ import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
+import java.util.Objects;
import java.util.function.Consumer;
import java.util.stream.Collectors;
@@ -519,7 +520,8 @@ public class FrameWriterTest extends
InitializedNullHandlingTest
inputSegment = new FrameSegment(inputFrame,
FrameReader.create(signature));
}
- try (final CursorHolder cursorHolder =
inputSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ try (final CursorHolder cursorHolder =
Objects.requireNonNull(inputSegment.as(CursorFactory.class))
+
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
int numRows = 0;
diff --git
a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index 67e4c5b1abe..3477d3c7db7 100644
---
a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++
b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -130,6 +130,7 @@ import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.search.ContainsSearchQuerySpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
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.TestHelper;
@@ -13661,7 +13662,7 @@ public class GroupByQueryRunnerTest extends
InitializedNullHandlingTest
private void assumeTimeOrdered()
{
try (final CursorHolder cursorHolder =
-
originalRunner.getSegment().asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN))
{
+
originalRunner.getSegment().as(CursorFactory.class).makeCursorHolder(CursorBuildSpec.FULL_SCAN))
{
Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) ==
Order.ASCENDING);
}
}
diff --git
a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
index 1cda0eb016b..149baccd056 100644
---
a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
+++
b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
@@ -36,6 +36,7 @@ import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.column.ColumnType;
@@ -59,6 +60,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.function.BiFunction;
import java.util.function.Supplier;
@@ -650,8 +652,8 @@ public class NestedDataGroupByQueryTest extends
InitializedNullHandlingTest
boolean allCanVectorize = segments.stream()
.allMatch(
s -> {
- final CursorHolder cursorHolder =
s.asCursorFactory()
-
.makeCursorHolder(spec);
+ final CursorHolder cursorHolder =
Objects.requireNonNull(s.as(CursorFactory.class))
+
.makeCursorHolder(spec);
final boolean canVectorize =
cursorHolder.canVectorize();
cursorHolder.close();
return canVectorize;
diff --git
a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
index 4f8d4e79fc7..bfacd11d227 100644
---
a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
+++
b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java
@@ -27,6 +27,7 @@ import org.apache.druid.query.extraction.MapLookupExtractor;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.RowBasedCursorFactory;
@@ -124,25 +125,24 @@ public class LookupSegmentTest
public void test_asQueryableIndex()
{
Assert.assertNull(LOOKUP_SEGMENT.as(QueryableIndex.class));
- Assert.assertNull(LOOKUP_SEGMENT.asQueryableIndex());
}
@Test
- public void test_asStorageAdapter_getRowSignature()
+ public void test_asCursorFactory_getRowSignature()
{
Assert.assertEquals(
RowSignature.builder()
.add("k", ColumnType.STRING)
.add("v", ColumnType.STRING)
.build(),
- LOOKUP_SEGMENT.asCursorFactory().getRowSignature()
+ LOOKUP_SEGMENT.as(CursorFactory.class).getRowSignature()
);
}
@Test
- public void test_asStorageAdapter_getColumnCapabilitiesK()
+ public void test_asCursorFactory_getColumnCapabilitiesK()
{
- final ColumnCapabilities capabilities =
LOOKUP_SEGMENT.asCursorFactory().getColumnCapabilities("k");
+ final ColumnCapabilities capabilities =
LOOKUP_SEGMENT.as(CursorFactory.class).getColumnCapabilities("k");
Assert.assertEquals(ValueType.STRING, capabilities.getType());
@@ -154,9 +154,9 @@ public class LookupSegmentTest
}
@Test
- public void test_asStorageAdapter_getColumnCapabilitiesV()
+ public void test_asCursorFactory_getColumnCapabilitiesV()
{
- final ColumnCapabilities capabilities =
LOOKUP_SEGMENT.asCursorFactory().getColumnCapabilities("v");
+ final ColumnCapabilities capabilities =
LOOKUP_SEGMENT.as(CursorFactory.class).getColumnCapabilities("v");
// Note: the "v" column does not actually have multiple values, but the
RowBasedStorageAdapter doesn't allow
// reporting complete single-valued capabilities. It would be good to
change this in the future, so query engines
@@ -167,12 +167,12 @@ public class LookupSegmentTest
}
@Test
- public void test_asStorageAdapter_makeCursor()
+ public void test_asCursorFactory_makeCursor()
{
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setInterval(Intervals.of("1970/PT1H"))
.build();
- try (final CursorHolder cursorHolder =
LOOKUP_SEGMENT.asCursorFactory().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder =
LOOKUP_SEGMENT.as(CursorFactory.class).makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final List<Pair<String, String>> kvs = new ArrayList<>();
@@ -196,10 +196,10 @@ public class LookupSegmentTest
}
@Test
- public void test_asStorageAdapter_isRowBasedAdapter()
+ public void test_asCursorFactory_isRowBasedAdapter()
{
// This allows us to assume that LookupSegmentTest is further exercising
makeCursor and verifying misc.
// methods like getMinTime, getMaxTime, getMetadata, etc, without checking
them explicitly in _this_ test class.
- MatcherAssert.assertThat(LOOKUP_SEGMENT.asCursorFactory(),
CoreMatchers.instanceOf(RowBasedCursorFactory.class));
+ MatcherAssert.assertThat(LOOKUP_SEGMENT.as(CursorFactory.class),
CoreMatchers.instanceOf(RowBasedCursorFactory.class));
}
}
diff --git
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
index 65929faa61f..4553a0a2e67 100644
---
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
+++
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
@@ -40,6 +40,7 @@ import org.apache.druid.segment.ArrayListSegment;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@@ -289,7 +290,7 @@ public class RowsAndColumnsDecoratorTest extends
SemanticTestBase
if (interval != null) {
builder.setInterval(interval);
}
- try (final CursorHolder cursorHolder =
seggy.asCursorFactory().makeCursorHolder(builder.build())) {
+ try (final CursorHolder cursorHolder =
seggy.as(CursorFactory.class).makeCursorHolder(builder.build())) {
final Cursor cursor = cursorHolder.asCursor();
vals = new ArrayList<>();
diff --git
a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
index 2ecb3627968..7e3269fe5d2 100644
---
a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
+++
b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.filter.FilterTuning;
import org.apache.druid.query.filter.NullFilter;
import org.apache.druid.query.filter.OrDimFilter;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestHelper;
@@ -665,7 +666,7 @@ public class NestedDataTimeseriesQueryTest extends
InitializedNullHandlingTest
boolean allCanVectorize = segments.stream()
.allMatch(
s -> {
- final CursorHolder cursorHolder =
s.asCursorFactory()
+ final CursorHolder cursorHolder =
s.as(CursorFactory.class)
.makeCursorHolder(spec);
final boolean canVectorize =
cursorHolder.canVectorize();
cursorHolder.close();
diff --git
a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
index 06e9a262b6e..93bb462578e 100644
---
a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
+++
b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
@@ -92,6 +92,7 @@ import org.apache.druid.query.lookup.LookupExtractionFn;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
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.TestHelper;
@@ -7291,7 +7292,7 @@ public class TopNQueryRunnerTest extends
InitializedNullHandlingTest
private void assumeTimeOrdered()
{
try (final CursorHolder cursorHolder =
-
runner.getSegment().asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN))
{
+
runner.getSegment().as(CursorFactory.class).makeCursorHolder(CursorBuildSpec.FULL_SCAN))
{
Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) ==
Order.ASCENDING);
}
}
diff --git
a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java
b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java
index b09356ca8a7..646bc97033f 100644
---
a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java
@@ -57,6 +57,7 @@ import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
+import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.nio.ByteBuffer;
@@ -189,17 +190,14 @@ public class CursorHolderPreaggTest extends
InitializedNullHandlingTest
@Nullable
@Override
- public QueryableIndex asQueryableIndex()
+ public <T> T as(@Nonnull Class<T> clazz)
{
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) cursorFactory;
+ }
return null;
}
- @Override
- public CursorFactory asCursorFactory()
- {
- return cursorFactory;
- }
-
@Override
public void close()
{
diff --git
a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
index 774af2c30c2..d33c7e2aa4a 100644
---
a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
@@ -70,25 +70,13 @@ public class ReferenceCountingSegmentTest
return dataInterval;
}
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return index;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return cursorFactory;
- }
-
@Override
public <T> T as(Class<T> clazz)
{
if (clazz.equals(QueryableIndex.class)) {
- return (T) asQueryableIndex();
+ return (T) index;
} else if (clazz.equals(CursorFactory.class)) {
- return (T) asCursorFactory();
+ return (T) cursorFactory;
} else if (clazz.equals(IndexedTable.class)) {
return (T) indexedTable;
}
@@ -171,8 +159,8 @@ public class ReferenceCountingSegmentTest
{
Assert.assertEquals(segmentId, segment.getId());
Assert.assertEquals(dataInterval, segment.getDataInterval());
- Assert.assertEquals(index, segment.asQueryableIndex());
- Assert.assertEquals(cursorFactory, segment.asCursorFactory());
+ Assert.assertEquals(index, segment.as(QueryableIndex.class));
+ Assert.assertEquals(cursorFactory, segment.as(CursorFactory.class));
}
@Test
diff --git
a/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java
b/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java
index 59bde904e3c..15ee60a7e9d 100644
--- a/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java
+++ b/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java
@@ -53,19 +53,6 @@ public class TestSegmentForAs implements Segment
return id.getInterval();
}
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return as(QueryableIndex.class);
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return as(CursorFactory.class);
- }
-
@SuppressWarnings("unchecked")
@Nullable
@Override
diff --git
a/processing/src/test/java/org/apache/druid/segment/TestSegmentUtils.java
b/processing/src/test/java/org/apache/druid/segment/TestSegmentUtils.java
index 690eeab94a7..f5762a16574 100644
--- a/processing/src/test/java/org/apache/druid/segment/TestSegmentUtils.java
+++ b/processing/src/test/java/org/apache/druid/segment/TestSegmentUtils.java
@@ -40,6 +40,7 @@ import org.joda.time.Interval;
import org.junit.Assert;
import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
@@ -170,25 +171,13 @@ public class TestSegmentUtils
return interval;
}
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return INDEX;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new QueryableIndexCursorFactory(INDEX);
- }
-
@Override
public <T> T as(@Nonnull Class<T> clazz)
{
if (clazz.equals(QueryableIndex.class)) {
- return (T) asQueryableIndex();
+ return (T) INDEX;
} else if (clazz.equals(CursorFactory.class)) {
- return (T) asCursorFactory();
+ return (T) new QueryableIndexCursorFactory(INDEX);
}
return null;
}
@@ -224,10 +213,14 @@ public class TestSegmentUtils
);
}
+ @Nullable
@Override
- public CursorFactory asCursorFactory()
+ public <T> T as(@Nonnull Class<T> clazz)
{
- return segment.asCursorFactory();
+ if (CursorFactory.class.isAssignableFrom(clazz)) {
+ return (T) segment.as(CursorFactory.class);
+ }
+ return null;
}
}
diff --git
a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java
b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java
index bddbf7f453f..fa73bf7e983 100644
---
a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java
@@ -621,11 +621,11 @@ public abstract class BaseFilterTest extends
InitializedNullHandlingTest
)
.put(
"rowBasedWithoutTypeSignature",
- input ->
Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asCursorFactory(), ()
-> {})
+ input ->
Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().as(CursorFactory.class),
() -> {})
)
.put(
"rowBasedWithTypeSignature",
- input ->
Pair.of(input.buildRowBasedSegmentWithTypeSignature().asCursorFactory(), () ->
{})
+ input ->
Pair.of(input.buildRowBasedSegmentWithTypeSignature().as(CursorFactory.class),
() -> {})
)
.put("frame (row-based)", input -> {
// remove variant type columns from row frames since
they aren't currently supported
@@ -649,7 +649,7 @@ public abstract class BaseFilterTest extends
InitializedNullHandlingTest
)
);
final FrameSegment segment =
input.buildFrameSegment(FrameType.ROW_BASED);
- return Pair.of(segment.asCursorFactory(), segment);
+ return Pair.of(segment.as(CursorFactory.class), segment);
})
.put("frame (columnar)", input -> {
// remove array type columns from columnar frames since
they aren't currently supported
@@ -673,7 +673,7 @@ public abstract class BaseFilterTest extends
InitializedNullHandlingTest
)
);
final FrameSegment segment =
input.buildFrameSegment(FrameType.COLUMNAR);
- return Pair.of(segment.asCursorFactory(), segment);
+ return Pair.of(segment.as(CursorFactory.class), segment);
})
.build();
diff --git
a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java
b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java
index c3bec0e591a..4c872990909 100644
---
a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java
@@ -31,6 +31,7 @@ import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.OrDimFilter;
import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.TopNOptimizationInspector;
import org.apache.druid.segment.VirtualColumns;
@@ -80,14 +81,14 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
"c1.countryIsoCode",
"c1.countryName"
),
-
Lists.newArrayList(makeFactToCountrySegment().asCursorFactory().getRowSignature().getColumnNames())
+
Lists.newArrayList(makeFactToCountrySegment().as(CursorFactory.class).getRowSignature().getColumnNames())
);
}
@Test
public void test_getColumnCapabilities_factToCountryFactColumn()
{
- final ColumnCapabilities capabilities =
makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("countryIsoCode");
+ final ColumnCapabilities capabilities =
makeFactToCountrySegment().as(CursorFactory.class).getColumnCapabilities("countryIsoCode");
Assert.assertEquals(ValueType.STRING, capabilities.getType());
Assert.assertTrue(capabilities.hasBitmapIndexes());
@@ -99,7 +100,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
@Test
public void test_getColumnCapabilities_factToCountryJoinColumn()
{
- final ColumnCapabilities capabilities =
makeFactToCountrySegment().asCursorFactory().getColumnCapabilities(
+ final ColumnCapabilities capabilities =
makeFactToCountrySegment().as(CursorFactory.class).getColumnCapabilities(
FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode"
);
@@ -113,7 +114,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
@Test
public void test_getColumnCapabilities_factToCountryNonexistentFactColumn()
{
- final ColumnCapabilities capabilities =
makeFactToCountrySegment().asCursorFactory()
+ final ColumnCapabilities capabilities =
makeFactToCountrySegment().as(CursorFactory.class)
.getColumnCapabilities("nonexistent");
Assert.assertNull(capabilities);
@@ -122,7 +123,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
@Test
public void test_getColumnCapabilities_factToCountryNonexistentJoinColumn()
{
- final ColumnCapabilities capabilities =
makeFactToCountrySegment().asCursorFactory()
+ final ColumnCapabilities capabilities =
makeFactToCountrySegment().as(CursorFactory.class)
.getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX +
"nonexistent");
Assert.assertNull(capabilities);
@@ -133,7 +134,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
{
Assert.assertEquals(
"hyperUnique",
-
makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("channel_uniques").getComplexTypeName()
+
makeFactToCountrySegment().as(CursorFactory.class).getColumnCapabilities("channel_uniques").getComplexTypeName()
);
}
@@ -142,7 +143,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
{
Assert.assertEquals(
"COMPLEX<hyperUnique>",
-
makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("channel_uniques").asTypeString()
+
makeFactToCountrySegment().as(CursorFactory.class).getColumnCapabilities("channel_uniques").asTypeString()
);
}
@@ -151,7 +152,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
{
Assert.assertEquals(
"STRING",
- makeFactToCountrySegment().asCursorFactory()
+ makeFactToCountrySegment().as(CursorFactory.class)
.getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName")
.asTypeString()
);
@@ -169,7 +170,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -227,7 +228,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -283,7 +284,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -334,7 +335,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -387,7 +388,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -432,7 +433,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -473,7 +474,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -505,7 +506,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -535,7 +536,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -570,7 +571,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -604,7 +605,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -639,7 +640,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -679,7 +680,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -717,7 +718,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -755,7 +756,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -793,7 +794,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -844,7 +845,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -916,7 +917,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -971,7 +972,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1029,7 +1030,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1095,7 +1096,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1157,7 +1158,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1198,7 +1199,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1260,7 +1261,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1306,7 +1307,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1358,7 +1359,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1407,7 +1408,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1453,7 +1454,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1500,7 +1501,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1554,7 +1555,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
JoinTestHelper.readCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1590,7 +1591,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
JoinTestHelper.readCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1626,7 +1627,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
JoinTestHelper.readCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1662,7 +1663,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
JoinTestHelper.readCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1684,7 +1685,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1714,7 +1715,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1743,7 +1744,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1769,7 +1770,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
baseFilter,
joinableClauses,
joinFilterPreAnalysis
@@ -1806,7 +1807,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
);
JoinTestHelper.verifyCursor(
new HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
baseFilter,
joinableClauses,
joinFilterPreAnalysis
@@ -1841,7 +1842,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
baseFilter,
joinableClauses,
joinFilterPreAnalysis
@@ -1896,7 +1897,7 @@ public class HashJoinSegmentCursorFactoryTest extends
BaseHashJoinSegmentCursorF
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
baseFilter,
joinableClauses,
joinFilterPreAnalysis
diff --git
a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java
b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java
index 55e245bdb1c..d0267719c91 100644
---
a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java
@@ -155,19 +155,6 @@ public class HashJoinSegmentTest extends
InitializedNullHandlingTest
return referencedSegment.getDataInterval();
}
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- return referencedSegment.asQueryableIndex();
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return referencedSegment.asCursorFactory();
- }
-
@Nullable
@Override
public <T> T as(@Nonnull Class<T> clazz)
@@ -233,15 +220,14 @@ public class HashJoinSegmentTest extends
InitializedNullHandlingTest
@Test
public void test_asQueryableIndex()
{
- Assert.assertNull(hashJoinSegment.asQueryableIndex());
Assert.assertNull(hashJoinSegment.as(QueryableIndex.class));
}
@Test
- public void test_asStorageAdapter()
+ public void test_asCursorFactory()
{
Assert.assertThat(
- hashJoinSegment.asCursorFactory(),
+ hashJoinSegment.as(CursorFactory.class),
CoreMatchers.instanceOf(HashJoinSegmentCursorFactory.class)
);
}
diff --git
a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java
b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java
index b59f422bd9a..153fa071cb4 100644
---
a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java
@@ -32,6 +32,7 @@ import org.apache.druid.query.filter.ExpressionDimFilter;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnType;
@@ -75,7 +76,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -148,7 +149,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -200,7 +201,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -263,7 +264,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -328,7 +329,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -394,7 +395,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -467,7 +468,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
virtualColumns
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses.getJoinableClauses(),
joinFilterPreAnalysis
@@ -570,7 +571,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -693,7 +694,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -784,7 +785,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
"Cannot build hash-join matcher on non-equi-join condition:
\"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") ==
countryIsoCode");
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -829,7 +830,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -927,7 +928,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1013,7 +1014,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1083,7 +1084,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1137,7 +1138,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1190,7 +1191,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1242,7 +1243,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1295,7 +1296,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1346,7 +1347,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1401,7 +1402,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1455,7 +1456,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1506,7 +1507,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1556,7 +1557,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1610,7 +1611,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1663,7 +1664,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1715,7 +1716,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1778,7 +1779,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1845,7 +1846,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1914,7 +1915,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -1979,7 +1980,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses.getJoinableClauses(),
joinFilterPreAnalysis
@@ -2055,7 +2056,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses.getJoinableClauses(),
joinFilterPreAnalysis
@@ -2225,7 +2226,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
VirtualColumns.EMPTY
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -2384,7 +2385,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
@@ -2460,7 +2461,7 @@ public class JoinFilterAnalyzerTest extends
BaseHashJoinSegmentCursorFactoryTest
);
HashJoinSegmentCursorFactory cursorFactory = new
HashJoinSegmentCursorFactory(
- factSegment.asCursorFactory(),
+ factSegment.as(CursorFactory.class),
null,
joinableClauses,
joinFilterPreAnalysis
diff --git
a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
index 364e12ccd2d..6852077dbab 100644
---
a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java
@@ -40,9 +40,11 @@ import
org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
import org.apache.druid.timeline.SegmentId;
import org.junit.Test;
+import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.List;
+import java.util.Objects;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@@ -173,10 +175,14 @@ public class PostJoinCursorTest extends
BaseHashJoinSegmentCursorFactoryTest
cursorFactory = new InfiniteCursorFactory(new
QueryableIndexCursorFactory(index), countDownLatch);
}
+ @Nullable
@Override
- public CursorFactory asCursorFactory()
+ public <T> T as(@Nonnull Class<T> clazz)
{
- return cursorFactory;
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) cursorFactory;
+ }
+ return super.as(clazz);
}
}
@@ -265,7 +271,8 @@ public class PostJoinCursorTest extends
BaseHashJoinSegmentCursorFactoryTest
joinFilterPreAnalysis
);
- try (final CursorHolder cursorHolder =
hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ try (final CursorHolder cursorHolder =
Objects.requireNonNull(hashJoinSegment.as(CursorFactory.class))
+
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
Cursor cursor = cursorHolder.asCursor();
((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher()
diff --git
a/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java
b/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java
index 8a877d6f8ed..076da95da30 100644
---
a/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java
@@ -36,6 +36,7 @@ import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
+import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import static org.junit.Assert.assertThrows;
@@ -57,7 +58,7 @@ public class TombstoneSegmentizerFactoryTest
.build();
Segment segment = factory.factorize(tombstone, null, true, null);
- Assert.assertNotNull(segment.asCursorFactory());
+ Assert.assertNotNull(segment.as(CursorFactory.class));
Assert.assertEquals("foo_2021-01-01T00:00:00.000Z_2022-01-01T00:00:00.000Z_1",
segment.getId().toString());
Assert.assertEquals(expectedInterval, segment.getDataInterval());
@@ -72,7 +73,7 @@ public class TombstoneSegmentizerFactoryTest
assertThrows(UnsupportedOperationException.class, () ->
queryableIndex.getColumnHolder(null));
Assert.assertTrue(segment.isTombstone());
- Assert.assertNotNull(segment.asCursorFactory());
+ Assert.assertNotNull(segment.as(CursorFactory.class));
}
@@ -95,43 +96,40 @@ public class TombstoneSegmentizerFactoryTest
@Nullable
@Override
- public QueryableIndex asQueryableIndex()
+ public <T> T as(@Nonnull Class<T> clazz)
{
- return null;
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return new CursorFactory()
- {
- @Override
- public CursorHolder makeCursorHolder(CursorBuildSpec spec)
+ if (CursorFactory.class.equals(clazz)) {
+ return (T) new CursorFactory()
{
- return new CursorHolder()
+ @Override
+ public CursorHolder makeCursorHolder(CursorBuildSpec spec)
{
- @Nullable
- @Override
- public Cursor asCursor()
+ return new CursorHolder()
{
- return null;
- }
- };
- }
-
- @Override
- public RowSignature getRowSignature()
- {
- return RowSignature.empty();
- }
+ @Nullable
+ @Override
+ public Cursor asCursor()
+ {
+ return null;
+ }
+ };
+ }
+
+ @Override
+ public RowSignature getRowSignature()
+ {
+ return RowSignature.empty();
+ }
- @Override
- @Nullable
- public ColumnCapabilities getColumnCapabilities(String column)
- {
- return null;
- }
- };
+ @Override
+ @Nullable
+ public ColumnCapabilities getColumnCapabilities(String column)
+ {
+ return null;
+ }
+ };
+ }
+ return null;
}
@Override
diff --git
a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
index 528364a1c03..6e087d1883a 100644
---
a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
+++
b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
@@ -32,6 +32,7 @@ import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.DoubleColumnSelector;
import org.apache.druid.segment.IndexSpec;
@@ -58,6 +59,7 @@ import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.List;
+import java.util.Objects;
public class NestedFieldColumnSelectorsTest extends InitializedNullHandlingTest
{
@@ -348,7 +350,9 @@ public class NestedFieldColumnSelectorsTest extends
InitializedNullHandlingTest
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setVirtualColumns(virtualColumns)
.build();
- final CursorHolder cursorHolder =
closer.register(segment.asCursorFactory().makeCursorHolder(buildSpec));
+ final CursorHolder cursorHolder = closer.register(
+
Objects.requireNonNull(segment.as(CursorFactory.class)).makeCursorHolder(buildSpec)
+ );
final Cursor cursor = cursorHolder.asCursor();
return cursor.getColumnSelectorFactory();
}
@@ -371,7 +375,9 @@ public class NestedFieldColumnSelectorsTest extends
InitializedNullHandlingTest
Assert.assertEquals(1, segments.size());
Segment segment = segments.get(0);
final CursorBuildSpec buildSpec =
CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build();
- VectorCursor cursor =
closer.register(segment.asCursorFactory().makeCursorHolder(buildSpec)).asVectorCursor();
+ VectorCursor cursor = closer.register(
+
Objects.requireNonNull(segment.as(CursorFactory.class)).makeCursorHolder(buildSpec)
+ ).asVectorCursor();
return cursor.getColumnSelectorFactory();
}
}
diff --git
a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java
b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java
index 68c0e970123..42927b11860 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java
@@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.Query;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
@@ -63,6 +64,7 @@ import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger;
@@ -394,7 +396,7 @@ public class Sink implements Iterable<FireHydrant>,
Overshadowable<Sink>
@GuardedBy("hydrantLock")
private void overwriteIndexDimensions(Segment segment)
{
- RowSignature rowSignature = segment.asCursorFactory().getRowSignature();
+ RowSignature rowSignature =
Objects.requireNonNull(segment.as(CursorFactory.class)).getRowSignature();
for (String dim : rowSignature.getColumnNames()) {
columnsExcludingCurrIndex.add(dim);
rowSignature.getColumnType(dim).ifPresent(type ->
columnTypeExcludingCurrIndex.put(dim, type));
@@ -415,7 +417,7 @@ public class Sink implements Iterable<FireHydrant>,
Overshadowable<Sink>
}
// Add columns from the currHydrant that do not yet exist in
columnsExcludingCurrIndex.
- RowSignature currSignature =
currHydrant.getHydrantSegment().asCursorFactory().getRowSignature();
+ RowSignature currSignature =
currHydrant.getHydrantSegment().as(CursorFactory.class).getRowSignature();
for (String dim : currSignature.getColumnNames()) {
if (!columnsExcludingCurrIndex.contains(dim)) {
diff --git
a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java
b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java
index 67d72c6ced0..36e34767832 100644
---
a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java
+++
b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java
@@ -1251,7 +1251,7 @@ public class SegmentLocalCacheManagerTest
Assert.assertEquals(tombstone.getId(), segment.getId());
Assert.assertEquals(interval, segment.getDataInterval());
- final CursorFactory cursorFactory = segment.asCursorFactory();
+ final CursorFactory cursorFactory = segment.as(CursorFactory.class);
Assert.assertNotNull(cursorFactory);
Assert.assertTrue(segment.isTombstone());
diff --git
a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java
b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java
index 79f31c7c814..21044588814 100644
---
a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java
+++
b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java
@@ -21,9 +21,7 @@ package org.apache.druid.segment.realtime;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
-import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.IncrementalIndexSegment;
-import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentReference;
@@ -37,6 +35,7 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
+import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.IOException;
@@ -191,15 +190,9 @@ public class FireHydrantTest extends
InitializedNullHandlingTest
@Nullable
@Override
- public QueryableIndex asQueryableIndex()
+ public <T> T as(@Nonnull Class<T> clazz)
{
- return incrementalIndexSegment.asQueryableIndex();
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- return incrementalIndexSegment.asCursorFactory();
+ return incrementalIndexSegment.as(clazz);
}
@Override
diff --git
a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java
b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java
index 55a882c3218..e5d0f5240bf 100644
---
a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java
+++
b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java
@@ -31,9 +31,7 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.PhysicalSegmentInspector;
-import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.TestIndex;
@@ -60,7 +58,6 @@ import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;
-import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
@@ -261,19 +258,6 @@ public class SegmentManagerThreadSafetyTest
return segment.getInterval();
}
- @Nullable
- @Override
- public QueryableIndex asQueryableIndex()
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CursorFactory asCursorFactory()
- {
- throw new UnsupportedOperationException();
- }
-
@Override
public <T> T as(Class<T> clazz)
{
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]