mirror of https://github.com/apache/druid.git
Refactoring of Storage Adapters (#4710)
* Factor QueryableIndexColumnSelectorFactory and IncrementalIndexColumnSelectorFactory out of QueryableIndexStorageAdapter and IncrementalIndexStorageAdapter; Add Offset.getBaseReadableOffset(); Remove OffsetHolder interface; Replace Cursor extends ColumnSelectorFactory with composition; Reduce indirection in ColumnValueSelectors created by QueryableIndexColumnSelectorFactory * Don't override clone() in FilteredOffset (the prev. implementation was broken); Some warnings fixed * Simplify Cursors in QueryableIndexStorageAdapter * Address comments * Remove unused and unimplemented methods from GenericColumn interface * Comments
This commit is contained in:
parent
594a66f3c0
commit
4d109a358a
|
@ -176,7 +176,7 @@ public class ExpressionBenchmark
|
||||||
Sequences.map(
|
Sequences.map(
|
||||||
cursors,
|
cursors,
|
||||||
cursor -> {
|
cursor -> {
|
||||||
final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor);
|
final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory());
|
||||||
bufferAggregator.init(aggregationBuffer, 0);
|
bufferAggregator.init(aggregationBuffer, 0);
|
||||||
|
|
||||||
while (!cursor.isDone()) {
|
while (!cursor.isDone()) {
|
||||||
|
|
|
@ -514,7 +514,9 @@ public class FilterPartitionBenchmark
|
||||||
{
|
{
|
||||||
List<String> strings = new ArrayList<String>();
|
List<String> strings = new ArrayList<String>();
|
||||||
List<DimensionSelector> selectors = new ArrayList<>();
|
List<DimensionSelector> selectors = new ArrayList<>();
|
||||||
selectors.add(input.makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null)));
|
selectors.add(
|
||||||
|
input.getColumnSelectorFactory().makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null))
|
||||||
|
);
|
||||||
//selectors.add(input.makeDimensionSelector(new DefaultDimensionSpec("dimB", null)));
|
//selectors.add(input.makeDimensionSelector(new DefaultDimensionSpec("dimB", null)));
|
||||||
while (!input.isDone()) {
|
while (!input.isDone()) {
|
||||||
for (DimensionSelector selector : selectors) {
|
for (DimensionSelector selector : selectors) {
|
||||||
|
@ -540,7 +542,7 @@ public class FilterPartitionBenchmark
|
||||||
public List<Long> apply(Cursor input)
|
public List<Long> apply(Cursor input)
|
||||||
{
|
{
|
||||||
List<Long> longvals = new ArrayList<Long>();
|
List<Long> longvals = new ArrayList<Long>();
|
||||||
LongColumnSelector selector = input.makeLongColumnSelector("sumLongSequential");
|
LongColumnSelector selector = input.getColumnSelectorFactory().makeLongColumnSelector("sumLongSequential");
|
||||||
while (!input.isDone()) {
|
while (!input.isDone()) {
|
||||||
long rowval = selector.getLong();
|
long rowval = selector.getLong();
|
||||||
blackhole.consume(rowval);
|
blackhole.consume(rowval);
|
||||||
|
|
|
@ -144,10 +144,10 @@ public class IncrementalIndexReadBenchmark
|
||||||
Cursor cursor = Sequences.toList(Sequences.limit(cursors, 1), Lists.<Cursor>newArrayList()).get(0);
|
Cursor cursor = Sequences.toList(Sequences.limit(cursors, 1), Lists.<Cursor>newArrayList()).get(0);
|
||||||
|
|
||||||
List<DimensionSelector> selectors = new ArrayList<>();
|
List<DimensionSelector> selectors = new ArrayList<>();
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimSequential"));
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimZipf", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimZipf"));
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimUniform", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimUniform"));
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimSequentialHalfNull", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimSequentialHalfNull"));
|
||||||
|
|
||||||
cursor.reset();
|
cursor.reset();
|
||||||
while (!cursor.isDone()) {
|
while (!cursor.isDone()) {
|
||||||
|
@ -179,10 +179,10 @@ public class IncrementalIndexReadBenchmark
|
||||||
Cursor cursor = Sequences.toList(Sequences.limit(cursors, 1), Lists.<Cursor>newArrayList()).get(0);
|
Cursor cursor = Sequences.toList(Sequences.limit(cursors, 1), Lists.<Cursor>newArrayList()).get(0);
|
||||||
|
|
||||||
List<DimensionSelector> selectors = new ArrayList<>();
|
List<DimensionSelector> selectors = new ArrayList<>();
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimSequential"));
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimZipf", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimZipf"));
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimUniform", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimUniform"));
|
||||||
selectors.add(cursor.makeDimensionSelector(new DefaultDimensionSpec("dimSequentialHalfNull", null)));
|
selectors.add(makeDimensionSelector(cursor, "dimSequentialHalfNull"));
|
||||||
|
|
||||||
cursor.reset();
|
cursor.reset();
|
||||||
while (!cursor.isDone()) {
|
while (!cursor.isDone()) {
|
||||||
|
@ -205,4 +205,9 @@ public class IncrementalIndexReadBenchmark
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static DimensionSelector makeDimensionSelector(Cursor cursor, String name)
|
||||||
|
{
|
||||||
|
return cursor.getColumnSelectorFactory().makeDimensionSelector(new DefaultDimensionSpec(name, null));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -129,19 +129,21 @@ public class ScanQueryEngine
|
||||||
@Override
|
@Override
|
||||||
public Iterator<ScanResultValue> make()
|
public Iterator<ScanResultValue> make()
|
||||||
{
|
{
|
||||||
final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
final LongColumnSelector timestampColumnSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
||||||
|
|
||||||
final List<ColumnSelectorPlus<SelectQueryEngine.SelectColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
final List<ColumnSelectorPlus<SelectQueryEngine.SelectColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
||||||
DimensionHandlerUtils.createColumnSelectorPluses(
|
DimensionHandlerUtils.createColumnSelectorPluses(
|
||||||
STRATEGY_FACTORY,
|
STRATEGY_FACTORY,
|
||||||
Lists.newArrayList(dims),
|
Lists.newArrayList(dims),
|
||||||
cursor
|
cursor.getColumnSelectorFactory()
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
||||||
for (String metric : metrics) {
|
for (String metric : metrics) {
|
||||||
final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric);
|
final ObjectColumnSelector metricSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeObjectColumnSelector(metric);
|
||||||
metSelectors.put(metric, metricSelector);
|
metSelectors.put(metric, metricSelector);
|
||||||
}
|
}
|
||||||
final int batchSize = query.getBatchSize();
|
final int batchSize = query.getBatchSize();
|
||||||
|
|
|
@ -332,7 +332,7 @@ public class GroupByQueryEngine
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
final DimensionSelector selector = cursor.makeDimensionSelector(dimSpec);
|
final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimSpec);
|
||||||
if (selector != null) {
|
if (selector != null) {
|
||||||
if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) {
|
if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) {
|
||||||
throw new UnsupportedOperationException(
|
throw new UnsupportedOperationException(
|
||||||
|
@ -349,7 +349,7 @@ public class GroupByQueryEngine
|
||||||
sizesRequired = new int[aggregatorSpecs.size()];
|
sizesRequired = new int[aggregatorSpecs.size()];
|
||||||
for (int i = 0; i < aggregatorSpecs.size(); ++i) {
|
for (int i = 0; i < aggregatorSpecs.size(); ++i) {
|
||||||
AggregatorFactory aggregatorSpec = aggregatorSpecs.get(i);
|
AggregatorFactory aggregatorSpec = aggregatorSpecs.get(i);
|
||||||
aggregators[i] = aggregatorSpec.factorizeBuffered(cursor);
|
aggregators[i] = aggregatorSpec.factorizeBuffered(cursor.getColumnSelectorFactory());
|
||||||
metricNames[i] = aggregatorSpec.getName();
|
metricNames[i] = aggregatorSpec.getName();
|
||||||
sizesRequired[i] = aggregatorSpec.getMaxIntermediateSize();
|
sizesRequired[i] = aggregatorSpec.getMaxIntermediateSize();
|
||||||
}
|
}
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class GroupByQueryEngineV2
|
||||||
.createColumnSelectorPluses(
|
.createColumnSelectorPluses(
|
||||||
STRATEGY_FACTORY,
|
STRATEGY_FACTORY,
|
||||||
query.getDimensions(),
|
query.getDimensions(),
|
||||||
cursor
|
cursor.getColumnSelectorFactory()
|
||||||
);
|
);
|
||||||
GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
|
GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
|
||||||
|
|
||||||
|
@ -434,7 +434,7 @@ public class GroupByQueryEngineV2
|
||||||
return new BufferHashGrouper<>(
|
return new BufferHashGrouper<>(
|
||||||
Suppliers.ofInstance(buffer),
|
Suppliers.ofInstance(buffer),
|
||||||
keySerde,
|
keySerde,
|
||||||
cursor,
|
cursor.getColumnSelectorFactory(),
|
||||||
query.getAggregatorSpecs()
|
query.getAggregatorSpecs()
|
||||||
.toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]),
|
.toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]),
|
||||||
querySpecificConfig.getBufferGrouperMaxSize(),
|
querySpecificConfig.getBufferGrouperMaxSize(),
|
||||||
|
@ -587,7 +587,7 @@ public class GroupByQueryEngineV2
|
||||||
{
|
{
|
||||||
return new BufferArrayGrouper(
|
return new BufferArrayGrouper(
|
||||||
Suppliers.ofInstance(buffer),
|
Suppliers.ofInstance(buffer),
|
||||||
cursor,
|
cursor.getColumnSelectorFactory(),
|
||||||
query.getAggregatorSpecs()
|
query.getAggregatorSpecs()
|
||||||
.toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]),
|
.toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]),
|
||||||
cardinality
|
cardinality
|
||||||
|
|
|
@ -270,12 +270,9 @@ public class SegmentAnalyzer
|
||||||
@Override
|
@Override
|
||||||
public Long accumulate(Long accumulated, Cursor cursor)
|
public Long accumulate(Long accumulated, Cursor cursor)
|
||||||
{
|
{
|
||||||
DimensionSelector selector = cursor.makeDimensionSelector(
|
DimensionSelector selector = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
columnName,
|
.makeDimensionSelector(new DefaultDimensionSpec(columnName, columnName));
|
||||||
columnName
|
|
||||||
)
|
|
||||||
);
|
|
||||||
if (selector == null) {
|
if (selector == null) {
|
||||||
return accumulated;
|
return accumulated;
|
||||||
}
|
}
|
||||||
|
|
|
@ -117,7 +117,7 @@ public class CursorOnlyStrategy extends SearchStrategy
|
||||||
DimensionHandlerUtils.createColumnSelectorPluses(
|
DimensionHandlerUtils.createColumnSelectorPluses(
|
||||||
SearchQueryRunner.SEARCH_COLUMN_SELECTOR_STRATEGY_FACTORY,
|
SearchQueryRunner.SEARCH_COLUMN_SELECTOR_STRATEGY_FACTORY,
|
||||||
dimsToSearch,
|
dimsToSearch,
|
||||||
cursor
|
cursor.getColumnSelectorFactory()
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -241,13 +241,14 @@ public class SelectQueryEngine
|
||||||
query.isDescending()
|
query.isDescending()
|
||||||
);
|
);
|
||||||
|
|
||||||
final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
final LongColumnSelector timestampColumnSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
||||||
|
|
||||||
final List<ColumnSelectorPlus<SelectColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
final List<ColumnSelectorPlus<SelectColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
||||||
DimensionHandlerUtils.createColumnSelectorPluses(
|
DimensionHandlerUtils.createColumnSelectorPluses(
|
||||||
STRATEGY_FACTORY,
|
STRATEGY_FACTORY,
|
||||||
Lists.newArrayList(dims),
|
Lists.newArrayList(dims),
|
||||||
cursor
|
cursor.getColumnSelectorFactory()
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -257,7 +258,8 @@ public class SelectQueryEngine
|
||||||
|
|
||||||
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
||||||
for (String metric : metrics) {
|
for (String metric : metrics) {
|
||||||
final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric);
|
final ObjectColumnSelector metricSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeObjectColumnSelector(metric);
|
||||||
metSelectors.put(metric, metricSelector);
|
metSelectors.put(metric, metricSelector);
|
||||||
builder.addMetric(metric);
|
builder.addMetric(metric);
|
||||||
}
|
}
|
||||||
|
|
|
@ -103,7 +103,8 @@ public class TimeBoundaryQueryRunnerFactory
|
||||||
if (cursor.isDone()) {
|
if (cursor.isDone()) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
final LongColumnSelector timestampColumnSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
||||||
final DateTime timestamp = DateTimes.utc(timestampColumnSelector.getLong());
|
final DateTime timestamp = DateTimes.utc(timestampColumnSelector.getLong());
|
||||||
return new Result<>(adapter.getInterval().getStart(), timestamp);
|
return new Result<>(adapter.getInterval().getStart(), timestamp);
|
||||||
}
|
}
|
||||||
|
|
|
@ -66,7 +66,7 @@ public class TimeseriesQueryEngine
|
||||||
String[] aggregatorNames = new String[aggregatorSpecs.size()];
|
String[] aggregatorNames = new String[aggregatorSpecs.size()];
|
||||||
|
|
||||||
for (int i = 0; i < aggregatorSpecs.size(); i++) {
|
for (int i = 0; i < aggregatorSpecs.size(); i++) {
|
||||||
aggregators[i] = aggregatorSpecs.get(i).factorize(cursor);
|
aggregators[i] = aggregatorSpecs.get(i).factorize(cursor.getColumnSelectorFactory());
|
||||||
aggregatorNames[i] = aggregatorSpecs.get(i).getName();
|
aggregatorNames[i] = aggregatorSpecs.get(i).getName();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -45,7 +45,7 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
||||||
Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()];
|
Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()];
|
||||||
int aggregatorIndex = 0;
|
int aggregatorIndex = 0;
|
||||||
for (AggregatorFactory spec : aggregatorSpecs) {
|
for (AggregatorFactory spec : aggregatorSpecs) {
|
||||||
aggregators[aggregatorIndex] = spec.factorize(cursor);
|
aggregators[aggregatorIndex] = spec.factorize(cursor.getColumnSelectorFactory());
|
||||||
++aggregatorIndex;
|
++aggregatorIndex;
|
||||||
}
|
}
|
||||||
return aggregators;
|
return aggregators;
|
||||||
|
@ -56,7 +56,7 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
||||||
BufferAggregator[] aggregators = new BufferAggregator[aggregatorSpecs.size()];
|
BufferAggregator[] aggregators = new BufferAggregator[aggregatorSpecs.size()];
|
||||||
int aggregatorIndex = 0;
|
int aggregatorIndex = 0;
|
||||||
for (AggregatorFactory spec : aggregatorSpecs) {
|
for (AggregatorFactory spec : aggregatorSpecs) {
|
||||||
aggregators[aggregatorIndex] = spec.factorizeBuffered(cursor);
|
aggregators[aggregatorIndex] = spec.factorizeBuffered(cursor.getColumnSelectorFactory());
|
||||||
++aggregatorIndex;
|
++aggregatorIndex;
|
||||||
}
|
}
|
||||||
return aggregators;
|
return aggregators;
|
||||||
|
|
|
@ -85,7 +85,7 @@ public class TopNMapFn
|
||||||
final ColumnSelectorPlus selectorPlus = DimensionHandlerUtils.createColumnSelectorPlus(
|
final ColumnSelectorPlus selectorPlus = DimensionHandlerUtils.createColumnSelectorPlus(
|
||||||
STRATEGY_FACTORY,
|
STRATEGY_FACTORY,
|
||||||
query.getDimensionSpec(),
|
query.getDimensionSpec(),
|
||||||
cursor
|
cursor.getColumnSelectorFactory()
|
||||||
);
|
);
|
||||||
|
|
||||||
if (selectorPlus.getSelector() == null) {
|
if (selectorPlus.getSelector() == null) {
|
||||||
|
|
|
@ -27,6 +27,7 @@ import io.druid.extendedset.intset.EmptyIntIterator;
|
||||||
import io.druid.java.util.common.RE;
|
import io.druid.java.util.common.RE;
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.data.Offset;
|
import io.druid.segment.data.Offset;
|
||||||
|
import io.druid.segment.data.ReadableOffset;
|
||||||
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||||
import org.roaringbitmap.IntIterator;
|
import org.roaringbitmap.IntIterator;
|
||||||
|
|
||||||
|
@ -202,6 +203,12 @@ public class BitmapOffset extends Offset
|
||||||
value = valueForReset;
|
value = valueForReset;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ReadableOffset getBaseReadableOffset()
|
||||||
|
{
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("MethodDoesntCallSuperMethod")
|
@SuppressWarnings("MethodDoesntCallSuperMethod")
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
|
|
|
@ -24,8 +24,9 @@ import org.joda.time.DateTime;
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
|
||||||
public interface Cursor extends ColumnSelectorFactory
|
public interface Cursor
|
||||||
{
|
{
|
||||||
|
ColumnSelectorFactory getColumnSelectorFactory();
|
||||||
DateTime getTime();
|
DateTime getTime();
|
||||||
void advance();
|
void advance();
|
||||||
void advanceUninterruptibly();
|
void advanceUninterruptibly();
|
||||||
|
|
|
@ -134,13 +134,14 @@ public final class DimensionHandlerUtils
|
||||||
* @param <ColumnSelectorStrategyClass> The strategy type created by the provided strategy factory.
|
* @param <ColumnSelectorStrategyClass> The strategy type created by the provided strategy factory.
|
||||||
* @param strategyFactory A factory provided by query engines that generates type-handling strategies
|
* @param strategyFactory A factory provided by query engines that generates type-handling strategies
|
||||||
* @param dimensionSpecs The set of columns to generate ColumnSelectorPlus objects for
|
* @param dimensionSpecs The set of columns to generate ColumnSelectorPlus objects for
|
||||||
* @param cursor Used to create value selectors for columns.
|
* @param columnSelectorFactory Used to create value selectors for columns.
|
||||||
* @return An array of ColumnSelectorPlus objects, in the order of the columns specified in dimensionSpecs
|
* @return An array of ColumnSelectorPlus objects, in the order of the columns specified in dimensionSpecs
|
||||||
*/
|
*/
|
||||||
public static <ColumnSelectorStrategyClass extends ColumnSelectorStrategy> ColumnSelectorPlus<ColumnSelectorStrategyClass>[] createColumnSelectorPluses(
|
public static <ColumnSelectorStrategyClass extends ColumnSelectorStrategy>
|
||||||
|
ColumnSelectorPlus<ColumnSelectorStrategyClass>[] createColumnSelectorPluses(
|
||||||
ColumnSelectorStrategyFactory<ColumnSelectorStrategyClass> strategyFactory,
|
ColumnSelectorStrategyFactory<ColumnSelectorStrategyClass> strategyFactory,
|
||||||
List<DimensionSpec> dimensionSpecs,
|
List<DimensionSpec> dimensionSpecs,
|
||||||
ColumnSelectorFactory cursor
|
ColumnSelectorFactory columnSelectorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
int dimCount = dimensionSpecs.size();
|
int dimCount = dimensionSpecs.size();
|
||||||
|
@ -150,12 +151,12 @@ public final class DimensionHandlerUtils
|
||||||
final String dimName = dimSpec.getDimension();
|
final String dimName = dimSpec.getDimension();
|
||||||
final ColumnValueSelector selector = getColumnValueSelectorFromDimensionSpec(
|
final ColumnValueSelector selector = getColumnValueSelectorFromDimensionSpec(
|
||||||
dimSpec,
|
dimSpec,
|
||||||
cursor
|
columnSelectorFactory
|
||||||
);
|
);
|
||||||
ColumnSelectorStrategyClass strategy = makeStrategy(
|
ColumnSelectorStrategyClass strategy = makeStrategy(
|
||||||
strategyFactory,
|
strategyFactory,
|
||||||
dimSpec,
|
dimSpec,
|
||||||
cursor.getColumnCapabilities(dimSpec.getDimension()),
|
columnSelectorFactory.getColumnCapabilities(dimSpec.getDimension()),
|
||||||
selector
|
selector
|
||||||
);
|
);
|
||||||
final ColumnSelectorPlus<ColumnSelectorStrategyClass> selectorPlus = new ColumnSelectorPlus<>(
|
final ColumnSelectorPlus<ColumnSelectorStrategyClass> selectorPlus = new ColumnSelectorPlus<>(
|
||||||
|
|
|
@ -25,7 +25,7 @@ import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.TimeAndDimsHolder;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
@ -217,7 +217,7 @@ public interface DimensionIndexer
|
||||||
*/
|
*/
|
||||||
DimensionSelector makeDimensionSelector(
|
DimensionSelector makeDimensionSelector(
|
||||||
DimensionSpec spec,
|
DimensionSpec spec,
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
TimeAndDimsHolder currEntry,
|
||||||
IncrementalIndex.DimensionDesc desc
|
IncrementalIndex.DimensionDesc desc
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -229,10 +229,7 @@ public interface DimensionIndexer
|
||||||
* @param desc Descriptor object for this dimension within an IncrementalIndex
|
* @param desc Descriptor object for this dimension within an IncrementalIndex
|
||||||
* @return A new object that reads rows from currEntry
|
* @return A new object that reads rows from currEntry
|
||||||
*/
|
*/
|
||||||
LongColumnSelector makeLongColumnSelector(
|
LongColumnSelector makeLongColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc);
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
|
||||||
IncrementalIndex.DimensionDesc desc
|
|
||||||
);
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -242,10 +239,7 @@ public interface DimensionIndexer
|
||||||
* @param desc Descriptor object for this dimension within an IncrementalIndex
|
* @param desc Descriptor object for this dimension within an IncrementalIndex
|
||||||
* @return A new object that reads rows from currEntry
|
* @return A new object that reads rows from currEntry
|
||||||
*/
|
*/
|
||||||
FloatColumnSelector makeFloatColumnSelector(
|
FloatColumnSelector makeFloatColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc);
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
|
||||||
IncrementalIndex.DimensionDesc desc
|
|
||||||
);
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -255,10 +249,7 @@ public interface DimensionIndexer
|
||||||
* @param desc Descriptor object for this dimension within an IncrementalIndex
|
* @param desc Descriptor object for this dimension within an IncrementalIndex
|
||||||
* @return A new object that reads rows from currEntry
|
* @return A new object that reads rows from currEntry
|
||||||
*/
|
*/
|
||||||
DoubleColumnSelector makeDoubleColumnSelector(
|
DoubleColumnSelector makeDoubleColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc);
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
|
||||||
IncrementalIndex.DimensionDesc desc
|
|
||||||
);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compares the row values for this DimensionIndexer's dimension from a TimeAndDims key.
|
* Compares the row values for this DimensionIndexer's dimension from a TimeAndDims key.
|
||||||
|
|
|
@ -26,7 +26,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.TimeAndDimsHolder;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -86,19 +86,16 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
DimensionSpec spec, IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
DimensionSpec spec,
|
||||||
|
TimeAndDimsHolder currEntry,
|
||||||
|
IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new DoubleWrappingDimensionSelector(
|
return new DoubleWrappingDimensionSelector(makeDoubleColumnSelector(currEntry, desc), spec.getExtractionFn());
|
||||||
makeDoubleColumnSelector(currEntry, desc),
|
|
||||||
spec.getExtractionFn()
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LongColumnSelector makeLongColumnSelector(
|
public LongColumnSelector makeLongColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc)
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
final int dimIndex = desc.getIndex();
|
final int dimIndex = desc.getIndex();
|
||||||
class IndexerLongColumnSelector implements LongColumnSelector
|
class IndexerLongColumnSelector implements LongColumnSelector
|
||||||
|
@ -127,9 +124,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatColumnSelector makeFloatColumnSelector(
|
public FloatColumnSelector makeFloatColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc)
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
final int dimIndex = desc.getIndex();
|
final int dimIndex = desc.getIndex();
|
||||||
class IndexerFloatColumnSelector implements FloatColumnSelector
|
class IndexerFloatColumnSelector implements FloatColumnSelector
|
||||||
|
@ -158,9 +153,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DoubleColumnSelector makeDoubleColumnSelector(
|
public DoubleColumnSelector makeDoubleColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc)
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
final int dimIndex = desc.getIndex();
|
final int dimIndex = desc.getIndex();
|
||||||
class IndexerDoubleColumnSelector implements DoubleColumnSelector
|
class IndexerDoubleColumnSelector implements DoubleColumnSelector
|
||||||
|
|
|
@ -27,31 +27,32 @@ import io.druid.query.filter.RowOffsetMatcherFactory;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.data.Offset;
|
import io.druid.segment.data.Offset;
|
||||||
|
import io.druid.segment.data.ReadableOffset;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.historical.HistoricalCursor;
|
|
||||||
import io.druid.segment.historical.OffsetHolder;
|
|
||||||
import org.roaringbitmap.IntIterator;
|
import org.roaringbitmap.IntIterator;
|
||||||
|
|
||||||
public final class FilteredOffset extends Offset
|
public final class FilteredOffset extends Offset
|
||||||
{
|
{
|
||||||
private Offset baseOffset;
|
private final Offset baseOffset;
|
||||||
private final ValueMatcher filterMatcher;
|
private final ValueMatcher filterMatcher;
|
||||||
|
|
||||||
FilteredOffset(
|
FilteredOffset(
|
||||||
HistoricalCursor cursor,
|
Offset baseOffset,
|
||||||
|
ColumnSelectorFactory columnSelectorFactory,
|
||||||
boolean descending,
|
boolean descending,
|
||||||
Filter postFilter,
|
Filter postFilter,
|
||||||
ColumnSelectorBitmapIndexSelector bitmapIndexSelector
|
ColumnSelectorBitmapIndexSelector bitmapIndexSelector
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
this.baseOffset = baseOffset;
|
||||||
RowOffsetMatcherFactory rowOffsetMatcherFactory = new CursorOffsetHolderRowOffsetMatcherFactory(
|
RowOffsetMatcherFactory rowOffsetMatcherFactory = new CursorOffsetHolderRowOffsetMatcherFactory(
|
||||||
cursor,
|
baseOffset.getBaseReadableOffset(),
|
||||||
descending
|
descending
|
||||||
);
|
);
|
||||||
if (postFilter instanceof BooleanFilter) {
|
if (postFilter instanceof BooleanFilter) {
|
||||||
filterMatcher = ((BooleanFilter) postFilter).makeMatcher(
|
filterMatcher = ((BooleanFilter) postFilter).makeMatcher(
|
||||||
bitmapIndexSelector,
|
bitmapIndexSelector,
|
||||||
cursor,
|
columnSelectorFactory,
|
||||||
rowOffsetMatcherFactory
|
rowOffsetMatcherFactory
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
|
@ -60,45 +61,19 @@ public final class FilteredOffset extends Offset
|
||||||
postFilter.getBitmapIndex(bitmapIndexSelector)
|
postFilter.getBitmapIndex(bitmapIndexSelector)
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
filterMatcher = postFilter.makeMatcher(cursor);
|
filterMatcher = postFilter.makeMatcher(columnSelectorFactory);
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void reset(Offset baseOffset)
|
|
||||||
{
|
|
||||||
this.baseOffset = baseOffset;
|
|
||||||
if (baseOffset.withinBounds()) {
|
|
||||||
if (!filterMatcher.matches()) {
|
|
||||||
BaseQuery.checkInterrupted();
|
|
||||||
incrementInterruptibly();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
incrementIfNeededOnCreationOrReset();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void increment()
|
public void increment()
|
||||||
{
|
{
|
||||||
|
while (!Thread.currentThread().isInterrupted()) {
|
||||||
baseOffset.increment();
|
baseOffset.increment();
|
||||||
|
if (!baseOffset.withinBounds() || filterMatcher.matches()) {
|
||||||
while (baseOffset.withinBounds() && !Thread.currentThread().isInterrupted()) {
|
|
||||||
if (filterMatcher.matches()) {
|
|
||||||
return;
|
return;
|
||||||
} else {
|
|
||||||
baseOffset.increment();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void incrementInterruptibly()
|
|
||||||
{
|
|
||||||
baseOffset.increment();
|
|
||||||
while (baseOffset.withinBounds()) {
|
|
||||||
BaseQuery.checkInterrupted();
|
|
||||||
if (filterMatcher.matches()) {
|
|
||||||
return;
|
|
||||||
} else {
|
|
||||||
baseOffset.increment();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -113,14 +88,32 @@ public final class FilteredOffset extends Offset
|
||||||
public void reset()
|
public void reset()
|
||||||
{
|
{
|
||||||
baseOffset.reset();
|
baseOffset.reset();
|
||||||
|
incrementIfNeededOnCreationOrReset();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void incrementIfNeededOnCreationOrReset()
|
||||||
|
{
|
||||||
|
if (baseOffset.withinBounds()) {
|
||||||
|
if (!filterMatcher.matches()) {
|
||||||
|
increment();
|
||||||
|
// increment() returns early if it detects the current Thread is interrupted. It will leave this
|
||||||
|
// FilteredOffset in an illegal state, because it may point to an offset that should be filtered. So must to
|
||||||
|
// call BaseQuery.checkInterrupted() and thereby throw a QueryInterruptedException.
|
||||||
|
BaseQuery.checkInterrupted();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ReadableOffset getBaseReadableOffset()
|
||||||
|
{
|
||||||
|
return baseOffset.getBaseReadableOffset();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
{
|
{
|
||||||
FilteredOffset offset = (FilteredOffset) super.clone();
|
throw new UnsupportedOperationException("FilteredOffset should not be cloned");
|
||||||
offset.baseOffset = offset.baseOffset.clone();
|
|
||||||
return offset;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -138,12 +131,12 @@ public final class FilteredOffset extends Offset
|
||||||
|
|
||||||
private static class CursorOffsetHolderRowOffsetMatcherFactory implements RowOffsetMatcherFactory
|
private static class CursorOffsetHolderRowOffsetMatcherFactory implements RowOffsetMatcherFactory
|
||||||
{
|
{
|
||||||
private final OffsetHolder holder;
|
private final ReadableOffset offset;
|
||||||
private final boolean descending;
|
private final boolean descending;
|
||||||
|
|
||||||
CursorOffsetHolderRowOffsetMatcherFactory(OffsetHolder holder, boolean descending)
|
CursorOffsetHolderRowOffsetMatcherFactory(ReadableOffset offset, boolean descending)
|
||||||
{
|
{
|
||||||
this.holder = holder;
|
this.offset = offset;
|
||||||
this.descending = descending;
|
this.descending = descending;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -168,7 +161,7 @@ public final class FilteredOffset extends Offset
|
||||||
@Override
|
@Override
|
||||||
public boolean matches()
|
public boolean matches()
|
||||||
{
|
{
|
||||||
int currentOffset = holder.getReadableOffset().getOffset();
|
int currentOffset = offset.getOffset();
|
||||||
while (iterOffset > currentOffset && iter.hasNext()) {
|
while (iterOffset > currentOffset && iter.hasNext()) {
|
||||||
iterOffset = iter.next();
|
iterOffset = iter.next();
|
||||||
}
|
}
|
||||||
|
@ -179,8 +172,7 @@ public final class FilteredOffset extends Offset
|
||||||
@Override
|
@Override
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
{
|
{
|
||||||
inspector.visit("holder", holder);
|
inspector.visit("offset", offset);
|
||||||
inspector.visit("offset", holder.getReadableOffset());
|
|
||||||
inspector.visit("iter", iter);
|
inspector.visit("iter", iter);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -192,7 +184,7 @@ public final class FilteredOffset extends Offset
|
||||||
@Override
|
@Override
|
||||||
public boolean matches()
|
public boolean matches()
|
||||||
{
|
{
|
||||||
int currentOffset = holder.getReadableOffset().getOffset();
|
int currentOffset = offset.getOffset();
|
||||||
while (iterOffset < currentOffset && iter.hasNext()) {
|
while (iterOffset < currentOffset && iter.hasNext()) {
|
||||||
iterOffset = iter.next();
|
iterOffset = iter.next();
|
||||||
}
|
}
|
||||||
|
@ -203,8 +195,7 @@ public final class FilteredOffset extends Offset
|
||||||
@Override
|
@Override
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
{
|
{
|
||||||
inspector.visit("holder", holder);
|
inspector.visit("offset", offset);
|
||||||
inspector.visit("offset", holder.getReadableOffset());
|
|
||||||
inspector.visit("iter", iter);
|
inspector.visit("iter", iter);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -26,7 +26,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.TimeAndDimsHolder;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -87,7 +87,7 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
DimensionSpec spec, IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
DimensionSpec spec, TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new FloatWrappingDimensionSelector(
|
return new FloatWrappingDimensionSelector(
|
||||||
|
@ -98,7 +98,7 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LongColumnSelector makeLongColumnSelector(
|
public LongColumnSelector makeLongColumnSelector(
|
||||||
final IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
final TimeAndDimsHolder currEntry,
|
||||||
final IncrementalIndex.DimensionDesc desc
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -130,7 +130,7 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatColumnSelector makeFloatColumnSelector(
|
public FloatColumnSelector makeFloatColumnSelector(
|
||||||
final IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
final TimeAndDimsHolder currEntry,
|
||||||
final IncrementalIndex.DimensionDesc desc
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -161,7 +161,8 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DoubleColumnSelector makeDoubleColumnSelector(
|
public DoubleColumnSelector makeDoubleColumnSelector(
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
final TimeAndDimsHolder currEntry,
|
||||||
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final int dimIndex = desc.getIndex();
|
final int dimIndex = desc.getIndex();
|
||||||
|
|
|
@ -26,7 +26,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.TimeAndDimsHolder;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -87,7 +87,7 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
DimensionSpec spec, IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
DimensionSpec spec, TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new LongWrappingDimensionSelector(
|
return new LongWrappingDimensionSelector(
|
||||||
|
@ -98,7 +98,7 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LongColumnSelector makeLongColumnSelector(
|
public LongColumnSelector makeLongColumnSelector(
|
||||||
final IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
final TimeAndDimsHolder currEntry,
|
||||||
final IncrementalIndex.DimensionDesc desc
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -129,7 +129,7 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatColumnSelector makeFloatColumnSelector(
|
public FloatColumnSelector makeFloatColumnSelector(
|
||||||
final IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
final TimeAndDimsHolder currEntry,
|
||||||
final IncrementalIndex.DimensionDesc desc
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -161,7 +161,8 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DoubleColumnSelector makeDoubleColumnSelector(
|
public DoubleColumnSelector makeDoubleColumnSelector(
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
final TimeAndDimsHolder currEntry,
|
||||||
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final int dimIndex = desc.getIndex();
|
final int dimIndex = desc.getIndex();
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.segment;
|
||||||
|
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.data.Offset;
|
import io.druid.segment.data.Offset;
|
||||||
|
import io.druid.segment.data.ReadableOffset;
|
||||||
|
|
||||||
public class NoFilterOffset extends Offset
|
public class NoFilterOffset extends Offset
|
||||||
{
|
{
|
||||||
|
@ -55,6 +56,12 @@ public class NoFilterOffset extends Offset
|
||||||
currentOffset = initialOffset;
|
currentOffset = initialOffset;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ReadableOffset getBaseReadableOffset()
|
||||||
|
{
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
{
|
{
|
||||||
|
|
|
@ -0,0 +1,385 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.segment;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import io.druid.java.util.common.io.Closer;
|
||||||
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.segment.column.Column;
|
||||||
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
|
import io.druid.segment.column.ComplexColumn;
|
||||||
|
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||||
|
import io.druid.segment.column.GenericColumn;
|
||||||
|
import io.druid.segment.column.ValueType;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import io.druid.segment.data.ReadableOffset;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The basic implementation of {@link ColumnSelectorFactory} over a historical segment (i. e. {@link QueryableIndex}).
|
||||||
|
* It's counterpart for incremental index is {@link io.druid.segment.incremental.IncrementalIndexColumnSelectorFactory}.
|
||||||
|
*/
|
||||||
|
class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
|
{
|
||||||
|
private final QueryableIndex index;
|
||||||
|
private final VirtualColumns virtualColumns;
|
||||||
|
private final boolean descending;
|
||||||
|
private final Closer closer;
|
||||||
|
protected final ReadableOffset offset;
|
||||||
|
|
||||||
|
private final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
|
||||||
|
private final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
|
||||||
|
private final Map<String, Object> objectColumnCache = Maps.newHashMap();
|
||||||
|
|
||||||
|
QueryableIndexColumnSelectorFactory(
|
||||||
|
QueryableIndex index,
|
||||||
|
VirtualColumns virtualColumns,
|
||||||
|
boolean descending,
|
||||||
|
Closer closer,
|
||||||
|
ReadableOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.index = index;
|
||||||
|
this.virtualColumns = virtualColumns;
|
||||||
|
this.descending = descending;
|
||||||
|
this.closer = closer;
|
||||||
|
this.offset = offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(dimensionSpec.getDimension())) {
|
||||||
|
return virtualColumns.makeDimensionSelector(dimensionSpec, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec));
|
||||||
|
}
|
||||||
|
|
||||||
|
private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensionSpec)
|
||||||
|
{
|
||||||
|
final String dimension = dimensionSpec.getDimension();
|
||||||
|
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
|
||||||
|
|
||||||
|
final Column columnDesc = index.getColumn(dimension);
|
||||||
|
if (columnDesc == null) {
|
||||||
|
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
|
return new SingleScanTimeDimSelector(
|
||||||
|
makeLongColumnSelector(dimension),
|
||||||
|
extractionFn,
|
||||||
|
descending
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (columnDesc.getCapabilities().getType() == ValueType.LONG) {
|
||||||
|
return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (columnDesc.getCapabilities().getType() == ValueType.FLOAT) {
|
||||||
|
return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (columnDesc.getCapabilities().getType() == ValueType.DOUBLE) {
|
||||||
|
return new DoubleWrappingDimensionSelector(makeDoubleColumnSelector(dimension), extractionFn);
|
||||||
|
}
|
||||||
|
|
||||||
|
DictionaryEncodedColumn<String> cachedColumn = dictionaryColumnCache.get(dimension);
|
||||||
|
if (cachedColumn == null) {
|
||||||
|
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||||
|
closer.register(cachedColumn);
|
||||||
|
dictionaryColumnCache.put(dimension, cachedColumn);
|
||||||
|
}
|
||||||
|
|
||||||
|
final DictionaryEncodedColumn<String> column = cachedColumn;
|
||||||
|
if (column == null) {
|
||||||
|
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
||||||
|
} else {
|
||||||
|
return column.makeDimensionSelector(offset, extractionFn);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.makeFloatColumnSelector(columnName, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
||||||
|
|
||||||
|
if (cachedMetricVals == null) {
|
||||||
|
Column holder = index.getColumn(columnName);
|
||||||
|
if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) {
|
||||||
|
cachedMetricVals = holder.getGenericColumn();
|
||||||
|
closer.register(cachedMetricVals);
|
||||||
|
genericColumnCache.put(columnName, cachedMetricVals);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedMetricVals == null) {
|
||||||
|
return ZeroFloatColumnSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
return cachedMetricVals.makeFloatSingleValueRowSelector(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DoubleColumnSelector makeDoubleColumnSelector(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.makeDoubleColumnSelector(columnName, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
||||||
|
|
||||||
|
if (cachedMetricVals == null) {
|
||||||
|
Column holder = index.getColumn(columnName);
|
||||||
|
if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) {
|
||||||
|
cachedMetricVals = holder.getGenericColumn();
|
||||||
|
closer.register(cachedMetricVals);
|
||||||
|
genericColumnCache.put(columnName, cachedMetricVals);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedMetricVals == null) {
|
||||||
|
return ZeroDoubleColumnSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
return cachedMetricVals.makeDoubleSingleValueRowSelector(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public LongColumnSelector makeLongColumnSelector(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.makeLongColumnSelector(columnName, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
||||||
|
|
||||||
|
if (cachedMetricVals == null) {
|
||||||
|
Column holder = index.getColumn(columnName);
|
||||||
|
if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) {
|
||||||
|
cachedMetricVals = holder.getGenericColumn();
|
||||||
|
closer.register(cachedMetricVals);
|
||||||
|
genericColumnCache.put(columnName, cachedMetricVals);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedMetricVals == null) {
|
||||||
|
return ZeroLongColumnSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
return cachedMetricVals.makeLongSingleValueRowSelector(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(column)) {
|
||||||
|
return virtualColumns.makeObjectColumnSelector(column, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
Object cachedColumnVals = objectColumnCache.get(column);
|
||||||
|
|
||||||
|
if (cachedColumnVals == null) {
|
||||||
|
Column holder = index.getColumn(column);
|
||||||
|
|
||||||
|
if (holder != null) {
|
||||||
|
final ColumnCapabilities capabilities = holder.getCapabilities();
|
||||||
|
|
||||||
|
if (capabilities.isDictionaryEncoded()) {
|
||||||
|
cachedColumnVals = holder.getDictionaryEncoding();
|
||||||
|
} else if (capabilities.getType() == ValueType.COMPLEX) {
|
||||||
|
cachedColumnVals = holder.getComplexColumn();
|
||||||
|
} else {
|
||||||
|
cachedColumnVals = holder.getGenericColumn();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedColumnVals != null) {
|
||||||
|
closer.register((Closeable) cachedColumnVals);
|
||||||
|
objectColumnCache.put(column, cachedColumnVals);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedColumnVals == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedColumnVals instanceof GenericColumn) {
|
||||||
|
final GenericColumn columnVals = (GenericColumn) cachedColumnVals;
|
||||||
|
final ValueType type = columnVals.getType();
|
||||||
|
|
||||||
|
if (columnVals.hasMultipleValues()) {
|
||||||
|
throw new UnsupportedOperationException(
|
||||||
|
"makeObjectColumnSelector does not support multi-value GenericColumns"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (type == ValueType.FLOAT) {
|
||||||
|
return new ObjectColumnSelector<Float>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<Float> classOfObject()
|
||||||
|
{
|
||||||
|
return Float.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Float get()
|
||||||
|
{
|
||||||
|
return columnVals.getFloatSingleValueRow(offset.getOffset());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
if (type == ValueType.DOUBLE) {
|
||||||
|
return new ObjectColumnSelector<Double>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<Double> classOfObject()
|
||||||
|
{
|
||||||
|
return Double.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Double get()
|
||||||
|
{
|
||||||
|
return columnVals.getDoubleSingleValueRow(offset.getOffset());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
if (type == ValueType.LONG) {
|
||||||
|
return new ObjectColumnSelector<Long>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<Long> classOfObject()
|
||||||
|
{
|
||||||
|
return Long.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Long get()
|
||||||
|
{
|
||||||
|
return columnVals.getLongSingleValueRow(offset.getOffset());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
if (type == ValueType.STRING) {
|
||||||
|
return new ObjectColumnSelector<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<String> classOfObject()
|
||||||
|
{
|
||||||
|
return String.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String get()
|
||||||
|
{
|
||||||
|
return columnVals.getStringSingleValueRow(offset.getOffset());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cachedColumnVals instanceof DictionaryEncodedColumn) {
|
||||||
|
final DictionaryEncodedColumn<String> columnVals = (DictionaryEncodedColumn) cachedColumnVals;
|
||||||
|
if (columnVals.hasMultipleValues()) {
|
||||||
|
return new ObjectColumnSelector<Object>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<Object> classOfObject()
|
||||||
|
{
|
||||||
|
return Object.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Nullable
|
||||||
|
public Object get()
|
||||||
|
{
|
||||||
|
final IndexedInts multiValueRow = columnVals.getMultiValueRow(offset.getOffset());
|
||||||
|
if (multiValueRow.size() == 0) {
|
||||||
|
return null;
|
||||||
|
} else if (multiValueRow.size() == 1) {
|
||||||
|
return columnVals.lookupName(multiValueRow.get(0));
|
||||||
|
} else {
|
||||||
|
final String[] strings = new String[multiValueRow.size()];
|
||||||
|
for (int i = 0; i < multiValueRow.size(); i++) {
|
||||||
|
strings[i] = columnVals.lookupName(multiValueRow.get(i));
|
||||||
|
}
|
||||||
|
return strings;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return new ObjectColumnSelector<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<String> classOfObject()
|
||||||
|
{
|
||||||
|
return String.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String get()
|
||||||
|
{
|
||||||
|
return columnVals.lookupName(columnVals.getSingleValueRow(offset.getOffset()));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final ComplexColumn columnVals = (ComplexColumn) cachedColumnVals;
|
||||||
|
return new ObjectColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class classOfObject()
|
||||||
|
{
|
||||||
|
return columnVals.getClazz();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object get()
|
||||||
|
{
|
||||||
|
return columnVals.getRowValue(offset.getOffset());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Nullable
|
||||||
|
public ColumnCapabilities getColumnCapabilities(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.getColumnCapabilities(columnName);
|
||||||
|
}
|
||||||
|
|
||||||
|
return QueryableIndexStorageAdapter.getColumnCapabilites(index, columnName);
|
||||||
|
}
|
||||||
|
}
|
|
@ -22,7 +22,6 @@ package io.druid.segment;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import io.druid.java.util.common.DateTimes;
|
import io.druid.java.util.common.DateTimes;
|
||||||
|
@ -34,29 +33,22 @@ import io.druid.query.BaseQuery;
|
||||||
import io.druid.query.BitmapResultFactory;
|
import io.druid.query.BitmapResultFactory;
|
||||||
import io.druid.query.DefaultBitmapResultFactory;
|
import io.druid.query.DefaultBitmapResultFactory;
|
||||||
import io.druid.query.QueryMetrics;
|
import io.druid.query.QueryMetrics;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.column.BitmapIndex;
|
import io.druid.segment.column.BitmapIndex;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ComplexColumn;
|
import io.druid.segment.column.ComplexColumn;
|
||||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
|
||||||
import io.druid.segment.column.GenericColumn;
|
import io.druid.segment.column.GenericColumn;
|
||||||
import io.druid.segment.column.ValueType;
|
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
|
||||||
import io.druid.segment.data.Offset;
|
import io.druid.segment.data.Offset;
|
||||||
import io.druid.segment.data.ReadableOffset;
|
import io.druid.segment.data.ReadableOffset;
|
||||||
import io.druid.segment.filter.AndFilter;
|
import io.druid.segment.filter.AndFilter;
|
||||||
import io.druid.segment.historical.HistoricalCursor;
|
import io.druid.segment.historical.HistoricalCursor;
|
||||||
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.Closeable;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -69,9 +61,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
{
|
{
|
||||||
private final QueryableIndex index;
|
private final QueryableIndex index;
|
||||||
|
|
||||||
public QueryableIndexStorageAdapter(
|
public QueryableIndexStorageAdapter(QueryableIndex index)
|
||||||
QueryableIndex index
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
}
|
}
|
||||||
|
@ -140,6 +130,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@Nullable
|
||||||
public Comparable getMinValue(String dimension)
|
public Comparable getMinValue(String dimension)
|
||||||
{
|
{
|
||||||
Column column = index.getColumn(dimension);
|
Column column = index.getColumn(dimension);
|
||||||
|
@ -151,6 +142,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@Nullable
|
||||||
public Comparable getMaxValue(String dimension)
|
public Comparable getMaxValue(String dimension)
|
||||||
{
|
{
|
||||||
Column column = index.getColumn(dimension);
|
Column column = index.getColumn(dimension);
|
||||||
|
@ -168,6 +160,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@Nullable
|
||||||
public ColumnCapabilities getColumnCapabilities(String column)
|
public ColumnCapabilities getColumnCapabilities(String column)
|
||||||
{
|
{
|
||||||
return getColumnCapabilites(index, column);
|
return getColumnCapabilites(index, column);
|
||||||
|
@ -199,7 +192,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
@Nullable QueryMetrics<?> queryMetrics
|
@Nullable QueryMetrics<?> queryMetrics
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Interval actualInterval = interval;
|
|
||||||
|
|
||||||
DateTime minTime = getMinTime();
|
DateTime minTime = getMinTime();
|
||||||
long minDataTimestamp = minTime.getMillis();
|
long minDataTimestamp = minTime.getMillis();
|
||||||
|
@ -207,16 +199,11 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
long maxDataTimestamp = maxTime.getMillis();
|
long maxDataTimestamp = maxTime.getMillis();
|
||||||
final Interval dataInterval = new Interval(minTime, gran.bucketEnd(maxTime));
|
final Interval dataInterval = new Interval(minTime, gran.bucketEnd(maxTime));
|
||||||
|
|
||||||
if (!actualInterval.overlaps(dataInterval)) {
|
if (!interval.overlaps(dataInterval)) {
|
||||||
return Sequences.empty();
|
return Sequences.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (actualInterval.getStart().isBefore(dataInterval.getStart())) {
|
final Interval actualInterval = interval.overlap(dataInterval);
|
||||||
actualInterval = actualInterval.withStart(dataInterval.getStart());
|
|
||||||
}
|
|
||||||
if (actualInterval.getEnd().isAfter(dataInterval.getEnd())) {
|
|
||||||
actualInterval = actualInterval.withEnd(dataInterval.getEnd());
|
|
||||||
}
|
|
||||||
|
|
||||||
final ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
|
final ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
|
||||||
index.getBitmapFactoryForDimensions(),
|
index.getBitmapFactoryForDimensions(),
|
||||||
|
@ -324,7 +311,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ColumnCapabilities getColumnCapabilites(ColumnSelector index, String columnName)
|
@Nullable
|
||||||
|
static ColumnCapabilities getColumnCapabilites(ColumnSelector index, String columnName)
|
||||||
{
|
{
|
||||||
Column columnObj = index.getColumn(columnName);
|
Column columnObj = index.getColumn(columnName);
|
||||||
if (columnObj == null) {
|
if (columnObj == null) {
|
||||||
|
@ -375,10 +363,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
{
|
{
|
||||||
final Offset baseOffset = offset.clone();
|
final Offset baseOffset = offset.clone();
|
||||||
|
|
||||||
final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
|
|
||||||
final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
|
|
||||||
final Map<String, Object> objectColumnCache = Maps.newHashMap();
|
|
||||||
|
|
||||||
final GenericColumn timestamps = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
final GenericColumn timestamps = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||||
|
|
||||||
final Closer closer = Closer.create();
|
final Closer closer = Closer.create();
|
||||||
|
@ -432,29 +416,84 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
final Offset initOffset = offset.clone();
|
final Offset baseCursorOffset = offset.clone();
|
||||||
|
final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory(
|
||||||
|
index,
|
||||||
|
virtualColumns,
|
||||||
|
descending,
|
||||||
|
closer,
|
||||||
|
baseCursorOffset.getBaseReadableOffset()
|
||||||
|
);
|
||||||
final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis());
|
final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis());
|
||||||
|
|
||||||
abstract class QueryableIndexBaseCursor<OffsetType extends Offset> implements HistoricalCursor
|
if (postFilter == null) {
|
||||||
|
return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket);
|
||||||
|
} else {
|
||||||
|
FilteredOffset filteredOffset = new FilteredOffset(
|
||||||
|
baseCursorOffset,
|
||||||
|
columnSelectorFactory,
|
||||||
|
descending,
|
||||||
|
postFilter,
|
||||||
|
bitmapIndexSelector
|
||||||
|
);
|
||||||
|
return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, myBucket);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
),
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class QueryableIndexCursor implements HistoricalCursor
|
||||||
{
|
{
|
||||||
OffsetType cursorOffset;
|
private final Offset cursorOffset;
|
||||||
|
private final ColumnSelectorFactory columnSelectorFactory;
|
||||||
|
private final DateTime bucketStart;
|
||||||
|
|
||||||
|
QueryableIndexCursor(Offset cursorOffset, ColumnSelectorFactory columnSelectorFactory, DateTime bucketStart)
|
||||||
|
{
|
||||||
|
this.cursorOffset = cursorOffset;
|
||||||
|
this.columnSelectorFactory = columnSelectorFactory;
|
||||||
|
this.bucketStart = bucketStart;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OffsetType getOffset()
|
public Offset getOffset()
|
||||||
{
|
{
|
||||||
return cursorOffset;
|
return cursorOffset;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ReadableOffset getReadableOffset()
|
public ColumnSelectorFactory getColumnSelectorFactory()
|
||||||
{
|
{
|
||||||
return cursorOffset;
|
return columnSelectorFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DateTime getTime()
|
public DateTime getTime()
|
||||||
{
|
{
|
||||||
return myBucket;
|
return bucketStart;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void advance()
|
||||||
|
{
|
||||||
|
cursorOffset.increment();
|
||||||
|
// Must call BaseQuery.checkInterrupted() after cursorOffset.increment(), not before, because
|
||||||
|
// FilteredOffset.increment() is a potentially long, not an "instant" operation (unlike to all other subclasses
|
||||||
|
// of Offset) and it returns early on interruption, leaving itself in an illegal state. We should not let
|
||||||
|
// aggregators, etc. access this illegal state and throw a QueryInterruptedException by calling
|
||||||
|
// BaseQuery.checkInterrupted().
|
||||||
|
BaseQuery.checkInterrupted();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void advanceUninterruptibly()
|
||||||
|
{
|
||||||
|
cursorOffset.increment();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -479,459 +518,21 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
return isDone() || Thread.currentThread().isInterrupted();
|
return isDone() || Thread.currentThread().isInterrupted();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public DimensionSelector makeDimensionSelector(
|
|
||||||
DimensionSpec dimensionSpec
|
|
||||||
)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(dimensionSpec.getDimension())) {
|
|
||||||
return virtualColumns.makeDimensionSelector(dimensionSpec, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec));
|
|
||||||
}
|
|
||||||
|
|
||||||
private DimensionSelector makeDimensionSelectorUndecorated(
|
|
||||||
DimensionSpec dimensionSpec
|
|
||||||
)
|
|
||||||
{
|
|
||||||
final String dimension = dimensionSpec.getDimension();
|
|
||||||
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
|
|
||||||
|
|
||||||
final Column columnDesc = index.getColumn(dimension);
|
|
||||||
if (columnDesc == null) {
|
|
||||||
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
|
||||||
return new SingleScanTimeDimSelector(
|
|
||||||
makeLongColumnSelector(dimension),
|
|
||||||
extractionFn,
|
|
||||||
descending
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (columnDesc.getCapabilities().getType() == ValueType.LONG) {
|
|
||||||
return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (columnDesc.getCapabilities().getType() == ValueType.FLOAT) {
|
|
||||||
return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (columnDesc.getCapabilities().getType() == ValueType.DOUBLE) {
|
|
||||||
return new DoubleWrappingDimensionSelector(makeDoubleColumnSelector(dimension), extractionFn);
|
|
||||||
}
|
|
||||||
DictionaryEncodedColumn<String> cachedColumn = dictionaryColumnCache.get(dimension);
|
|
||||||
if (cachedColumn == null) {
|
|
||||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
|
||||||
closer.register(cachedColumn);
|
|
||||||
dictionaryColumnCache.put(dimension, cachedColumn);
|
|
||||||
}
|
|
||||||
|
|
||||||
final DictionaryEncodedColumn<String> column = cachedColumn;
|
|
||||||
if (column == null) {
|
|
||||||
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
|
||||||
} else {
|
|
||||||
return column.makeDimensionSelector(this, extractionFn);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.makeFloatColumnSelector(columnName, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
|
||||||
Column holder = index.getColumn(columnName);
|
|
||||||
if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) {
|
|
||||||
cachedMetricVals = holder.getGenericColumn();
|
|
||||||
closer.register(cachedMetricVals);
|
|
||||||
genericColumnCache.put(columnName, cachedMetricVals);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
|
||||||
return ZeroFloatColumnSelector.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
final GenericColumn metricVals = cachedMetricVals;
|
|
||||||
return new HistoricalFloatColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public float getFloat()
|
|
||||||
{
|
|
||||||
return metricVals.getFloatSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public float get(int offset)
|
|
||||||
{
|
|
||||||
return metricVals.getFloatSingleValueRow(offset);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
inspector.visit("metricVals", metricVals);
|
|
||||||
inspector.visit("cursorOffset", getReadableOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public DoubleColumnSelector makeDoubleColumnSelector(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.makeDoubleColumnSelector(columnName, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
|
||||||
Column holder = index.getColumn(columnName);
|
|
||||||
if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) {
|
|
||||||
cachedMetricVals = holder.getGenericColumn();
|
|
||||||
closer.register(cachedMetricVals);
|
|
||||||
genericColumnCache.put(columnName, cachedMetricVals);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
|
||||||
return ZeroDoubleColumnSelector.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
final GenericColumn metricVals = cachedMetricVals;
|
|
||||||
return new DoubleColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public double getDouble()
|
|
||||||
{
|
|
||||||
return metricVals.getDoubleSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
inspector.visit("metricVals", metricVals);
|
|
||||||
inspector.visit("cursorOffset", getReadableOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LongColumnSelector makeLongColumnSelector(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.makeLongColumnSelector(columnName, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
|
||||||
Column holder = index.getColumn(columnName);
|
|
||||||
if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) {
|
|
||||||
cachedMetricVals = holder.getGenericColumn();
|
|
||||||
closer.register(cachedMetricVals);
|
|
||||||
genericColumnCache.put(columnName, cachedMetricVals);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
|
||||||
return ZeroLongColumnSelector.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
final GenericColumn metricVals = cachedMetricVals;
|
|
||||||
return new LongColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public long getLong()
|
|
||||||
{
|
|
||||||
return metricVals.getLongSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
inspector.visit("metricVals", metricVals);
|
|
||||||
inspector.visit("cursorOffset", getReadableOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(column)) {
|
|
||||||
return virtualColumns.makeObjectColumnSelector(column, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
Object cachedColumnVals = objectColumnCache.get(column);
|
|
||||||
|
|
||||||
if (cachedColumnVals == null) {
|
|
||||||
Column holder = index.getColumn(column);
|
|
||||||
|
|
||||||
if (holder != null) {
|
|
||||||
final ColumnCapabilities capabilities = holder.getCapabilities();
|
|
||||||
|
|
||||||
if (capabilities.isDictionaryEncoded()) {
|
|
||||||
cachedColumnVals = holder.getDictionaryEncoding();
|
|
||||||
} else if (capabilities.getType() == ValueType.COMPLEX) {
|
|
||||||
cachedColumnVals = holder.getComplexColumn();
|
|
||||||
} else {
|
|
||||||
cachedColumnVals = holder.getGenericColumn();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedColumnVals != null) {
|
|
||||||
closer.register((Closeable) cachedColumnVals);
|
|
||||||
objectColumnCache.put(column, cachedColumnVals);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedColumnVals == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedColumnVals instanceof GenericColumn) {
|
|
||||||
final GenericColumn columnVals = (GenericColumn) cachedColumnVals;
|
|
||||||
final ValueType type = columnVals.getType();
|
|
||||||
|
|
||||||
if (columnVals.hasMultipleValues()) {
|
|
||||||
throw new UnsupportedOperationException(
|
|
||||||
"makeObjectColumnSelector does not support multi-value GenericColumns"
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (type == ValueType.FLOAT) {
|
|
||||||
return new ObjectColumnSelector<Float>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return Float.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Float get()
|
|
||||||
{
|
|
||||||
return columnVals.getFloatSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
if (type == ValueType.DOUBLE) {
|
|
||||||
return new ObjectColumnSelector<Double>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return Double.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Double get()
|
|
||||||
{
|
|
||||||
return columnVals.getDoubleSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
if (type == ValueType.LONG) {
|
|
||||||
return new ObjectColumnSelector<Long>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return Long.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Long get()
|
|
||||||
{
|
|
||||||
return columnVals.getLongSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
if (type == ValueType.STRING) {
|
|
||||||
return new ObjectColumnSelector<String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return String.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String get()
|
|
||||||
{
|
|
||||||
return columnVals.getStringSingleValueRow(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cachedColumnVals instanceof DictionaryEncodedColumn) {
|
|
||||||
final DictionaryEncodedColumn<String> columnVals = (DictionaryEncodedColumn) cachedColumnVals;
|
|
||||||
if (columnVals.hasMultipleValues()) {
|
|
||||||
return new ObjectColumnSelector<Object>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return Object.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object get()
|
|
||||||
{
|
|
||||||
int currentOffset = getReadableOffset().getOffset();
|
|
||||||
final IndexedInts multiValueRow = columnVals.getMultiValueRow(currentOffset);
|
|
||||||
if (multiValueRow.size() == 0) {
|
|
||||||
return null;
|
|
||||||
} else if (multiValueRow.size() == 1) {
|
|
||||||
return columnVals.lookupName(multiValueRow.get(0));
|
|
||||||
} else {
|
|
||||||
final String[] strings = new String[multiValueRow.size()];
|
|
||||||
for (int i = 0; i < multiValueRow.size(); i++) {
|
|
||||||
strings[i] = columnVals.lookupName(multiValueRow.get(i));
|
|
||||||
}
|
|
||||||
return strings;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
} else {
|
|
||||||
return new ObjectColumnSelector<String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return String.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String get()
|
|
||||||
{
|
|
||||||
int currentOffset = getReadableOffset().getOffset();
|
|
||||||
return columnVals.lookupName(columnVals.getSingleValueRow(currentOffset));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
final ComplexColumn columnVals = (ComplexColumn) cachedColumnVals;
|
|
||||||
return new ObjectColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return columnVals.getClazz();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object get()
|
|
||||||
{
|
|
||||||
return columnVals.getRowValue(getReadableOffset().getOffset());
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ColumnCapabilities getColumnCapabilities(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.getColumnCapabilities(columnName);
|
|
||||||
}
|
|
||||||
|
|
||||||
return getColumnCapabilites(index, columnName);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (postFilter == null) {
|
|
||||||
return new QueryableIndexBaseCursor<Offset>()
|
|
||||||
{
|
|
||||||
{
|
|
||||||
reset();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void advance()
|
|
||||||
{
|
|
||||||
BaseQuery.checkInterrupted();
|
|
||||||
cursorOffset.increment();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void advanceUninterruptibly()
|
|
||||||
{
|
|
||||||
cursorOffset.increment();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void reset()
|
public void reset()
|
||||||
{
|
{
|
||||||
cursorOffset = initOffset.clone();
|
cursorOffset.reset();
|
||||||
}
|
|
||||||
};
|
|
||||||
} else {
|
|
||||||
return new QueryableIndexBaseCursor<FilteredOffset>()
|
|
||||||
{
|
|
||||||
private Offset baseOffset;
|
|
||||||
|
|
||||||
{
|
|
||||||
cursorOffset = new FilteredOffset(this, descending, postFilter, bitmapIndexSelector);
|
|
||||||
reset();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ReadableOffset getReadableOffset()
|
|
||||||
{
|
|
||||||
return baseOffset;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void advance()
|
|
||||||
{
|
|
||||||
BaseQuery.checkInterrupted();
|
|
||||||
cursorOffset.incrementInterruptibly();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void advanceUninterruptibly()
|
|
||||||
{
|
|
||||||
if (!Thread.currentThread().isInterrupted()) {
|
|
||||||
cursorOffset.increment();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void reset()
|
|
||||||
{
|
|
||||||
baseOffset = initOffset.clone();
|
|
||||||
cursorOffset.reset(baseOffset);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
||||||
),
|
|
||||||
closer
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract static class TimestampCheckingOffset extends Offset
|
public abstract static class TimestampCheckingOffset extends Offset
|
||||||
{
|
{
|
||||||
protected final Offset baseOffset;
|
final Offset baseOffset;
|
||||||
protected final GenericColumn timestamps;
|
final GenericColumn timestamps;
|
||||||
protected final long timeLimit;
|
final long timeLimit;
|
||||||
protected final boolean allWithinThreshold;
|
final boolean allWithinThreshold;
|
||||||
|
|
||||||
public TimestampCheckingOffset(
|
TimestampCheckingOffset(
|
||||||
Offset baseOffset,
|
Offset baseOffset,
|
||||||
GenericColumn timestamps,
|
GenericColumn timestamps,
|
||||||
long timeLimit,
|
long timeLimit,
|
||||||
|
@ -969,6 +570,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
baseOffset.reset();
|
baseOffset.reset();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ReadableOffset getBaseReadableOffset()
|
||||||
|
{
|
||||||
|
return baseOffset.getBaseReadableOffset();
|
||||||
|
}
|
||||||
|
|
||||||
protected abstract boolean timeInRange(long current);
|
protected abstract boolean timeInRange(long current);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -977,6 +584,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
baseOffset.increment();
|
baseOffset.increment();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("MethodDoesntCallSuperMethod")
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
{
|
{
|
||||||
|
@ -994,7 +602,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
|
|
||||||
public static class AscendingTimestampCheckingOffset extends TimestampCheckingOffset
|
public static class AscendingTimestampCheckingOffset extends TimestampCheckingOffset
|
||||||
{
|
{
|
||||||
public AscendingTimestampCheckingOffset(
|
AscendingTimestampCheckingOffset(
|
||||||
Offset baseOffset,
|
Offset baseOffset,
|
||||||
GenericColumn timestamps,
|
GenericColumn timestamps,
|
||||||
long timeLimit,
|
long timeLimit,
|
||||||
|
@ -1017,6 +625,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
"<" + timeLimit + "::" + baseOffset;
|
"<" + timeLimit + "::" + baseOffset;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("MethodDoesntCallSuperMethod")
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
{
|
{
|
||||||
|
@ -1026,7 +635,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
|
|
||||||
public static class DescendingTimestampCheckingOffset extends TimestampCheckingOffset
|
public static class DescendingTimestampCheckingOffset extends TimestampCheckingOffset
|
||||||
{
|
{
|
||||||
public DescendingTimestampCheckingOffset(
|
DescendingTimestampCheckingOffset(
|
||||||
Offset baseOffset,
|
Offset baseOffset,
|
||||||
GenericColumn timestamps,
|
GenericColumn timestamps,
|
||||||
long timeLimit,
|
long timeLimit,
|
||||||
|
@ -1050,6 +659,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
"::" + baseOffset;
|
"::" + baseOffset;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("MethodDoesntCallSuperMethod")
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
{
|
{
|
||||||
|
|
|
@ -48,7 +48,9 @@ public interface StorageAdapter extends CursorFactory
|
||||||
public int getDimensionCardinality(String column);
|
public int getDimensionCardinality(String column);
|
||||||
public DateTime getMinTime();
|
public DateTime getMinTime();
|
||||||
public DateTime getMaxTime();
|
public DateTime getMaxTime();
|
||||||
|
@Nullable
|
||||||
public Comparable getMinValue(String column);
|
public Comparable getMinValue(String column);
|
||||||
|
@Nullable
|
||||||
public Comparable getMaxValue(String column);
|
public Comparable getMaxValue(String column);
|
||||||
public Capabilities getCapabilities();
|
public Capabilities getCapabilities();
|
||||||
|
|
||||||
|
|
|
@ -40,7 +40,7 @@ import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.TimeAndDimsHolder;
|
||||||
import it.unimi.dsi.fastutil.ints.IntArrays;
|
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
import it.unimi.dsi.fastutil.objects.Object2IntMap;
|
import it.unimi.dsi.fastutil.objects.Object2IntMap;
|
||||||
|
@ -370,7 +370,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
final DimensionSpec spec,
|
final DimensionSpec spec,
|
||||||
final IncrementalIndexStorageAdapter.EntryHolder currEntry,
|
final TimeAndDimsHolder currEntry,
|
||||||
final IncrementalIndex.DimensionDesc desc
|
final IncrementalIndex.DimensionDesc desc
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -553,25 +553,19 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LongColumnSelector makeLongColumnSelector(
|
public LongColumnSelector makeLongColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc)
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
return ZeroLongColumnSelector.instance();
|
return ZeroLongColumnSelector.instance();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatColumnSelector makeFloatColumnSelector(
|
public FloatColumnSelector makeFloatColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc)
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
return ZeroFloatColumnSelector.instance();
|
return ZeroFloatColumnSelector.instance();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DoubleColumnSelector makeDoubleColumnSelector(
|
public DoubleColumnSelector makeDoubleColumnSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc)
|
||||||
IncrementalIndexStorageAdapter.EntryHolder currEntry, IncrementalIndex.DimensionDesc desc
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
return ZeroDoubleColumnSelector.instance();
|
return ZeroDoubleColumnSelector.instance();
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,7 @@ package io.druid.segment.column;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.historical.OffsetHolder;
|
import io.druid.segment.data.ReadableOffset;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
|
@ -38,5 +38,5 @@ public interface DictionaryEncodedColumn<ActualType extends Comparable> extends
|
||||||
public int lookupId(ActualType name);
|
public int lookupId(ActualType name);
|
||||||
public int getCardinality();
|
public int getCardinality();
|
||||||
|
|
||||||
DimensionSelector makeDimensionSelector(OffsetHolder offsetHolder, ExtractionFn extractionFn);
|
DimensionSelector makeDimensionSelector(ReadableOffset offset, ExtractionFn extractionFn);
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,9 +21,10 @@ package io.druid.segment.column;
|
||||||
|
|
||||||
import io.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
import io.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||||
import io.druid.query.monomorphicprocessing.HotLoopCallee;
|
import io.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
import io.druid.segment.data.IndexedFloats;
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.data.IndexedLongs;
|
import io.druid.segment.data.ReadableOffset;
|
||||||
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
|
@ -37,18 +38,18 @@ public interface GenericColumn extends HotLoopCallee, Closeable
|
||||||
|
|
||||||
@CalledFromHotLoop
|
@CalledFromHotLoop
|
||||||
public String getStringSingleValueRow(int rowNum);
|
public String getStringSingleValueRow(int rowNum);
|
||||||
|
|
||||||
@CalledFromHotLoop
|
@CalledFromHotLoop
|
||||||
public Indexed<String> getStringMultiValueRow(int rowNum);
|
float getFloatSingleValueRow(int rowNum);
|
||||||
|
HistoricalFloatColumnSelector makeFloatSingleValueRowSelector(ReadableOffset offset);
|
||||||
|
|
||||||
@CalledFromHotLoop
|
@CalledFromHotLoop
|
||||||
public float getFloatSingleValueRow(int rowNum);
|
long getLongSingleValueRow(int rowNum);
|
||||||
@CalledFromHotLoop
|
LongColumnSelector makeLongSingleValueRowSelector(ReadableOffset offset);
|
||||||
public IndexedFloats getFloatMultiValueRow(int rowNum);
|
|
||||||
@CalledFromHotLoop
|
|
||||||
public long getLongSingleValueRow(int rowNum);
|
|
||||||
@CalledFromHotLoop
|
|
||||||
public IndexedLongs getLongMultiValueRow(int rowNum);
|
|
||||||
@CalledFromHotLoop
|
@CalledFromHotLoop
|
||||||
double getDoubleSingleValueRow(int rowNum);
|
double getDoubleSingleValueRow(int rowNum);
|
||||||
|
DoubleColumnSelector makeDoubleSingleValueRowSelector(ReadableOffset offset);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void close();
|
void close();
|
||||||
|
|
|
@ -20,10 +20,11 @@
|
||||||
package io.druid.segment.column;
|
package io.druid.segment.column;
|
||||||
|
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.data.IndexedDoubles;
|
import io.druid.segment.data.IndexedDoubles;
|
||||||
import io.druid.segment.data.IndexedFloats;
|
import io.druid.segment.data.ReadableOffset;
|
||||||
import io.druid.segment.data.IndexedLongs;
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
|
|
||||||
public class IndexedDoublesGenericColumn implements GenericColumn
|
public class IndexedDoublesGenericColumn implements GenericColumn
|
||||||
|
@ -59,12 +60,6 @@ public class IndexedDoublesGenericColumn implements GenericColumn
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Indexed<String> getStringMultiValueRow(int rowNum)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public float getFloatSingleValueRow(int rowNum)
|
public float getFloatSingleValueRow(int rowNum)
|
||||||
{
|
{
|
||||||
|
@ -72,9 +67,9 @@ public class IndexedDoublesGenericColumn implements GenericColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedFloats getFloatMultiValueRow(int rowNum)
|
public HistoricalFloatColumnSelector makeFloatSingleValueRowSelector(ReadableOffset offset)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return column.makeFloatColumnSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -84,9 +79,9 @@ public class IndexedDoublesGenericColumn implements GenericColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedLongs getLongMultiValueRow(int rowNum)
|
public LongColumnSelector makeLongSingleValueRowSelector(ReadableOffset offset)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return column.makeLongColumnSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -95,6 +90,12 @@ public class IndexedDoublesGenericColumn implements GenericColumn
|
||||||
return column.get(rowNum);
|
return column.get(rowNum);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DoubleColumnSelector makeDoubleSingleValueRowSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return column.makeDoubleColumnSelector(offset);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
{
|
{
|
||||||
|
|
|
@ -20,9 +20,11 @@
|
||||||
package io.druid.segment.column;
|
package io.druid.segment.column;
|
||||||
|
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.data.IndexedFloats;
|
import io.druid.segment.data.IndexedFloats;
|
||||||
import io.druid.segment.data.IndexedLongs;
|
import io.druid.segment.data.ReadableOffset;
|
||||||
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -59,12 +61,6 @@ public class IndexedFloatsGenericColumn implements GenericColumn
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Indexed<String> getStringMultiValueRow(int rowNum)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public float getFloatSingleValueRow(int rowNum)
|
public float getFloatSingleValueRow(int rowNum)
|
||||||
{
|
{
|
||||||
|
@ -72,9 +68,9 @@ public class IndexedFloatsGenericColumn implements GenericColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedFloats getFloatMultiValueRow(int rowNum)
|
public HistoricalFloatColumnSelector makeFloatSingleValueRowSelector(ReadableOffset offset)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return column.makeFloatColumnSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -84,9 +80,9 @@ public class IndexedFloatsGenericColumn implements GenericColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedLongs getLongMultiValueRow(int rowNum)
|
public LongColumnSelector makeLongSingleValueRowSelector(ReadableOffset offset)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return column.makeLongColumnSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -95,6 +91,12 @@ public class IndexedFloatsGenericColumn implements GenericColumn
|
||||||
return (double) column.get(rowNum);
|
return (double) column.get(rowNum);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DoubleColumnSelector makeDoubleSingleValueRowSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return column.makeDoubleColumnSelector(offset);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
{
|
{
|
||||||
|
|
|
@ -20,9 +20,11 @@
|
||||||
package io.druid.segment.column;
|
package io.druid.segment.column;
|
||||||
|
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
import io.druid.segment.data.IndexedFloats;
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.data.IndexedLongs;
|
import io.druid.segment.data.IndexedLongs;
|
||||||
|
import io.druid.segment.data.ReadableOffset;
|
||||||
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -59,12 +61,6 @@ public class IndexedLongsGenericColumn implements GenericColumn
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Indexed<String> getStringMultiValueRow(int rowNum)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public float getFloatSingleValueRow(int rowNum)
|
public float getFloatSingleValueRow(int rowNum)
|
||||||
{
|
{
|
||||||
|
@ -72,9 +68,9 @@ public class IndexedLongsGenericColumn implements GenericColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedFloats getFloatMultiValueRow(int rowNum)
|
public HistoricalFloatColumnSelector makeFloatSingleValueRowSelector(ReadableOffset offset)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return column.makeFloatColumnSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -84,9 +80,9 @@ public class IndexedLongsGenericColumn implements GenericColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedLongs getLongMultiValueRow(int rowNum)
|
public LongColumnSelector makeLongSingleValueRowSelector(ReadableOffset offset)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return column.makeLongColumnSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -95,6 +91,12 @@ public class IndexedLongsGenericColumn implements GenericColumn
|
||||||
return (double) column.get(rowNum);
|
return (double) column.get(rowNum);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DoubleColumnSelector makeDoubleSingleValueRowSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return column.makeDoubleColumnSelector(offset);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
{
|
{
|
||||||
|
|
|
@ -31,10 +31,10 @@ import io.druid.segment.IdLookup;
|
||||||
import io.druid.segment.data.CachingIndexed;
|
import io.druid.segment.data.CachingIndexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedMultivalue;
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
|
import io.druid.segment.data.ReadableOffset;
|
||||||
import io.druid.segment.data.SingleIndexedInt;
|
import io.druid.segment.data.SingleIndexedInt;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.historical.HistoricalDimensionSelector;
|
import io.druid.segment.historical.HistoricalDimensionSelector;
|
||||||
import io.druid.segment.historical.OffsetHolder;
|
|
||||||
import io.druid.segment.historical.SingleValueHistoricalDimensionSelector;
|
import io.druid.segment.historical.SingleValueHistoricalDimensionSelector;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -105,10 +105,7 @@ public class SimpleDictionaryEncodedColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public HistoricalDimensionSelector makeDimensionSelector(
|
public HistoricalDimensionSelector makeDimensionSelector(final ReadableOffset offset, final ExtractionFn extractionFn)
|
||||||
final OffsetHolder offsetHolder,
|
|
||||||
final ExtractionFn extractionFn
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
abstract class QueryableDimensionSelector implements HistoricalDimensionSelector, IdLookup
|
abstract class QueryableDimensionSelector implements HistoricalDimensionSelector, IdLookup
|
||||||
{
|
{
|
||||||
|
@ -158,7 +155,7 @@ public class SimpleDictionaryEncodedColumn
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
{
|
{
|
||||||
return multiValueColumn.get(offsetHolder.getReadableOffset().getOffset());
|
return multiValueColumn.get(offset.getOffset());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -183,8 +180,7 @@ public class SimpleDictionaryEncodedColumn
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
{
|
{
|
||||||
inspector.visit("multiValueColumn", multiValueColumn);
|
inspector.visit("multiValueColumn", multiValueColumn);
|
||||||
inspector.visit("offsetHolder", offsetHolder);
|
inspector.visit("offset", offset);
|
||||||
inspector.visit("offset", offsetHolder.getReadableOffset());
|
|
||||||
inspector.visit("extractionFn", extractionFn);
|
inspector.visit("extractionFn", extractionFn);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -202,7 +198,7 @@ public class SimpleDictionaryEncodedColumn
|
||||||
@Override
|
@Override
|
||||||
public int getRowValue()
|
public int getRowValue()
|
||||||
{
|
{
|
||||||
return column.get(offsetHolder.getReadableOffset().getOffset());
|
return column.get(offset.getOffset());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -273,8 +269,7 @@ public class SimpleDictionaryEncodedColumn
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
{
|
{
|
||||||
inspector.visit("column", column);
|
inspector.visit("column", column);
|
||||||
inspector.visit("offsetHolder", offsetHolder);
|
inspector.visit("offset", offset);
|
||||||
inspector.visit("offset", offsetHolder.getReadableOffset());
|
|
||||||
inspector.visit("extractionFn", extractionFn);
|
inspector.visit("extractionFn", extractionFn);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,11 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
public interface IndexedDoubles extends Closeable
|
public interface IndexedDoubles extends Closeable
|
||||||
|
@ -29,5 +34,68 @@ public interface IndexedDoubles extends Closeable
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void close();
|
void close();
|
||||||
|
|
||||||
|
default DoubleColumnSelector makeDoubleColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new DoubleColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public double getDouble()
|
||||||
|
{
|
||||||
|
return IndexedDoubles.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedDoubles.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
default HistoricalFloatColumnSelector makeFloatColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new HistoricalFloatColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public float get(int offset)
|
||||||
|
{
|
||||||
|
return (float) IndexedDoubles.this.get(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
return (float) IndexedDoubles.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedDoubles.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
default LongColumnSelector makeLongColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new LongColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
return (long) IndexedDoubles.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedDoubles.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,11 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -32,4 +37,67 @@ public interface IndexedFloats extends Closeable
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void close();
|
void close();
|
||||||
|
|
||||||
|
default HistoricalFloatColumnSelector makeFloatColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new HistoricalFloatColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
return IndexedFloats.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float get(int offset)
|
||||||
|
{
|
||||||
|
return IndexedFloats.this.get(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedFloats.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
default DoubleColumnSelector makeDoubleColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new DoubleColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public double getDouble()
|
||||||
|
{
|
||||||
|
return IndexedFloats.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedFloats.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
default LongColumnSelector makeLongColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new LongColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
return (long) IndexedFloats.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedFloats.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,11 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
import io.druid.segment.historical.HistoricalFloatColumnSelector;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -32,4 +37,67 @@ public interface IndexedLongs extends Closeable
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void close();
|
void close();
|
||||||
|
|
||||||
|
default LongColumnSelector makeLongColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new LongColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
return IndexedLongs.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedLongs.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
default HistoricalFloatColumnSelector makeFloatColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new HistoricalFloatColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
return (float) IndexedLongs.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float get(int offset)
|
||||||
|
{
|
||||||
|
return (float) IndexedLongs.this.get(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedLongs.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
default DoubleColumnSelector makeDoubleColumnSelector(ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return new DoubleColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public double getDouble()
|
||||||
|
{
|
||||||
|
return (double) IndexedLongs.this.get(offset.getOffset());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("indexed", IndexedLongs.this);
|
||||||
|
inspector.visit("offset", offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,10 @@ import io.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||||
* io.druid.query.topn.Historical1SimpleDoubleAggPooledTopNScannerPrototype} and {@link
|
* io.druid.query.topn.Historical1SimpleDoubleAggPooledTopNScannerPrototype} and {@link
|
||||||
* io.druid.query.topn.HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype} during
|
* io.druid.query.topn.HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype} during
|
||||||
* specialization, and specialized version of those prototypes must be able to any subclass of Offset.
|
* specialization, and specialized version of those prototypes must be able to any subclass of Offset.
|
||||||
|
*
|
||||||
|
* This interface is the core "pointer" interface that is used to create {@link io.druid.segment.ColumnValueSelector}s
|
||||||
|
* over historical segments. It's counterpart for incremental index is {@link
|
||||||
|
* io.druid.segment.incremental.TimeAndDimsHolder}.
|
||||||
*/
|
*/
|
||||||
@SubclassesMustBePublic
|
@SubclassesMustBePublic
|
||||||
public abstract class Offset implements ReadableOffset, Cloneable
|
public abstract class Offset implements ReadableOffset, Cloneable
|
||||||
|
@ -48,6 +52,13 @@ public abstract class Offset implements ReadableOffset, Cloneable
|
||||||
*/
|
*/
|
||||||
public abstract void reset();
|
public abstract void reset();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the same offset ("this") or a readable "view" of this offset, which always returns the same value from
|
||||||
|
* {@link #getOffset()}, as this offset. This method is useful for "unwrapping" such offsets as {@link
|
||||||
|
* io.druid.segment.FilteredOffset} and reduce reference indirection, when only {@link ReadableOffset} API is needed.
|
||||||
|
*/
|
||||||
|
public abstract ReadableOffset getBaseReadableOffset();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Offset clone()
|
public Offset clone()
|
||||||
{
|
{
|
||||||
|
|
|
@ -20,7 +20,9 @@
|
||||||
package io.druid.segment.historical;
|
package io.druid.segment.historical;
|
||||||
|
|
||||||
import io.druid.segment.Cursor;
|
import io.druid.segment.Cursor;
|
||||||
|
import io.druid.segment.data.Offset;
|
||||||
|
|
||||||
public interface HistoricalCursor extends Cursor, OffsetHolder
|
public interface HistoricalCursor extends Cursor
|
||||||
{
|
{
|
||||||
|
Offset getOffset();
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,35 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.historical;
|
|
||||||
|
|
||||||
import io.druid.segment.data.Offset;
|
|
||||||
import io.druid.segment.data.ReadableOffset;
|
|
||||||
|
|
||||||
public interface OffsetHolder
|
|
||||||
{
|
|
||||||
Offset getOffset();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Should return the same, or a "view" of the same offset as {@link #getOffset()}. The difference is that smaller
|
|
||||||
* interface allows to return unwrapped underlying offset sometimes, e. g. {@link
|
|
||||||
* io.druid.segment.FilteredOffset#baseOffset}, instead of the wrapper {@link io.druid.segment.FilteredOffset}.
|
|
||||||
*/
|
|
||||||
ReadableOffset getReadableOffset();
|
|
||||||
}
|
|
|
@ -0,0 +1,341 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental;
|
||||||
|
|
||||||
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.DimensionIndexer;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
|
import io.druid.segment.DoubleColumnSelector;
|
||||||
|
import io.druid.segment.DoubleWrappingDimensionSelector;
|
||||||
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
import io.druid.segment.FloatWrappingDimensionSelector;
|
||||||
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
import io.druid.segment.LongWrappingDimensionSelector;
|
||||||
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
|
import io.druid.segment.SingleScanTimeDimSelector;
|
||||||
|
import io.druid.segment.VirtualColumns;
|
||||||
|
import io.druid.segment.ZeroDoubleColumnSelector;
|
||||||
|
import io.druid.segment.ZeroFloatColumnSelector;
|
||||||
|
import io.druid.segment.ZeroLongColumnSelector;
|
||||||
|
import io.druid.segment.column.Column;
|
||||||
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
|
import io.druid.segment.column.ValueType;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The basic implementation of {@link ColumnSelectorFactory} over an {@link IncrementalIndex}. It's counterpart for
|
||||||
|
* historical segments is {@link io.druid.segment.QueryableIndexColumnSelectorFactory}.
|
||||||
|
*/
|
||||||
|
class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
|
{
|
||||||
|
private final IncrementalIndex<?> index;
|
||||||
|
private final VirtualColumns virtualColumns;
|
||||||
|
private final boolean descending;
|
||||||
|
private final TimeAndDimsHolder timeAndDimsHolder;
|
||||||
|
|
||||||
|
IncrementalIndexColumnSelectorFactory(
|
||||||
|
IncrementalIndex<?> index,
|
||||||
|
VirtualColumns virtualColumns,
|
||||||
|
boolean descending,
|
||||||
|
TimeAndDimsHolder timeAndDimsHolder
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.index = index;
|
||||||
|
this.virtualColumns = virtualColumns;
|
||||||
|
this.descending = descending;
|
||||||
|
this.timeAndDimsHolder = timeAndDimsHolder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(dimensionSpec.getDimension())) {
|
||||||
|
return virtualColumns.makeDimensionSelector(dimensionSpec, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec));
|
||||||
|
}
|
||||||
|
|
||||||
|
private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensionSpec)
|
||||||
|
{
|
||||||
|
final String dimension = dimensionSpec.getDimension();
|
||||||
|
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
|
||||||
|
|
||||||
|
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
|
return new SingleScanTimeDimSelector(
|
||||||
|
makeLongColumnSelector(dimension),
|
||||||
|
extractionFn,
|
||||||
|
descending
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension());
|
||||||
|
if (dimensionDesc == null) {
|
||||||
|
// not a dimension, column may be a metric
|
||||||
|
ColumnCapabilities capabilities = getColumnCapabilities(dimension);
|
||||||
|
if (capabilities == null) {
|
||||||
|
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
||||||
|
}
|
||||||
|
if (capabilities.getType() == ValueType.LONG) {
|
||||||
|
return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn);
|
||||||
|
}
|
||||||
|
if (capabilities.getType() == ValueType.FLOAT) {
|
||||||
|
return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn);
|
||||||
|
}
|
||||||
|
if (capabilities.getType() == ValueType.DOUBLE) {
|
||||||
|
return new DoubleWrappingDimensionSelector(makeDoubleColumnSelector(dimension), extractionFn);
|
||||||
|
}
|
||||||
|
|
||||||
|
// if we can't wrap the base column, just return a column of all nulls
|
||||||
|
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
||||||
|
} else {
|
||||||
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
return indexer.makeDimensionSelector(dimensionSpec, timeAndDimsHolder, dimensionDesc);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.makeFloatColumnSelector(columnName, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer dimIndex = index.getDimensionIndex(columnName);
|
||||||
|
if (dimIndex != null) {
|
||||||
|
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName);
|
||||||
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
return indexer.makeFloatColumnSelector(timeAndDimsHolder, dimensionDesc);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||||
|
if (metricIndexInt == null) {
|
||||||
|
return ZeroFloatColumnSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
final int metricIndex = metricIndexInt;
|
||||||
|
return new FloatColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
return index.getMetricFloatValue(timeAndDimsHolder.getValue(), metricIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("index", index);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DoubleColumnSelector makeDoubleColumnSelector(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.makeDoubleColumnSelector(columnName, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer dimIndex = index.getDimensionIndex(columnName);
|
||||||
|
if (dimIndex != null) {
|
||||||
|
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName);
|
||||||
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
return indexer.makeDoubleColumnSelector(timeAndDimsHolder, dimensionDesc);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||||
|
if (metricIndexInt == null) {
|
||||||
|
return ZeroDoubleColumnSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
final int metricIndex = metricIndexInt;
|
||||||
|
return new DoubleColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public double getDouble()
|
||||||
|
{
|
||||||
|
return index.getMetricDoubleValue(timeAndDimsHolder.getValue(), metricIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("index", index);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public LongColumnSelector makeLongColumnSelector(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.makeLongColumnSelector(columnName, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (columnName.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
|
class TimeLongColumnSelector implements LongColumnSelector
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
return timeAndDimsHolder.getKey().getTimestamp();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
// nothing to inspect
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new TimeLongColumnSelector();
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer dimIndex = index.getDimensionIndex(columnName);
|
||||||
|
if (dimIndex != null) {
|
||||||
|
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName);
|
||||||
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
return indexer.makeLongColumnSelector(timeAndDimsHolder, dimensionDesc);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||||
|
if (metricIndexInt == null) {
|
||||||
|
return ZeroLongColumnSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
final int metricIndex = metricIndexInt;
|
||||||
|
|
||||||
|
return new LongColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
return index.getMetricLongValue(timeAndDimsHolder.getValue(), metricIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("index", index);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(column)) {
|
||||||
|
return virtualColumns.makeObjectColumnSelector(column, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (column.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
|
return new ObjectColumnSelector<Long>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<Long> classOfObject()
|
||||||
|
{
|
||||||
|
return Long.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Long get()
|
||||||
|
{
|
||||||
|
return timeAndDimsHolder.getKey().getTimestamp();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
final Integer metricIndexInt = index.getMetricIndex(column);
|
||||||
|
if (metricIndexInt != null) {
|
||||||
|
final int metricIndex = metricIndexInt;
|
||||||
|
final Class classOfObject = index.getMetricClass(column);
|
||||||
|
return new ObjectColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class classOfObject()
|
||||||
|
{
|
||||||
|
return classOfObject;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object get()
|
||||||
|
{
|
||||||
|
return index.getMetricObjectValue(
|
||||||
|
timeAndDimsHolder.getValue(),
|
||||||
|
metricIndex
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(column);
|
||||||
|
|
||||||
|
if (dimensionDesc == null) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
|
||||||
|
final int dimensionIndex = dimensionDesc.getIndex();
|
||||||
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
|
||||||
|
return new ObjectColumnSelector<Object>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class classOfObject()
|
||||||
|
{
|
||||||
|
return Object.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object get()
|
||||||
|
{
|
||||||
|
IncrementalIndex.TimeAndDims key = timeAndDimsHolder.getKey();
|
||||||
|
if (key == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
Object[] dims = key.getDims();
|
||||||
|
if (dimensionIndex >= dims.length) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
return indexer.convertUnsortedEncodedKeyComponentToActualArrayOrList(
|
||||||
|
dims[dimensionIndex], DimensionIndexer.ARRAY
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public ColumnCapabilities getColumnCapabilities(String columnName)
|
||||||
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.getColumnCapabilities(columnName);
|
||||||
|
}
|
||||||
|
|
||||||
|
return index.getCapabilities(columnName);
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package io.druid.segment.incremental;
|
package io.druid.segment.incremental;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -28,33 +27,17 @@ import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.query.BaseQuery;
|
import io.druid.query.BaseQuery;
|
||||||
import io.druid.query.QueryMetrics;
|
import io.druid.query.QueryMetrics;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
|
||||||
import io.druid.segment.Capabilities;
|
import io.druid.segment.Capabilities;
|
||||||
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.Cursor;
|
import io.druid.segment.Cursor;
|
||||||
import io.druid.segment.DimensionIndexer;
|
import io.druid.segment.DimensionIndexer;
|
||||||
import io.druid.segment.DimensionSelector;
|
|
||||||
import io.druid.segment.DimensionSelectorUtils;
|
|
||||||
import io.druid.segment.DoubleColumnSelector;
|
|
||||||
import io.druid.segment.DoubleWrappingDimensionSelector;
|
|
||||||
import io.druid.segment.FloatColumnSelector;
|
|
||||||
import io.druid.segment.FloatWrappingDimensionSelector;
|
|
||||||
import io.druid.segment.LongColumnSelector;
|
|
||||||
import io.druid.segment.LongWrappingDimensionSelector;
|
|
||||||
import io.druid.segment.Metadata;
|
import io.druid.segment.Metadata;
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
|
||||||
import io.druid.segment.SingleScanTimeDimSelector;
|
|
||||||
import io.druid.segment.StorageAdapter;
|
import io.druid.segment.StorageAdapter;
|
||||||
import io.druid.segment.VirtualColumns;
|
import io.druid.segment.VirtualColumns;
|
||||||
import io.druid.segment.ZeroDoubleColumnSelector;
|
|
||||||
import io.druid.segment.ZeroFloatColumnSelector;
|
|
||||||
import io.druid.segment.ZeroLongColumnSelector;
|
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ValueType;
|
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.ListIndexed;
|
import io.druid.segment.data.ListIndexed;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
@ -70,9 +53,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
{
|
{
|
||||||
private final IncrementalIndex<?> index;
|
private final IncrementalIndex<?> index;
|
||||||
|
|
||||||
public IncrementalIndexStorageAdapter(
|
public IncrementalIndexStorageAdapter(IncrementalIndex<?> index)
|
||||||
IncrementalIndex<?> index
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
}
|
}
|
||||||
|
@ -92,7 +73,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getAvailableDimensions()
|
public Indexed<String> getAvailableDimensions()
|
||||||
{
|
{
|
||||||
return new ListIndexed<String>(index.getDimensionNames(), String.class);
|
return new ListIndexed<>(index.getDimensionNames(), String.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -135,6 +116,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
return index.getMaxTime();
|
return index.getMaxTime();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public Comparable getMinValue(String column)
|
public Comparable getMinValue(String column)
|
||||||
{
|
{
|
||||||
|
@ -147,6 +129,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
return indexer.getMinValue();
|
return indexer.getMinValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public Comparable getMaxValue(String column)
|
public Comparable getMaxValue(String column)
|
||||||
{
|
{
|
||||||
|
@ -198,52 +181,65 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
return Sequences.empty();
|
return Sequences.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
Interval actualIntervalTmp = interval;
|
|
||||||
|
|
||||||
final Interval dataInterval = new Interval(getMinTime(), gran.bucketEnd(getMaxTime()));
|
final Interval dataInterval = new Interval(getMinTime(), gran.bucketEnd(getMaxTime()));
|
||||||
|
|
||||||
if (!actualIntervalTmp.overlaps(dataInterval)) {
|
if (!interval.overlaps(dataInterval)) {
|
||||||
return Sequences.empty();
|
return Sequences.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (actualIntervalTmp.getStart().isBefore(dataInterval.getStart())) {
|
final Interval actualInterval = interval.overlap(dataInterval);
|
||||||
actualIntervalTmp = actualIntervalTmp.withStart(dataInterval.getStart());
|
|
||||||
}
|
|
||||||
if (actualIntervalTmp.getEnd().isAfter(dataInterval.getEnd())) {
|
|
||||||
actualIntervalTmp = actualIntervalTmp.withEnd(dataInterval.getEnd());
|
|
||||||
}
|
|
||||||
|
|
||||||
final Interval actualInterval = actualIntervalTmp;
|
|
||||||
|
|
||||||
Iterable<Interval> iterable = gran.getIterable(actualInterval);
|
Iterable<Interval> iterable = gran.getIterable(actualInterval);
|
||||||
if (descending) {
|
if (descending) {
|
||||||
iterable = Lists.reverse(ImmutableList.copyOf(iterable));
|
iterable = Lists.reverse(ImmutableList.copyOf(iterable));
|
||||||
}
|
}
|
||||||
|
|
||||||
return Sequences.map(
|
return Sequences
|
||||||
Sequences.simple(iterable),
|
.simple(iterable)
|
||||||
new Function<Interval, Cursor>()
|
.map(i -> new IncrementalIndexCursor(virtualColumns, descending, filter, i, actualInterval, gran));
|
||||||
|
}
|
||||||
|
|
||||||
|
private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor)
|
||||||
{
|
{
|
||||||
EntryHolder currEntry = new EntryHolder();
|
return filter == null
|
||||||
|
? BooleanValueMatcher.of(true)
|
||||||
|
: filter.makeMatcher(cursor.getColumnSelectorFactory());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Cursor apply(@Nullable final Interval interval)
|
public Metadata getMetadata()
|
||||||
{
|
{
|
||||||
final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis());
|
return index.getMetadata();
|
||||||
|
}
|
||||||
|
|
||||||
return new Cursor()
|
private class IncrementalIndexCursor implements Cursor
|
||||||
{
|
{
|
||||||
private final ValueMatcher filterMatcher = makeFilterMatcher(filter, this);
|
private TimeAndDimsHolder currEntry;
|
||||||
|
private final ColumnSelectorFactory columnSelectorFactory;
|
||||||
|
private final ValueMatcher filterMatcher;
|
||||||
private final int maxRowIndex;
|
private final int maxRowIndex;
|
||||||
private Iterator<IncrementalIndex.TimeAndDims> baseIter;
|
private Iterator<IncrementalIndex.TimeAndDims> baseIter;
|
||||||
private Iterable<IncrementalIndex.TimeAndDims> cursorIterable;
|
private Iterable<IncrementalIndex.TimeAndDims> cursorIterable;
|
||||||
private boolean emptyRange;
|
private boolean emptyRange;
|
||||||
final DateTime time;
|
private final DateTime time;
|
||||||
int numAdvanced = -1;
|
private int numAdvanced;
|
||||||
boolean done;
|
private boolean done;
|
||||||
|
|
||||||
|
IncrementalIndexCursor(
|
||||||
|
VirtualColumns virtualColumns,
|
||||||
|
boolean descending,
|
||||||
|
Filter filter,
|
||||||
|
Interval interval,
|
||||||
|
Interval actualInterval,
|
||||||
|
Granularity gran
|
||||||
|
)
|
||||||
{
|
{
|
||||||
|
currEntry = new TimeAndDimsHolder();
|
||||||
|
columnSelectorFactory = new IncrementalIndexColumnSelectorFactory(index, virtualColumns, descending, currEntry);
|
||||||
|
filterMatcher = makeFilterMatcher(filter, this);
|
||||||
|
numAdvanced = -1;
|
||||||
maxRowIndex = index.getLastRowIndex();
|
maxRowIndex = index.getLastRowIndex();
|
||||||
|
final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis());
|
||||||
cursorIterable = index.getFacts().timeRangeIterable(
|
cursorIterable = index.getFacts().timeRangeIterable(
|
||||||
descending,
|
descending,
|
||||||
timeStart,
|
timeStart,
|
||||||
|
@ -255,6 +251,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
reset();
|
reset();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnSelectorFactory getColumnSelectorFactory()
|
||||||
|
{
|
||||||
|
return columnSelectorFactory;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DateTime getTime()
|
public DateTime getTime()
|
||||||
{
|
{
|
||||||
|
@ -376,335 +378,5 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
// so we still need to go through all rows to skip rows added after cursor created
|
// so we still need to go through all rows to skip rows added after cursor created
|
||||||
return rowIndex > maxRowIndex;
|
return rowIndex > maxRowIndex;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public DimensionSelector makeDimensionSelector(
|
|
||||||
DimensionSpec dimensionSpec
|
|
||||||
)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(dimensionSpec.getDimension())) {
|
|
||||||
return virtualColumns.makeDimensionSelector(dimensionSpec, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec));
|
|
||||||
}
|
|
||||||
|
|
||||||
private DimensionSelector makeDimensionSelectorUndecorated(
|
|
||||||
DimensionSpec dimensionSpec
|
|
||||||
)
|
|
||||||
{
|
|
||||||
final String dimension = dimensionSpec.getDimension();
|
|
||||||
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
|
|
||||||
|
|
||||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
|
||||||
DimensionSelector selector = new SingleScanTimeDimSelector(
|
|
||||||
makeLongColumnSelector(dimension),
|
|
||||||
extractionFn,
|
|
||||||
descending
|
|
||||||
);
|
|
||||||
return selector;
|
|
||||||
}
|
|
||||||
|
|
||||||
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension());
|
|
||||||
if (dimensionDesc == null) {
|
|
||||||
// not a dimension, column may be a metric
|
|
||||||
ColumnCapabilities capabilities = getColumnCapabilities(dimension);
|
|
||||||
if (capabilities == null) {
|
|
||||||
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
|
||||||
}
|
|
||||||
if (capabilities.getType() == ValueType.LONG) {
|
|
||||||
return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn);
|
|
||||||
}
|
|
||||||
if (capabilities.getType() == ValueType.FLOAT) {
|
|
||||||
return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn);
|
|
||||||
}
|
|
||||||
if (capabilities.getType() == ValueType.DOUBLE) {
|
|
||||||
return new DoubleWrappingDimensionSelector(makeDoubleColumnSelector(dimension), extractionFn);
|
|
||||||
}
|
|
||||||
|
|
||||||
// if we can't wrap the base column, just return a column of all nulls
|
|
||||||
return DimensionSelectorUtils.constantSelector(null, extractionFn);
|
|
||||||
} else {
|
|
||||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
|
||||||
return indexer.makeDimensionSelector(dimensionSpec, currEntry, dimensionDesc);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.makeFloatColumnSelector(columnName, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer dimIndex = index.getDimensionIndex(columnName);
|
|
||||||
if (dimIndex != null) {
|
|
||||||
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName);
|
|
||||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
|
||||||
return indexer.makeFloatColumnSelector(
|
|
||||||
currEntry,
|
|
||||||
dimensionDesc
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
|
||||||
if (metricIndexInt == null) {
|
|
||||||
return ZeroFloatColumnSelector.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
final int metricIndex = metricIndexInt;
|
|
||||||
return new FloatColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public float getFloat()
|
|
||||||
{
|
|
||||||
return index.getMetricFloatValue(currEntry.getValue(), metricIndex);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
inspector.visit("index", index);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LongColumnSelector makeLongColumnSelector(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.makeLongColumnSelector(columnName, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (columnName.equals(Column.TIME_COLUMN_NAME)) {
|
|
||||||
class TimeLongColumnSelector implements LongColumnSelector
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public long getLong()
|
|
||||||
{
|
|
||||||
return currEntry.getKey().getTimestamp();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
// nothing to inspect
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return new TimeLongColumnSelector();
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer dimIndex = index.getDimensionIndex(columnName);
|
|
||||||
if (dimIndex != null) {
|
|
||||||
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName);
|
|
||||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
|
||||||
return indexer.makeLongColumnSelector(
|
|
||||||
currEntry,
|
|
||||||
dimensionDesc
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
|
||||||
if (metricIndexInt == null) {
|
|
||||||
return ZeroLongColumnSelector.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
final int metricIndex = metricIndexInt;
|
|
||||||
|
|
||||||
return new LongColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public long getLong()
|
|
||||||
{
|
|
||||||
return index.getMetricLongValue(
|
|
||||||
currEntry.getValue(),
|
|
||||||
metricIndex
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
inspector.visit("index", index);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(column)) {
|
|
||||||
return virtualColumns.makeObjectColumnSelector(column, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (column.equals(Column.TIME_COLUMN_NAME)) {
|
|
||||||
return new ObjectColumnSelector<Long>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return Long.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Long get()
|
|
||||||
{
|
|
||||||
return currEntry.getKey().getTimestamp();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer metricIndexInt = index.getMetricIndex(column);
|
|
||||||
if (metricIndexInt != null) {
|
|
||||||
final int metricIndex = metricIndexInt;
|
|
||||||
final Class classOfObject = index.getMetricClass(column);
|
|
||||||
return new ObjectColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return classOfObject;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object get()
|
|
||||||
{
|
|
||||||
return index.getMetricObjectValue(
|
|
||||||
currEntry.getValue(),
|
|
||||||
metricIndex
|
|
||||||
);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(column);
|
|
||||||
|
|
||||||
if (dimensionDesc == null) {
|
|
||||||
return null;
|
|
||||||
} else {
|
|
||||||
|
|
||||||
final int dimensionIndex = dimensionDesc.getIndex();
|
|
||||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
|
||||||
|
|
||||||
return new ObjectColumnSelector<Object>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return Object.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object get()
|
|
||||||
{
|
|
||||||
IncrementalIndex.TimeAndDims key = currEntry.getKey();
|
|
||||||
if (key == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
Object[] dims = key.getDims();
|
|
||||||
if (dimensionIndex >= dims.length) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
return indexer.convertUnsortedEncodedKeyComponentToActualArrayOrList(
|
|
||||||
dims[dimensionIndex], DimensionIndexer.ARRAY
|
|
||||||
);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public DoubleColumnSelector makeDoubleColumnSelector(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.makeDoubleColumnSelector(columnName, this);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer dimIndex = index.getDimensionIndex(columnName);
|
|
||||||
if (dimIndex != null) {
|
|
||||||
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName);
|
|
||||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
|
||||||
return indexer.makeDoubleColumnSelector(
|
|
||||||
currEntry,
|
|
||||||
dimensionDesc
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
|
||||||
if (metricIndexInt == null) {
|
|
||||||
return ZeroDoubleColumnSelector.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
final int metricIndex = metricIndexInt;
|
|
||||||
return new DoubleColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public double getDouble()
|
|
||||||
{
|
|
||||||
return index.getMetricDoubleValue(currEntry.getValue(), metricIndex);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
|
||||||
{
|
|
||||||
inspector.visit("index", index);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
@Override
|
|
||||||
public ColumnCapabilities getColumnCapabilities(String columnName)
|
|
||||||
{
|
|
||||||
if (virtualColumns.exists(columnName)) {
|
|
||||||
return virtualColumns.getColumnCapabilities(columnName);
|
|
||||||
}
|
|
||||||
|
|
||||||
return index.getCapabilities(columnName);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor)
|
|
||||||
{
|
|
||||||
return filter == null
|
|
||||||
? BooleanValueMatcher.of(true)
|
|
||||||
: filter.makeMatcher(cursor);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class EntryHolder
|
|
||||||
{
|
|
||||||
IncrementalIndex.TimeAndDims currEntry = null;
|
|
||||||
|
|
||||||
public IncrementalIndex.TimeAndDims get()
|
|
||||||
{
|
|
||||||
return currEntry;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void set(IncrementalIndex.TimeAndDims currEntry)
|
|
||||||
{
|
|
||||||
this.currEntry = currEntry;
|
|
||||||
}
|
|
||||||
|
|
||||||
public IncrementalIndex.TimeAndDims getKey()
|
|
||||||
{
|
|
||||||
return currEntry;
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getValue()
|
|
||||||
{
|
|
||||||
return currEntry.getRowIndex();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Metadata getMetadata()
|
|
||||||
{
|
|
||||||
return index.getMetadata();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,56 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This interface is the core "pointer" interface that is used to create {@link io.druid.segment.ColumnValueSelector}s
|
||||||
|
* over incremental index. It's counterpart for historical segments is {@link io.druid.segment.data.Offset}.
|
||||||
|
*/
|
||||||
|
public class TimeAndDimsHolder
|
||||||
|
{
|
||||||
|
IncrementalIndex.TimeAndDims currEntry = null;
|
||||||
|
|
||||||
|
public IncrementalIndex.TimeAndDims get()
|
||||||
|
{
|
||||||
|
return currEntry;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void set(IncrementalIndex.TimeAndDims currEntry)
|
||||||
|
{
|
||||||
|
this.currEntry = currEntry;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method doesn't have well-defined semantics ("key" of what?), should be removed in favor of {@link #get()}.
|
||||||
|
*/
|
||||||
|
public IncrementalIndex.TimeAndDims getKey()
|
||||||
|
{
|
||||||
|
return currEntry;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method doesn't have well-defined semantics ("value" of what?), should be removed in favor of chaining
|
||||||
|
* get().getRowIndex().
|
||||||
|
*/
|
||||||
|
public int getValue()
|
||||||
|
{
|
||||||
|
return currEntry.getRowIndex();
|
||||||
|
}
|
||||||
|
}
|
|
@ -322,9 +322,9 @@ public abstract class BaseFilterTest
|
||||||
@Override
|
@Override
|
||||||
public List<String> apply(Cursor input)
|
public List<String> apply(Cursor input)
|
||||||
{
|
{
|
||||||
final DimensionSelector selector = input.makeDimensionSelector(
|
final DimensionSelector selector = input
|
||||||
new DefaultDimensionSpec(selectColumn, selectColumn)
|
.getColumnSelectorFactory()
|
||||||
);
|
.makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn));
|
||||||
|
|
||||||
final List<String> values = Lists.newArrayList();
|
final List<String> values = Lists.newArrayList();
|
||||||
|
|
||||||
|
@ -355,7 +355,7 @@ public abstract class BaseFilterTest
|
||||||
Aggregator agg = new FilteredAggregatorFactory(
|
Aggregator agg = new FilteredAggregatorFactory(
|
||||||
new CountAggregatorFactory("count"),
|
new CountAggregatorFactory("count"),
|
||||||
maybeOptimize(filter)
|
maybeOptimize(filter)
|
||||||
).factorize(input);
|
).factorize(input.getColumnSelectorFactory());
|
||||||
|
|
||||||
for (; !input.isDone(); input.advance()) {
|
for (; !input.isDone(); input.advance()) {
|
||||||
agg.aggregate();
|
agg.aggregate();
|
||||||
|
@ -417,9 +417,9 @@ public abstract class BaseFilterTest
|
||||||
@Override
|
@Override
|
||||||
public List<String> apply(Cursor input)
|
public List<String> apply(Cursor input)
|
||||||
{
|
{
|
||||||
final DimensionSelector selector = input.makeDimensionSelector(
|
final DimensionSelector selector = input
|
||||||
new DefaultDimensionSpec(selectColumn, selectColumn)
|
.getColumnSelectorFactory()
|
||||||
);
|
.makeDimensionSelector(new DefaultDimensionSpec(selectColumn, selectColumn));
|
||||||
|
|
||||||
final List<String> values = Lists.newArrayList();
|
final List<String> values = Lists.newArrayList();
|
||||||
|
|
||||||
|
|
|
@ -278,7 +278,9 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
Cursor cursor = Sequences.toList(Sequences.limit(cursorSequence, 1), Lists.<Cursor>newArrayList()).get(0);
|
Cursor cursor = Sequences.toList(Sequences.limit(cursorSequence, 1), Lists.<Cursor>newArrayList()).get(0);
|
||||||
DimensionSelector dimSelector;
|
DimensionSelector dimSelector;
|
||||||
|
|
||||||
dimSelector = cursor.makeDimensionSelector(new DefaultDimensionSpec("sally", "sally"));
|
dimSelector = cursor
|
||||||
|
.getColumnSelectorFactory()
|
||||||
|
.makeDimensionSelector(new DefaultDimensionSpec("sally", "sally"));
|
||||||
Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0)));
|
Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0)));
|
||||||
|
|
||||||
index.add(
|
index.add(
|
||||||
|
@ -292,7 +294,9 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
// Cursor reset should not be affected by out of order values
|
// Cursor reset should not be affected by out of order values
|
||||||
cursor.reset();
|
cursor.reset();
|
||||||
|
|
||||||
dimSelector = cursor.makeDimensionSelector(new DefaultDimensionSpec("sally", "sally"));
|
dimSelector = cursor
|
||||||
|
.getColumnSelectorFactory()
|
||||||
|
.makeDimensionSelector(new DefaultDimensionSpec("sally", "sally"));
|
||||||
Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0)));
|
Assert.assertEquals("bo", dimSelector.lookupName(dimSelector.getRow().get(0)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -430,12 +434,9 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
@Override
|
@Override
|
||||||
public Object apply(Cursor cursor)
|
public Object apply(Cursor cursor)
|
||||||
{
|
{
|
||||||
DimensionSelector dimSelector = cursor.makeDimensionSelector(
|
DimensionSelector dimSelector = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
"billy",
|
.makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
|
||||||
"billy"
|
|
||||||
)
|
|
||||||
);
|
|
||||||
int cardinality = dimSelector.getValueCardinality();
|
int cardinality = dimSelector.getValueCardinality();
|
||||||
|
|
||||||
//index gets more rows at this point, while other thread is iterating over the cursor
|
//index gets more rows at this point, while other thread is iterating over the cursor
|
||||||
|
@ -513,12 +514,9 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
@Override
|
@Override
|
||||||
public Object apply(Cursor cursor)
|
public Object apply(Cursor cursor)
|
||||||
{
|
{
|
||||||
DimensionSelector dimSelector1A = cursor.makeDimensionSelector(
|
DimensionSelector dimSelector1A = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
"billy",
|
.makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
|
||||||
"billy"
|
|
||||||
)
|
|
||||||
);
|
|
||||||
int cardinalityA = dimSelector1A.getValueCardinality();
|
int cardinalityA = dimSelector1A.getValueCardinality();
|
||||||
|
|
||||||
//index gets more rows at this point, while other thread is iterating over the cursor
|
//index gets more rows at this point, while other thread is iterating over the cursor
|
||||||
|
@ -535,12 +533,9 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
throw new RuntimeException(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
|
|
||||||
DimensionSelector dimSelector1B = cursor.makeDimensionSelector(
|
DimensionSelector dimSelector1B = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
"billy",
|
.makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
|
||||||
"billy"
|
|
||||||
)
|
|
||||||
);
|
|
||||||
//index gets more rows at this point, while other thread is iterating over the cursor
|
//index gets more rows at this point, while other thread is iterating over the cursor
|
||||||
try {
|
try {
|
||||||
index.add(
|
index.add(
|
||||||
|
@ -562,19 +557,13 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
throw new RuntimeException(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
|
|
||||||
DimensionSelector dimSelector1C = cursor.makeDimensionSelector(
|
DimensionSelector dimSelector1C = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
"billy",
|
.makeDimensionSelector(new DefaultDimensionSpec("billy", "billy"));
|
||||||
"billy"
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
DimensionSelector dimSelector2D = cursor.makeDimensionSelector(
|
DimensionSelector dimSelector2D = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
"billy2",
|
.makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2"));
|
||||||
"billy2"
|
|
||||||
)
|
|
||||||
);
|
|
||||||
//index gets more rows at this point, while other thread is iterating over the cursor
|
//index gets more rows at this point, while other thread is iterating over the cursor
|
||||||
try {
|
try {
|
||||||
index.add(
|
index.add(
|
||||||
|
@ -596,12 +585,9 @@ public class IncrementalIndexStorageAdapterTest
|
||||||
throw new RuntimeException(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
|
|
||||||
DimensionSelector dimSelector3E = cursor.makeDimensionSelector(
|
DimensionSelector dimSelector3E = cursor
|
||||||
new DefaultDimensionSpec(
|
.getColumnSelectorFactory()
|
||||||
"billy3",
|
.makeDimensionSelector(new DefaultDimensionSpec("billy3", "billy3"));
|
||||||
"billy3"
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
int rowNumInCursor = 0;
|
int rowNumInCursor = 0;
|
||||||
// and then, cursoring continues in the other thread
|
// and then, cursoring continues in the other thread
|
||||||
|
|
|
@ -85,13 +85,14 @@ public class IngestSegmentFirehose implements Firehose
|
||||||
@Override
|
@Override
|
||||||
public Sequence<InputRow> apply(final Cursor cursor)
|
public Sequence<InputRow> apply(final Cursor cursor)
|
||||||
{
|
{
|
||||||
final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
final LongColumnSelector timestampColumnSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeLongColumnSelector(Column.TIME_COLUMN_NAME);
|
||||||
|
|
||||||
final Map<String, DimensionSelector> dimSelectors = Maps.newHashMap();
|
final Map<String, DimensionSelector> dimSelectors = Maps.newHashMap();
|
||||||
for (String dim : dims) {
|
for (String dim : dims) {
|
||||||
final DimensionSelector dimSelector = cursor.makeDimensionSelector(
|
final DimensionSelector dimSelector = cursor
|
||||||
new DefaultDimensionSpec(dim, dim)
|
.getColumnSelectorFactory()
|
||||||
);
|
.makeDimensionSelector(new DefaultDimensionSpec(dim, dim));
|
||||||
// dimSelector is null if the dimension is not present
|
// dimSelector is null if the dimension is not present
|
||||||
if (dimSelector != null) {
|
if (dimSelector != null) {
|
||||||
dimSelectors.put(dim, dimSelector);
|
dimSelectors.put(dim, dimSelector);
|
||||||
|
@ -100,7 +101,8 @@ public class IngestSegmentFirehose implements Firehose
|
||||||
|
|
||||||
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
final Map<String, ObjectColumnSelector> metSelectors = Maps.newHashMap();
|
||||||
for (String metric : metrics) {
|
for (String metric : metrics) {
|
||||||
final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric);
|
final ObjectColumnSelector metricSelector =
|
||||||
|
cursor.getColumnSelectorFactory().makeObjectColumnSelector(metric);
|
||||||
if (metricSelector != null) {
|
if (metricSelector != null) {
|
||||||
metSelectors.put(metric, metricSelector);
|
metSelectors.put(metric, metricSelector);
|
||||||
}
|
}
|
||||||
|
|
|
@ -278,7 +278,9 @@ public class DumpSegment extends GuiceRunnable
|
||||||
final List<ObjectColumnSelector> selectors = Lists.newArrayList();
|
final List<ObjectColumnSelector> selectors = Lists.newArrayList();
|
||||||
|
|
||||||
for (String columnName : columnNames) {
|
for (String columnName : columnNames) {
|
||||||
selectors.add(makeSelector(columnName, index.getColumn(columnName), cursor));
|
selectors.add(
|
||||||
|
makeSelector(columnName, index.getColumn(columnName), cursor.getColumnSelectorFactory())
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
while (!cursor.isDone()) {
|
while (!cursor.isDone()) {
|
||||||
|
|
Loading…
Reference in New Issue