mirror of https://github.com/apache/druid.git
fix issue with group by limit pushdown for extractionFn, expressions, joins, etc (#9662)
* fix issue with group by limit pushdown for extractionFn, expressions, joins, etc * remove unused * fix test * revert unintended change * more tests * consider capabilities for StringGroupByColumnSelectorStrategy * fix test * fix and more test * revert because im scared
This commit is contained in:
parent
1b60148ec6
commit
0ff926b1a1
|
@ -52,6 +52,7 @@ import org.apache.druid.query.groupby.epinephelinae.column.NullableNumericGroupB
|
|||
import org.apache.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy;
|
||||
import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine;
|
||||
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||
import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
|
||||
import org.apache.druid.query.ordering.StringComparator;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
|
@ -74,6 +75,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Class that knows how to process a groupBy query on a single {@link StorageAdapter}. It returns a {@link Sequence}
|
||||
|
@ -335,6 +337,42 @@ public class GroupByQueryEngineV2
|
|||
});
|
||||
}
|
||||
|
||||
public static void convertRowTypesToOutputTypes(
|
||||
final List<DimensionSpec> dimensionSpecs,
|
||||
final ResultRow resultRow,
|
||||
final int resultRowDimensionStart
|
||||
)
|
||||
{
|
||||
for (int i = 0; i < dimensionSpecs.size(); i++) {
|
||||
DimensionSpec dimSpec = dimensionSpecs.get(i);
|
||||
final int resultRowIndex = resultRowDimensionStart + i;
|
||||
final ValueType outputType = dimSpec.getOutputType();
|
||||
|
||||
resultRow.set(
|
||||
resultRowIndex,
|
||||
DimensionHandlerUtils.convertObjectToType(resultRow.get(resultRowIndex), outputType)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* check if a column will operate correctly with {@link LimitedBufferHashGrouper} for query limit pushdown
|
||||
*/
|
||||
public static boolean canPushDownLimit(ColumnSelectorFactory columnSelectorFactory, String columnName)
|
||||
{
|
||||
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName);
|
||||
if (capabilities != null) {
|
||||
// strings can be pushed down if dictionaries are sorted and unique per id
|
||||
if (capabilities.getType() == ValueType.STRING) {
|
||||
return capabilities.areDictionaryValuesSorted().and(capabilities.areDictionaryValuesUnique()).isTrue();
|
||||
}
|
||||
// party on
|
||||
return true;
|
||||
}
|
||||
// we don't know what we don't know, don't assume otherwise
|
||||
return false;
|
||||
}
|
||||
|
||||
private static class GroupByStrategyFactory
|
||||
implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
|
||||
{
|
||||
|
@ -349,7 +387,7 @@ public class GroupByQueryEngineV2
|
|||
case STRING:
|
||||
DimensionSelector dimSelector = (DimensionSelector) selector;
|
||||
if (dimSelector.getValueCardinality() >= 0) {
|
||||
return new StringGroupByColumnSelectorStrategy(dimSelector::lookupName);
|
||||
return new StringGroupByColumnSelectorStrategy(dimSelector::lookupName, capabilities);
|
||||
} else {
|
||||
return new DictionaryBuildingStringGroupByColumnSelectorStrategy();
|
||||
}
|
||||
|
@ -555,16 +593,31 @@ public class GroupByQueryEngineV2
|
|||
@Override
|
||||
protected Grouper<ByteBuffer> newGrouper()
|
||||
{
|
||||
Grouper grouper = null;
|
||||
Grouper<ByteBuffer> grouper = null;
|
||||
final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory();
|
||||
final DefaultLimitSpec limitSpec = query.isApplyLimitPushDown() &&
|
||||
querySpecificConfig.isApplyLimitPushDownToSegment() ?
|
||||
(DefaultLimitSpec) query.getLimitSpec() : null;
|
||||
|
||||
final boolean canDoLimitPushdown;
|
||||
if (limitSpec != null) {
|
||||
LimitedBufferHashGrouper limitGrouper = new LimitedBufferHashGrouper<>(
|
||||
// there is perhaps a more graceful way this could be handled a bit more selectively, but for now just avoid
|
||||
// pushdown if it will prove problematic by checking grouping and ordering columns
|
||||
|
||||
canDoLimitPushdown = Stream.concat(
|
||||
query.getDimensions().stream().map(DimensionSpec::getDimension),
|
||||
limitSpec.getColumns().stream().map(OrderByColumnSpec::getDimension)
|
||||
).allMatch(col -> GroupByQueryEngineV2.canPushDownLimit(selectorFactory, col));
|
||||
} else {
|
||||
canDoLimitPushdown = false;
|
||||
}
|
||||
|
||||
if (canDoLimitPushdown) {
|
||||
LimitedBufferHashGrouper<ByteBuffer> limitGrouper = new LimitedBufferHashGrouper<>(
|
||||
Suppliers.ofInstance(buffer),
|
||||
keySerde,
|
||||
AggregatorAdapters.factorizeBuffered(
|
||||
cursor.getColumnSelectorFactory(),
|
||||
selectorFactory,
|
||||
query.getAggregatorSpecs()
|
||||
),
|
||||
querySpecificConfig.getBufferGrouperMaxSize(),
|
||||
|
@ -592,7 +645,7 @@ public class GroupByQueryEngineV2
|
|||
Suppliers.ofInstance(buffer),
|
||||
keySerde,
|
||||
AggregatorAdapters.factorizeBuffered(
|
||||
cursor.getColumnSelectorFactory(),
|
||||
selectorFactory,
|
||||
query.getAggregatorSpecs()
|
||||
),
|
||||
querySpecificConfig.getBufferGrouperMaxSize(),
|
||||
|
@ -834,24 +887,6 @@ public class GroupByQueryEngineV2
|
|||
}
|
||||
}
|
||||
|
||||
public static void convertRowTypesToOutputTypes(
|
||||
final List<DimensionSpec> dimensionSpecs,
|
||||
final ResultRow resultRow,
|
||||
final int resultRowDimensionStart
|
||||
)
|
||||
{
|
||||
for (int i = 0; i < dimensionSpecs.size(); i++) {
|
||||
DimensionSpec dimSpec = dimensionSpecs.get(i);
|
||||
final int resultRowIndex = resultRowDimensionStart + i;
|
||||
final ValueType outputType = dimSpec.getOutputType();
|
||||
|
||||
resultRow.set(
|
||||
resultRowIndex,
|
||||
DimensionHandlerUtils.convertObjectToType(resultRow.get(resultRowIndex), outputType)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private static class GroupByEngineKeySerde implements Grouper.KeySerde<ByteBuffer>
|
||||
{
|
||||
private final int keySize;
|
||||
|
|
|
@ -54,7 +54,7 @@ public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends Strin
|
|||
|
||||
public DictionaryBuildingStringGroupByColumnSelectorStrategy()
|
||||
{
|
||||
super(null);
|
||||
super(null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.query.ordering.StringComparator;
|
|||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -35,12 +36,16 @@ import java.util.function.IntFunction;
|
|||
|
||||
public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
|
||||
{
|
||||
@Nullable
|
||||
private final ColumnCapabilities capabilities;
|
||||
|
||||
@Nullable
|
||||
private final IntFunction<String> dictionaryLookup;
|
||||
|
||||
public StringGroupByColumnSelectorStrategy(IntFunction<String> dictionaryLookup)
|
||||
public StringGroupByColumnSelectorStrategy(IntFunction<String> dictionaryLookup, ColumnCapabilities capabilities)
|
||||
{
|
||||
this.dictionaryLookup = dictionaryLookup;
|
||||
this.capabilities = capabilities;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -148,7 +153,12 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
|
|||
@Override
|
||||
public Grouper.BufferComparator bufferComparator(int keyBufferPosition, @Nullable StringComparator stringComparator)
|
||||
{
|
||||
if (stringComparator == null || StringComparators.LEXICOGRAPHIC.equals(stringComparator)) {
|
||||
final boolean canCompareInts =
|
||||
capabilities != null &&
|
||||
capabilities.hasBitmapIndexes() &&
|
||||
capabilities.areDictionaryValuesSorted().and(capabilities.areDictionaryValuesUnique()).isTrue();
|
||||
|
||||
if (canCompareInts && (stringComparator == null || StringComparators.LEXICOGRAPHIC.equals(stringComparator))) {
|
||||
return (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Integer.compare(
|
||||
lhsBuffer.getInt(lhsPosition + keyBufferPosition),
|
||||
rhsBuffer.getInt(rhsPosition + keyBufferPosition)
|
||||
|
|
|
@ -25,11 +25,11 @@ import org.apache.druid.java.util.common.Pair;
|
|||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
import org.apache.druid.segment.Capabilities;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
|
@ -268,8 +268,8 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
|||
@Override
|
||||
public void skipTo(String previousStop)
|
||||
{
|
||||
Capabilities capabilities = storageAdapter.getCapabilities();
|
||||
if (capabilities.dimensionValuesSorted()) {
|
||||
ColumnCapabilities capabilities = storageAdapter.getColumnCapabilities(query.getDimensionSpec().getDimension());
|
||||
if (capabilities != null && capabilities.areDictionaryValuesSorted().isTrue()) {
|
||||
this.previousStop = previousStop;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,71 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
||||
public class Capabilities
|
||||
{
|
||||
private final boolean dimensionValuesSorted;
|
||||
|
||||
public static CapabilitiesBuilder builder()
|
||||
{
|
||||
return new CapabilitiesBuilder();
|
||||
}
|
||||
|
||||
private Capabilities(
|
||||
boolean dimensionValuesSorted
|
||||
)
|
||||
{
|
||||
this.dimensionValuesSorted = dimensionValuesSorted;
|
||||
}
|
||||
|
||||
/**
|
||||
* Is dimension value dictionary sorted?
|
||||
* @return
|
||||
*/
|
||||
public boolean dimensionValuesSorted()
|
||||
{
|
||||
return dimensionValuesSorted;
|
||||
}
|
||||
|
||||
public static class CapabilitiesBuilder
|
||||
{
|
||||
private boolean dimensionValuesSorted = false;
|
||||
|
||||
private CapabilitiesBuilder()
|
||||
{
|
||||
}
|
||||
|
||||
public CapabilitiesBuilder dimensionValuesSorted(boolean value)
|
||||
{
|
||||
dimensionValuesSorted = value;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Capabilities build()
|
||||
{
|
||||
return new Capabilities(
|
||||
dimensionValuesSorted
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.ISE;
|
|||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
@ -91,6 +92,8 @@ public class ColumnProcessors
|
|||
|
||||
return new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.STRING)
|
||||
.setDictionaryValuesSorted(dimensionSpec.getExtractionFn().preservesOrdering())
|
||||
.setDictionaryValuesUnique(dimensionSpec.getExtractionFn().getExtractionType() == ExtractionFn.ExtractionType.ONE_TO_ONE)
|
||||
.setHasMultipleValues(dimensionSpec.mustDecorate() || mayBeMultiValue(dimensionCapabilities));
|
||||
} else {
|
||||
// No transformation. Pass through.
|
||||
|
@ -129,7 +132,10 @@ public class ColumnProcessors
|
|||
return makeProcessor(expr.getBindingIfIdentifier(), processorFactory, selectorFactory);
|
||||
} else {
|
||||
return makeProcessorInternal(
|
||||
factory -> new ColumnCapabilitiesImpl().setType(exprTypeHint).setHasMultipleValues(true),
|
||||
factory -> new ColumnCapabilitiesImpl().setType(exprTypeHint)
|
||||
.setHasMultipleValues(true)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setDictionaryValuesSorted(false),
|
||||
factory -> ExpressionSelectors.makeDimensionSelector(factory, expr, null),
|
||||
factory -> ExpressionSelectors.makeColumnValueSelector(factory, expr),
|
||||
processorFactory,
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.query.dimension.ColumnSelectorStrategy;
|
|||
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
@ -54,15 +55,17 @@ public final class DimensionHandlerUtils
|
|||
public static final Float ZERO_FLOAT = 0.0f;
|
||||
public static final Long ZERO_LONG = 0L;
|
||||
|
||||
public static final ColumnCapabilities DEFAULT_STRING_CAPABILITIES =
|
||||
new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setHasBitmapIndexes(false);
|
||||
|
||||
private DimensionHandlerUtils()
|
||||
{
|
||||
}
|
||||
|
||||
public static final ColumnCapabilities DEFAULT_STRING_CAPABILITIES =
|
||||
new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setDictionaryEncoded(true)
|
||||
.setHasBitmapIndexes(true);
|
||||
|
||||
public static DimensionHandler<?, ?, ?> getHandlerFromCapabilities(
|
||||
String dimensionName,
|
||||
@Nullable ColumnCapabilities capabilities,
|
||||
|
@ -219,7 +222,16 @@ public final class DimensionHandlerUtils
|
|||
// Currently, all extractionFns output Strings, so the column will return String values via a
|
||||
// DimensionSelector if an extractionFn is present.
|
||||
if (dimSpec.getExtractionFn() != null) {
|
||||
capabilities = DEFAULT_STRING_CAPABILITIES;
|
||||
ExtractionFn fn = dimSpec.getExtractionFn();
|
||||
capabilities = ColumnCapabilitiesImpl.copyOf(capabilities)
|
||||
.setType(ValueType.STRING)
|
||||
.setDictionaryValuesUnique(
|
||||
capabilities.isDictionaryEncoded() &&
|
||||
fn.getExtractionType() == ExtractionFn.ExtractionType.ONE_TO_ONE
|
||||
)
|
||||
.setDictionaryValuesSorted(
|
||||
capabilities.isDictionaryEncoded() && fn.preservesOrdering()
|
||||
);
|
||||
}
|
||||
|
||||
// DimensionSpec's decorate only operates on DimensionSelectors, so if a spec mustDecorate(),
|
||||
|
|
|
@ -166,12 +166,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Capabilities getCapabilities()
|
||||
{
|
||||
return Capabilities.builder().dimensionValuesSorted(true).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
|
|
|
@ -107,7 +107,8 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
if (valueType != null) {
|
||||
return new ColumnCapabilitiesImpl()
|
||||
.setType(valueType)
|
||||
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
// Numeric types should be reported as complete, but not STRING or COMPLEX (because we don't have full info)
|
||||
.setIsComplete(valueType.isNumeric());
|
||||
} else {
|
||||
|
|
|
@ -112,12 +112,6 @@ public class RowBasedStorageAdapter<RowType> implements StorageAdapter
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Capabilities getCapabilities()
|
||||
{
|
||||
return Capabilities.builder().dimensionValuesSorted(false).build();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
|
|
|
@ -47,7 +47,6 @@ public interface StorageAdapter extends CursorFactory
|
|||
Comparable getMinValue(String column);
|
||||
@Nullable
|
||||
Comparable getMaxValue(String column);
|
||||
Capabilities getCapabilities();
|
||||
|
||||
/**
|
||||
* Returns capabilities of a particular column, if known. May be null if the column doesn't exist, or if
|
||||
|
|
|
@ -114,6 +114,8 @@ public class ColumnBuilder
|
|||
.setType(type)
|
||||
.setDictionaryEncoded(dictionaryEncoded)
|
||||
.setHasBitmapIndexes(bitmapIndex != null)
|
||||
.setDictionaryValuesSorted(dictionaryEncoded)
|
||||
.setDictionaryValuesUnique(dictionaryEncoded)
|
||||
.setHasSpatialIndexes(spatialIndex != null)
|
||||
.setHasMultipleValues(hasMultipleValues)
|
||||
.setIsComplete(true)
|
||||
|
|
|
@ -26,6 +26,8 @@ public interface ColumnCapabilities
|
|||
ValueType getType();
|
||||
|
||||
boolean isDictionaryEncoded();
|
||||
Capable areDictionaryValuesSorted();
|
||||
Capable areDictionaryValuesUnique();
|
||||
boolean isRunLengthEncoded();
|
||||
boolean hasBitmapIndexes();
|
||||
boolean hasSpatialIndexes();
|
||||
|
@ -38,6 +40,34 @@ public interface ColumnCapabilities
|
|||
* fail to set {@link #hasMultipleValues()} even when the associated column really could have multiple values.
|
||||
* Until this situation is sorted out, if this method returns false, callers are encouraged to ignore
|
||||
* {@link #hasMultipleValues()} and treat that property as if it were unknown.
|
||||
*
|
||||
* todo: replace all booleans with {@link Capable} and this method can be dropped
|
||||
*/
|
||||
boolean isComplete();
|
||||
|
||||
|
||||
enum Capable
|
||||
{
|
||||
FALSE,
|
||||
TRUE,
|
||||
UNKNOWN;
|
||||
|
||||
public boolean isTrue()
|
||||
{
|
||||
return this == TRUE;
|
||||
}
|
||||
|
||||
public Capable and(Capable other)
|
||||
{
|
||||
if (this == UNKNOWN || other == UNKNOWN) {
|
||||
return UNKNOWN;
|
||||
}
|
||||
return this == TRUE && other == TRUE ? TRUE : FALSE;
|
||||
}
|
||||
|
||||
public static Capable of(boolean bool)
|
||||
{
|
||||
return bool ? TRUE : FALSE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,15 @@ import javax.annotation.Nullable;
|
|||
*/
|
||||
public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
||||
{
|
||||
public static ColumnCapabilitiesImpl copyOf(final ColumnCapabilities other)
|
||||
{
|
||||
final ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.merge(other);
|
||||
capabilities.setFilterable(other.isFilterable());
|
||||
capabilities.setIsComplete(other.isComplete());
|
||||
return capabilities;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private ValueType type = null;
|
||||
|
||||
|
@ -40,23 +49,16 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
private boolean hasSpatialIndexes = false;
|
||||
private boolean hasMultipleValues = false;
|
||||
|
||||
// This is a query time concept and not persisted in the segment files.
|
||||
// These capabilities are computed at query time and not persisted in the segment files.
|
||||
@JsonIgnore
|
||||
private Capable dictionaryValuesSorted = Capable.UNKNOWN;
|
||||
@JsonIgnore
|
||||
private Capable dictionaryValuesUnique = Capable.UNKNOWN;
|
||||
@JsonIgnore
|
||||
private boolean filterable;
|
||||
|
||||
|
||||
@JsonIgnore
|
||||
private boolean complete = false;
|
||||
|
||||
public static ColumnCapabilitiesImpl copyOf(final ColumnCapabilities other)
|
||||
{
|
||||
final ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.merge(other);
|
||||
capabilities.setFilterable(other.isFilterable());
|
||||
capabilities.setIsComplete(other.isComplete());
|
||||
return capabilities;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public ValueType getType()
|
||||
|
@ -83,6 +85,30 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Capable areDictionaryValuesSorted()
|
||||
{
|
||||
return dictionaryValuesSorted;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setDictionaryValuesSorted(boolean dictionaryValuesSorted)
|
||||
{
|
||||
this.dictionaryValuesSorted = Capable.of(dictionaryValuesSorted);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Capable areDictionaryValuesUnique()
|
||||
{
|
||||
return dictionaryValuesUnique;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setDictionaryValuesUnique(boolean dictionaryValuesUnique)
|
||||
{
|
||||
this.dictionaryValuesUnique = Capable.of(dictionaryValuesUnique);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public boolean isRunLengthEncoded()
|
||||
|
@ -123,6 +149,12 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
return hasMultipleValues;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues)
|
||||
{
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFilterable()
|
||||
{
|
||||
|
@ -133,22 +165,16 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
filterable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComplete()
|
||||
{
|
||||
return complete;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setFilterable(boolean filterable)
|
||||
{
|
||||
this.filterable = filterable;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues)
|
||||
@Override
|
||||
public boolean isComplete()
|
||||
{
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
return this;
|
||||
return complete;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setIsComplete(boolean complete)
|
||||
|
@ -178,5 +204,7 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
this.hasMultipleValues |= other.hasMultipleValues();
|
||||
this.complete &= other.isComplete(); // these should always be the same?
|
||||
this.filterable &= other.isFilterable();
|
||||
this.dictionaryValuesSorted = this.dictionaryValuesSorted.and(other.areDictionaryValuesSorted());
|
||||
this.dictionaryValuesUnique = this.dictionaryValuesUnique.and(other.areDictionaryValuesUnique());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -665,6 +665,8 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
capabilities.setType(ValueType.STRING);
|
||||
capabilities.setDictionaryEncoded(true);
|
||||
capabilities.setHasBitmapIndexes(true);
|
||||
capabilities.setDictionaryValuesSorted(false);
|
||||
capabilities.setDictionaryValuesUnique(true);
|
||||
capabilities.setIsComplete(true);
|
||||
columnCapabilities.put(dimension, capabilities);
|
||||
}
|
||||
|
@ -924,6 +926,10 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.setDictionaryEncoded(type == ValueType.STRING);
|
||||
capabilities.setHasBitmapIndexes(type == ValueType.STRING);
|
||||
if (type == ValueType.STRING) {
|
||||
capabilities.setDictionaryValuesUnique(true);
|
||||
capabilities.setDictionaryValuesSorted(false);
|
||||
}
|
||||
capabilities.setType(type);
|
||||
capabilities.setIsComplete(true);
|
||||
return capabilities;
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.druid.query.BaseQuery;
|
|||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.segment.Capabilities;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.DimensionDictionarySelector;
|
||||
|
@ -140,13 +139,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
return indexer.getMaxValue();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Capabilities getCapabilities()
|
||||
{
|
||||
return Capabilities.builder().dimensionValuesSorted(false).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
{
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.guava.Sequence;
|
|||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.segment.Capabilities;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.Metadata;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
|
@ -146,16 +145,6 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Capabilities getCapabilities()
|
||||
{
|
||||
// Dictionaries in the joinables may not be sorted. Unfortunately this API does not let us be granular about what
|
||||
// is and isn't sorted, so return false globally. At the time of this writing, the only query affected by this
|
||||
// is a topN with lexicographic sort and 'previousStop' set (it will not be able to skip values based on
|
||||
// dictionary code).
|
||||
return Capabilities.builder().dimensionValuesSorted(false).build();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
|
|
|
@ -56,6 +56,9 @@ public class IndexedTableColumnSelectorFactory implements ColumnSelectorFactory
|
|||
capabilities.setDictionaryEncoded(true);
|
||||
}
|
||||
|
||||
capabilities.setDictionaryValuesSorted(false);
|
||||
capabilities.setDictionaryValuesUnique(false);
|
||||
|
||||
return capabilities.setIsComplete(true);
|
||||
} else {
|
||||
return null;
|
||||
|
|
|
@ -0,0 +1,141 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.groupby.epinephelinae;
|
||||
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class GroupByQueryEngineV2Test
|
||||
{
|
||||
private static final String DIM = "d0";
|
||||
ColumnSelectorFactory factory;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
factory = EasyMock.createMock(ColumnSelectorFactory.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCanPushDownLimitForSegmentStringSelector()
|
||||
{
|
||||
ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setHasBitmapIndexes(true)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(true)
|
||||
.setDictionaryValuesSorted(true)
|
||||
.setDictionaryValuesUnique(true)
|
||||
.setIsComplete(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
EasyMock.verify(factory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCanPushDownLimitForIncrementalStringSelector()
|
||||
{
|
||||
ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setHasBitmapIndexes(false)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(true)
|
||||
.setIsComplete(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
EasyMock.verify(factory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCanPushDownLimitForExpressionStringSelector()
|
||||
{
|
||||
ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setHasBitmapIndexes(false)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
EasyMock.verify(factory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCanPushDownLimitForJoinStringSelector()
|
||||
{
|
||||
ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setHasBitmapIndexes(false)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(true)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
EasyMock.verify(factory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCanPushDownLimitForNumericSelector()
|
||||
{
|
||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ValueType.LONG)
|
||||
.setHasBitmapIndexes(false)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).anyTimes();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
capabilities.setType(ValueType.DOUBLE);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
capabilities.setType(ValueType.FLOAT);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
EasyMock.verify(factory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCanPushDownLimitForComplexSelector()
|
||||
{
|
||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ValueType.COMPLEX)
|
||||
.setHasBitmapIndexes(false)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
EasyMock.verify(factory);
|
||||
}
|
||||
}
|
|
@ -35,7 +35,6 @@ import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory;
|
|||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.Capabilities;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
|
@ -296,12 +295,6 @@ public class TopNMetricSpecOptimizationsTest
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Capabilities getCapabilities()
|
||||
{
|
||||
return Capabilities.builder().dimensionValuesSorted(true).build();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
|
|
|
@ -0,0 +1,213 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.DoubleDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.LongDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.MapInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FloatSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class QueryableIndexColumnCapabilitiesTest extends InitializedNullHandlingTest
|
||||
{
|
||||
@ClassRule
|
||||
public static TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
private static IncrementalIndex INC_INDEX;
|
||||
private static QueryableIndex MMAP_INDEX;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws IOException
|
||||
{
|
||||
List<InputRow> rows = new ArrayList<>();
|
||||
MapInputRowParser parser = new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec("time", "auto", null),
|
||||
new DimensionsSpec(
|
||||
ImmutableList.<DimensionSchema>builder()
|
||||
.addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1", "d2")))
|
||||
.add(new DoubleDimensionSchema("d3"))
|
||||
.add(new FloatDimensionSchema("d4"))
|
||||
.add(new LongDimensionSchema("d5"))
|
||||
.build(),
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
);
|
||||
Map<String, Object> event =
|
||||
ImmutableMap.<String, Object>builder().put("time", DateTimes.nowUtc().getMillis())
|
||||
.put("d1", "some string")
|
||||
.put("d2", ImmutableList.of("some", "list"))
|
||||
.put("d3", 1.234)
|
||||
.put("d4", 1.234f)
|
||||
.put("d5", 10L)
|
||||
.build();
|
||||
rows.add(Iterables.getOnlyElement(parser.parseBatch(event)));
|
||||
IndexBuilder builder = IndexBuilder.create()
|
||||
.rows(rows)
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "d3"),
|
||||
new FloatSumAggregatorFactory("m2", "d4"),
|
||||
new LongSumAggregatorFactory("m3", "d5"),
|
||||
new HyperUniquesAggregatorFactory("m4", "d1")
|
||||
)
|
||||
.withDimensionsSpec(parser)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.tmpDir(temporaryFolder.newFolder());
|
||||
INC_INDEX = builder.buildIncrementalIndex();
|
||||
MMAP_INDEX = builder.buildMMappedIndex();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardown()
|
||||
{
|
||||
INC_INDEX.close();
|
||||
MMAP_INDEX.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumericColumns()
|
||||
{
|
||||
// incremental index
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities(ColumnHolder.TIME_COLUMN_NAME), ValueType.LONG);
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d3"), ValueType.DOUBLE);
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d4"), ValueType.FLOAT);
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d5"), ValueType.LONG);
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m1"), ValueType.DOUBLE);
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m2"), ValueType.FLOAT);
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m3"), ValueType.LONG);
|
||||
|
||||
// segment index
|
||||
assertNonStringColumnCapabilities(
|
||||
MMAP_INDEX.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getCapabilities(),
|
||||
ValueType.LONG
|
||||
);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d3").getCapabilities(), ValueType.DOUBLE);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d4").getCapabilities(), ValueType.FLOAT);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d5").getCapabilities(), ValueType.LONG);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m1").getCapabilities(), ValueType.DOUBLE);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m2").getCapabilities(), ValueType.FLOAT);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m3").getCapabilities(), ValueType.LONG);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringColumn()
|
||||
{
|
||||
ColumnCapabilities caps = INC_INDEX.getCapabilities("d1");
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
Assert.assertTrue(caps.hasBitmapIndexes());
|
||||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
|
||||
caps = MMAP_INDEX.getColumnHolder("d1").getCapabilities();
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
Assert.assertTrue(caps.hasBitmapIndexes());
|
||||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertTrue(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiStringColumn()
|
||||
{
|
||||
ColumnCapabilities caps = INC_INDEX.getCapabilities("d2");
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
Assert.assertTrue(caps.hasBitmapIndexes());
|
||||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
|
||||
caps = MMAP_INDEX.getColumnHolder("d2").getCapabilities();
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
Assert.assertTrue(caps.hasBitmapIndexes());
|
||||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertTrue(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComplexColumn()
|
||||
{
|
||||
assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m4"), ValueType.COMPLEX);
|
||||
assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m4").getCapabilities(), ValueType.COMPLEX);
|
||||
}
|
||||
|
||||
|
||||
private void assertNonStringColumnCapabilities(ColumnCapabilities caps, ValueType valueType)
|
||||
{
|
||||
Assert.assertEquals(valueType, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,144 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class RowBasedColumnSelectorFactoryTest
|
||||
{
|
||||
private static final String STRING_COLUMN_NAME = "string";
|
||||
private static final String LONG_COLUMN_NAME = "long";
|
||||
private static final String FLOAT_COLUMN_NAME = "float";
|
||||
private static final String DOUBLE_COLUMN_NAME = "double";
|
||||
private static final String COMPLEX_COLUMN_NAME = "complex";
|
||||
|
||||
private static final RowSignature ROW_SIGNATURE = RowSignature.builder()
|
||||
.add(ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG)
|
||||
.add(STRING_COLUMN_NAME, ValueType.STRING)
|
||||
.add(LONG_COLUMN_NAME, ValueType.LONG)
|
||||
.add(FLOAT_COLUMN_NAME, ValueType.FLOAT)
|
||||
.add(DOUBLE_COLUMN_NAME, ValueType.DOUBLE)
|
||||
.add(COMPLEX_COLUMN_NAME, ValueType.COMPLEX)
|
||||
.build();
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesTime()
|
||||
{
|
||||
// time column takes a special path
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, ColumnHolder.TIME_COLUMN_NAME);
|
||||
Assert.assertEquals(ValueType.LONG, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesString()
|
||||
{
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, STRING_COLUMN_NAME);
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesLong()
|
||||
{
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, LONG_COLUMN_NAME);
|
||||
Assert.assertEquals(ValueType.LONG, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesFloat()
|
||||
{
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, FLOAT_COLUMN_NAME);
|
||||
Assert.assertEquals(ValueType.FLOAT, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesDouble()
|
||||
{
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, DOUBLE_COLUMN_NAME);
|
||||
Assert.assertEquals(ValueType.DOUBLE, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesComplex()
|
||||
{
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, COMPLEX_COLUMN_NAME);
|
||||
Assert.assertEquals(ValueType.COMPLEX, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilitiesUnknownColumn()
|
||||
{
|
||||
ColumnCapabilities caps =
|
||||
RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, "wat");
|
||||
Assert.assertNull(caps);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class ColumnCapabilitiesTest
|
||||
{
|
||||
@Test
|
||||
public void testCapableAnd()
|
||||
{
|
||||
Assert.assertTrue(ColumnCapabilities.Capable.TRUE.and(ColumnCapabilities.Capable.TRUE).isTrue());
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.TRUE.and(ColumnCapabilities.Capable.FALSE).isTrue());
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.TRUE.and(ColumnCapabilities.Capable.UNKNOWN).isTrue());
|
||||
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.FALSE.and(ColumnCapabilities.Capable.TRUE).isTrue());
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.FALSE.and(ColumnCapabilities.Capable.FALSE).isTrue());
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.FALSE.and(ColumnCapabilities.Capable.UNKNOWN).isTrue());
|
||||
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.UNKNOWN.and(ColumnCapabilities.Capable.TRUE).isTrue());
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.UNKNOWN.and(ColumnCapabilities.Capable.FALSE).isTrue());
|
||||
Assert.assertFalse(ColumnCapabilities.Capable.UNKNOWN.and(ColumnCapabilities.Capable.UNKNOWN).isTrue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapableOfBoolean()
|
||||
{
|
||||
Assert.assertEquals(ColumnCapabilities.Capable.TRUE, ColumnCapabilities.Capable.of(true));
|
||||
Assert.assertEquals(ColumnCapabilities.Capable.FALSE, ColumnCapabilities.Capable.of(false));
|
||||
}
|
||||
}
|
|
@ -195,12 +195,6 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getCapabilities_factToCountry()
|
||||
{
|
||||
Assert.assertFalse(makeFactToCountrySegment().getCapabilities().dimensionValuesSorted());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getColumnCapabilities_factToCountryFactColumn()
|
||||
{
|
||||
|
@ -209,6 +203,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
Assert.assertEquals(ValueType.STRING, capabilities.getType());
|
||||
Assert.assertTrue(capabilities.hasBitmapIndexes());
|
||||
Assert.assertTrue(capabilities.isDictionaryEncoded());
|
||||
Assert.assertTrue(capabilities.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(capabilities.areDictionaryValuesUnique().isTrue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -220,6 +216,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
|
||||
Assert.assertEquals(ValueType.STRING, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.hasBitmapIndexes());
|
||||
Assert.assertFalse(capabilities.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(capabilities.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(capabilities.isDictionaryEncoded());
|
||||
}
|
||||
|
||||
|
|
|
@ -804,4 +804,28 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
Assert.assertTrue(selector.getObject().isNumericNull());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilities()
|
||||
{
|
||||
ColumnCapabilities caps = X_PLUS_Y.capabilities("expr");
|
||||
Assert.assertEquals(ValueType.FLOAT, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
|
||||
caps = Z_CONCAT_X.capabilities("expr");
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
Assert.assertFalse(caps.hasBitmapIndexes());
|
||||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -7394,6 +7394,47 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByLimitPushdownExtraction() throws Exception
|
||||
{
|
||||
cannotVectorize();
|
||||
|
||||
testQuery(
|
||||
"SELECT dim4, substring(dim5, 1, 1), count(*) FROM druid.numfoo WHERE dim4 = 'a' GROUP BY 1,2 LIMIT 2",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "_d0"),
|
||||
new ExtractionDimensionSpec("dim5", "_d1", new SubstringDimExtractionFn(0, 1))
|
||||
)
|
||||
)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING))
|
||||
.setDimFilter(selector("dim4", "a", null))
|
||||
.setAggregatorSpecs(
|
||||
aggregators(
|
||||
new CountAggregatorFactory("a0")
|
||||
)
|
||||
)
|
||||
.setLimitSpec(
|
||||
new DefaultLimitSpec(
|
||||
ImmutableList.of(),
|
||||
2
|
||||
)
|
||||
)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"a", "a", 2L},
|
||||
new Object[]{"a", "b", 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterOnTimeFloor() throws Exception
|
||||
{
|
||||
|
|
|
@ -241,7 +241,7 @@ public class CalciteTests
|
|||
new TimestampSpec(TIMESTAMP_COLUMN, "iso", null),
|
||||
new DimensionsSpec(
|
||||
ImmutableList.<DimensionSchema>builder()
|
||||
.addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")))
|
||||
.addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3", "dim4", "dim5")))
|
||||
.add(new DoubleDimensionSchema("d1"))
|
||||
.add(new DoubleDimensionSchema("d2"))
|
||||
.add(new FloatDimensionSchema("f1"))
|
||||
|
@ -382,6 +382,8 @@ public class CalciteTests
|
|||
.put("dim1", "")
|
||||
.put("dim2", ImmutableList.of("a"))
|
||||
.put("dim3", ImmutableList.of("a", "b"))
|
||||
.put("dim4", "a")
|
||||
.put("dim5", "aa")
|
||||
.build(),
|
||||
PARSER_NUMERIC_DIMS
|
||||
),
|
||||
|
@ -399,6 +401,8 @@ public class CalciteTests
|
|||
.put("dim1", "10.1")
|
||||
.put("dim2", ImmutableList.of())
|
||||
.put("dim3", ImmutableList.of("b", "c"))
|
||||
.put("dim4", "a")
|
||||
.put("dim5", "ab")
|
||||
.build(),
|
||||
PARSER_NUMERIC_DIMS
|
||||
),
|
||||
|
@ -416,6 +420,8 @@ public class CalciteTests
|
|||
.put("dim1", "2")
|
||||
.put("dim2", ImmutableList.of(""))
|
||||
.put("dim3", ImmutableList.of("d"))
|
||||
.put("dim4", "a")
|
||||
.put("dim5", "ba")
|
||||
.build(),
|
||||
PARSER_NUMERIC_DIMS
|
||||
),
|
||||
|
@ -427,6 +433,8 @@ public class CalciteTests
|
|||
.put("dim1", "1")
|
||||
.put("dim2", ImmutableList.of("a"))
|
||||
.put("dim3", ImmutableList.of(""))
|
||||
.put("dim4", "b")
|
||||
.put("dim5", "ad")
|
||||
.build(),
|
||||
PARSER_NUMERIC_DIMS
|
||||
),
|
||||
|
@ -438,6 +446,8 @@ public class CalciteTests
|
|||
.put("dim1", "def")
|
||||
.put("dim2", ImmutableList.of("abc"))
|
||||
.put("dim3", ImmutableList.of())
|
||||
.put("dim4", "b")
|
||||
.put("dim5", "aa")
|
||||
.build(),
|
||||
PARSER_NUMERIC_DIMS
|
||||
),
|
||||
|
@ -447,6 +457,8 @@ public class CalciteTests
|
|||
.put("m1", "6.0")
|
||||
.put("m2", "6.0")
|
||||
.put("dim1", "abc")
|
||||
.put("dim4", "b")
|
||||
.put("dim5", "ab")
|
||||
.build(),
|
||||
PARSER_NUMERIC_DIMS
|
||||
)
|
||||
|
|
Loading…
Reference in New Issue