mirror of https://github.com/apache/druid.git
add equality, null, and range filter (#14542)
changes: * new filters that preserve match value typing to better handle filtering different column types * sql planner uses new filters by default in sql compatible null handling mode * remove isFilterable from column capabilities * proper handling of array filtering, add array processor to column processors * javadoc for sql test filter functions * range filter support for arrays, tons more tests, fixes * add dimension selector tests for mixed type roots * support json equality * rename semantic index maker thingys to mostly have plural names since they typically make many indexes, e.g. StringValueSetIndex -> StringValueSetIndexes * add cooler equality index maker, ValueIndexes * fix missing string utf8 index supplier * expression array comparator stuff
This commit is contained in:
parent
0a8262edc8
commit
913416c669
|
@ -26,12 +26,12 @@ import org.apache.druid.collections.bitmap.MutableBitmap;
|
|||
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.IndexedUtf8ValueIndexes;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -72,7 +72,7 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
|
|||
public static class BenchmarkState
|
||||
{
|
||||
@Nullable
|
||||
private IndexedUtf8ValueSetIndex<?> stringValueSetIndex;
|
||||
private IndexedUtf8ValueIndexes<?> stringValueSetIndex;
|
||||
private final TreeSet<ByteBuffer> values = new TreeSet<>();
|
||||
private static final int START_INT = 10_000_000;
|
||||
|
||||
|
@ -112,7 +112,7 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
|
|||
);
|
||||
StringUtf8ColumnIndexSupplier<?> indexSupplier =
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null);
|
||||
stringValueSetIndex = (IndexedUtf8ValueSetIndex<?>) indexSupplier.as(StringValueSetIndex.class);
|
||||
stringValueSetIndex = (IndexedUtf8ValueIndexes<?>) indexSupplier.as(StringValueSetIndexes.class);
|
||||
List<Integer> filterValues = new ArrayList<>();
|
||||
List<Integer> nonFilterValues = new ArrayList<>();
|
||||
for (int i = 0; i < dictionarySize; i++) {
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchModule;
|
|||
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchToQuantilePostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
|
@ -444,7 +443,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
|
||||
|
@ -476,7 +475,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
|
@ -484,11 +483,11 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new TDigestSketchAggregatorFactory("a1:agg", "qsketch_m1", 100),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.datasketches.hll.HllSketch;
|
||||
import org.apache.datasketches.hll.TgtHllType;
|
||||
import org.apache.druid.error.InvalidInput;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringEncoding;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
|
@ -221,6 +222,8 @@ public class HllSketchBuildAggregatorFactory extends HllSketchAggregatorFactory
|
|||
}
|
||||
};
|
||||
break;
|
||||
case ARRAY:
|
||||
throw InvalidInput.exception("ARRAY types are not supported for hll sketch");
|
||||
default:
|
||||
updater = sketch -> {
|
||||
Object obj = selector.getObject();
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.query.aggregation.datasketches.hll.vector;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.StringEncoding;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper;
|
||||
import org.apache.druid.segment.VectorColumnProcessorFactory;
|
||||
|
@ -83,6 +84,15 @@ public class HllSketchBuildVectorProcessorFactory implements VectorColumnProcess
|
|||
return new LongHllSketchBuildVectorProcessor(helper, selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HllSketchBuildVectorProcessor makeArrayProcessor(
|
||||
ColumnCapabilities capabilities,
|
||||
VectorObjectSelector selector
|
||||
)
|
||||
{
|
||||
throw DruidException.defensive("ARRAY types are not supported for hll sketch");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HllSketchBuildVectorProcessor makeObjectProcessor(
|
||||
ColumnCapabilities capabilities,
|
||||
|
|
|
@ -179,7 +179,7 @@ public class KllDoublesSketchAggregatorFactory extends KllSketchAggregatorFactor
|
|||
SingleValueDimensionVectorSelector selector
|
||||
)
|
||||
{
|
||||
return new KllSketchNoOpBufferAggregator<KllDoublesSketch>(getEmptySketch());
|
||||
return new KllSketchNoOpBufferAggregator<>(getEmptySketch());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -188,7 +188,7 @@ public class KllDoublesSketchAggregatorFactory extends KllSketchAggregatorFactor
|
|||
MultiValueDimensionVectorSelector selector
|
||||
)
|
||||
{
|
||||
return new KllSketchNoOpBufferAggregator<KllDoublesSketch>(getEmptySketch());
|
||||
return new KllSketchNoOpBufferAggregator<>(getEmptySketch());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -209,6 +209,12 @@ public class KllDoublesSketchAggregatorFactory extends KllSketchAggregatorFactor
|
|||
return new KllDoublesSketchBuildVectorAggregator(selector, getK(), getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorAggregator makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
return new KllSketchNoOpBufferAggregator<>(getEmptySketch());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorAggregator makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
|
|
|
@ -179,7 +179,7 @@ public class KllFloatsSketchAggregatorFactory extends KllSketchAggregatorFactory
|
|||
SingleValueDimensionVectorSelector selector
|
||||
)
|
||||
{
|
||||
return new KllSketchNoOpBufferAggregator<KllFloatsSketch>(getEmptySketch());
|
||||
return new KllSketchNoOpBufferAggregator<>(getEmptySketch());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -188,7 +188,7 @@ public class KllFloatsSketchAggregatorFactory extends KllSketchAggregatorFactory
|
|||
MultiValueDimensionVectorSelector selector
|
||||
)
|
||||
{
|
||||
return new KllSketchNoOpBufferAggregator<KllFloatsSketch>(getEmptySketch());
|
||||
return new KllSketchNoOpBufferAggregator<>(getEmptySketch());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -209,6 +209,12 @@ public class KllFloatsSketchAggregatorFactory extends KllSketchAggregatorFactory
|
|||
return new KllFloatsSketchBuildVectorAggregator(selector, getK(), getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorAggregator makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
return new KllSketchNoOpBufferAggregator<>(getEmptySketch());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorAggregator makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
|
|
|
@ -208,6 +208,12 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
|
|||
return new DoublesSketchBuildVectorAggregator(selector, k, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorAggregator makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
return new NoopDoublesSketchBufferAggregator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorAggregator makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.datasketches.common.Util;
|
|||
import org.apache.datasketches.theta.SetOperation;
|
||||
import org.apache.datasketches.theta.Union;
|
||||
import org.apache.datasketches.thetacommon.ThetaUtil;
|
||||
import org.apache.druid.error.InvalidInput;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.aggregation.AggregateCombiner;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
|
@ -39,6 +40,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
|||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -78,6 +80,10 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (capabilities != null && capabilities.isArray()) {
|
||||
throw InvalidInput.exception("ARRAY types are not supported for theta sketch");
|
||||
}
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new SketchAggregator(selector, size);
|
||||
}
|
||||
|
@ -85,6 +91,10 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (capabilities != null && capabilities.isArray()) {
|
||||
throw InvalidInput.exception("ARRAY types are not supported for theta sketch");
|
||||
}
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
final SketchAggregator aggregator = new SketchAggregator(selector, size);
|
||||
return new AggregatorAndSize(aggregator, aggregator.getInitialSizeBytes());
|
||||
|
@ -94,6 +104,10 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (capabilities != null && capabilities.isArray()) {
|
||||
throw InvalidInput.exception("ARRAY types are not supported for theta sketch");
|
||||
}
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new SketchBufferAggregator(selector, size, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
|
|
@ -133,6 +133,12 @@ public class ToObjectVectorColumnProcessorFactory implements VectorColumnProcess
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Object[]> makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
return selector::getObjectVector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Object[]> makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
|
|
|
@ -187,7 +187,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
private static final List<AggregatorFactory> EXPECTED_FILTERED_AGGREGATORS =
|
||||
EXPECTED_PA_AGGREGATORS.stream()
|
||||
.limit(5)
|
||||
.map(factory -> new FilteredAggregatorFactory(factory, selector("dim2", "a", null)))
|
||||
.map(factory -> new FilteredAggregatorFactory(factory, equality("dim2", "a", ColumnType.STRING)))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
/**
|
||||
|
@ -344,7 +344,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new HllSketchBuildAggregatorFactory("a1", "dim2", null, null, null, null, ROUND),
|
||||
new FilteredAggregatorFactory(
|
||||
new HllSketchBuildAggregatorFactory("a2", "dim2", null, null, null, null, ROUND),
|
||||
BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
|
||||
not(equality("dim2", "", ColumnType.STRING))
|
||||
),
|
||||
new HllSketchBuildAggregatorFactory("a3", "v0", null, null, null, null, ROUND),
|
||||
new HllSketchBuildAggregatorFactory("a4", "v1", null, null, null, null, ROUND),
|
||||
|
@ -436,7 +436,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new LongSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("_a0:count"),
|
||||
BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null))
|
||||
notNull("a0")
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -480,7 +480,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new HllSketchBuildAggregatorFactory("a0", "m1", null, null, null, true, true)
|
||||
)
|
||||
)
|
||||
.setHavingSpec(having(selector("a0", "2", null)))
|
||||
.setHavingSpec(having(equality("a0", 2L, ColumnType.LONG)))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
|
@ -852,7 +852,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(
|
||||
aggregators(
|
||||
|
@ -895,7 +895,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING))
|
||||
|
@ -911,7 +911,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
true
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
|
@ -923,7 +923,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
false,
|
||||
true
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -954,7 +954,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING))
|
||||
|
@ -962,11 +962,11 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new HllSketchBuildAggregatorFactory("a0", "v0", null, null, null, null, true),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new HllSketchBuildAggregatorFactory("a1", "v0", null, null, null, null, true),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -48,10 +48,7 @@ import org.apache.druid.query.aggregation.post.ExpressionPostAggregator;
|
|||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
|
@ -161,11 +158,11 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new DoublesSketchAggregatorFactory("a4:agg", "v0", null),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a5:agg", "m1", null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a6:agg", "m1", null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
),
|
||||
new DoublesSketchAggregatorFactory("a8:agg", "cnt", null)
|
||||
))
|
||||
|
@ -223,11 +220,11 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new DoublesSketchAggregatorFactory("a2:agg", "qsketch_m1", 256),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a4:agg", "qsketch_m1", null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a5:agg", "qsketch_m1", null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
|
@ -325,11 +322,11 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new DoublesSketchAggregatorFactory("a4:agg", "v1", 128),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a5:agg", "v0", 128),
|
||||
new SelectorDimFilter("dim2", "abc", null)
|
||||
equality("dim2", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a6:agg", "v0", 128),
|
||||
new NotDimFilter(new SelectorDimFilter("dim2", "abc", null))
|
||||
not(equality("dim2", "abc", ColumnType.STRING))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
|
@ -728,7 +725,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(ImmutableList.of(
|
||||
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
|
||||
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
|
||||
|
@ -775,7 +772,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(ImmutableList.of(
|
||||
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
|
||||
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
|
||||
|
@ -815,7 +812,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
|
@ -823,19 +820,19 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, false),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, false),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -886,7 +883,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
|
@ -894,19 +891,19 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, true),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, true),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -71,6 +71,7 @@ public class SketchAggregatorFactoryTest
|
|||
ColumnSelectorFactory colSelectorFactory = EasyMock.mock(ColumnSelectorFactory.class);
|
||||
EasyMock.expect(colSelectorFactory.makeColumnValueSelector(EasyMock.anyString()))
|
||||
.andReturn(EasyMock.createMock(ColumnValueSelector.class)).anyTimes();
|
||||
EasyMock.expect(colSelectorFactory.getColumnCapabilities("x")).andReturn(null).anyTimes();
|
||||
EasyMock.replay(colSelectorFactory);
|
||||
|
||||
AggregatorAndSize aggregatorAndSize = AGGREGATOR_16384.factorizeWithSize(colSelectorFactory);
|
||||
|
|
|
@ -241,7 +241,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
|
||||
not(equality("dim2", "", ColumnType.STRING))
|
||||
),
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a3",
|
||||
|
@ -341,7 +341,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new LongSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("_a0:count"),
|
||||
BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null))
|
||||
notNull("a0")
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -831,7 +831,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(
|
||||
ImmutableList.of(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -890,7 +890,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
|
@ -905,7 +905,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -916,7 +916,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -927,7 +927,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -938,7 +938,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -971,7 +971,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
|
@ -986,7 +986,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -997,7 +997,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -1008,7 +1008,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
|
@ -1019,7 +1019,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
null,
|
||||
null
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.druid.query.expression.TestExprMacroTable;
|
|||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.BloomKFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
|
@ -489,7 +488,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(BaseCalciteQueryTest.bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(
|
||||
ImmutableList.of(
|
||||
new BloomFilterAggregatorFactory(
|
||||
|
@ -536,7 +535,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING))
|
||||
|
@ -548,7 +547,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new DefaultDimensionSpec("dim1", "a0:dim1"),
|
||||
TEST_NUM_ENTRIES
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new BloomFilterAggregatorFactory(
|
||||
|
@ -556,7 +555,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new DefaultDimensionSpec("l1", "a1:l1", ColumnType.LONG),
|
||||
TEST_NUM_ENTRIES
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -51,7 +51,7 @@ public class BloomFilterExpressionsTest extends InitializedNullHandlingTest
|
|||
ExprMacroTable macroTable = new ExprMacroTable(ImmutableList.of(createMacro, addMacro, testMacro));
|
||||
|
||||
Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers(
|
||||
new ImmutableMap.Builder<String, InputBindings.InputSupplier>()
|
||||
new ImmutableMap.Builder<String, InputBindings.InputSupplier<?>>()
|
||||
.put("bloomy", InputBindings.inputSupplier(BloomFilterExpressions.BLOOM_FILTER_TYPE, () -> new BloomKFilter(100)))
|
||||
.put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> SOME_STRING))
|
||||
.put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> SOME_LONG))
|
||||
|
|
|
@ -199,21 +199,24 @@ public class BloomDimFilterTest extends BaseFilterTest
|
|||
public void testMultiValueStringColumn() throws IOException
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
assertFilterMatches(
|
||||
assertFilterMatchesSkipArrays(
|
||||
new BloomDimFilter("dim2", bloomKFilter(1000, (String) null), null),
|
||||
ImmutableList.of("1", "2", "5")
|
||||
);
|
||||
} else {
|
||||
assertFilterMatches(
|
||||
assertFilterMatchesSkipArrays(
|
||||
new BloomDimFilter("dim2", bloomKFilter(1000, (String) null), null),
|
||||
ImmutableList.of("1", "5")
|
||||
);
|
||||
assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, ""), null), ImmutableList.of("2"));
|
||||
assertFilterMatchesSkipArrays(
|
||||
new BloomDimFilter("dim2", bloomKFilter(1000, ""), null),
|
||||
ImmutableList.of("2")
|
||||
);
|
||||
}
|
||||
assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "a"), null), ImmutableList.of("0", "3"));
|
||||
assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "b"), null), ImmutableList.of("0"));
|
||||
assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "c"), null), ImmutableList.of("4"));
|
||||
assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "d"), null), ImmutableList.of());
|
||||
assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "a"), null), ImmutableList.of("0", "3"));
|
||||
assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "b"), null), ImmutableList.of("0"));
|
||||
assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "c"), null), ImmutableList.of("4"));
|
||||
assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "d"), null), ImmutableList.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -289,8 +292,8 @@ public class BloomDimFilterTest extends BaseFilterTest
|
|||
ImmutableList.of("0", "1", "2", "5")
|
||||
);
|
||||
|
||||
assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "HELLO"), lookupFn), ImmutableList.of("0", "3"));
|
||||
assertFilterMatches(
|
||||
assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "HELLO"), lookupFn), ImmutableList.of("0", "3"));
|
||||
assertFilterMatchesSkipArrays(
|
||||
new BloomDimFilter("dim2", bloomKFilter(1000, "UNKNOWN"), lookupFn),
|
||||
ImmutableList.of("0", "1", "2", "4", "5")
|
||||
);
|
||||
|
@ -486,4 +489,17 @@ public class BloomDimFilterTest extends BaseFilterTest
|
|||
}
|
||||
return BloomKFilterHolder.fromBloomKFilter(filter);
|
||||
}
|
||||
|
||||
private static BloomKFilterHolder bloomKFilter(int expectedEntries, byte[]... values) throws IOException
|
||||
{
|
||||
BloomKFilter filter = new BloomKFilter(expectedEntries);
|
||||
for (byte[] value : values) {
|
||||
if (value == null) {
|
||||
filter.addBytes(null, 0, 0);
|
||||
} else {
|
||||
filter.addBytes(value);
|
||||
}
|
||||
}
|
||||
return BloomKFilterHolder.fromBloomKFilter(filter);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -84,6 +84,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testBloomFilterExprFilter() throws IOException
|
||||
{
|
||||
cannotVectorize();
|
||||
BloomKFilter filter = new BloomKFilter(1500);
|
||||
filter.addString("a-foo");
|
||||
filter.addString("-foo");
|
||||
|
|
|
@ -40,10 +40,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
|||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
|
@ -189,7 +186,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
|
@ -201,7 +198,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
),
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a8:agg",
|
||||
|
@ -293,7 +290,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
|
@ -305,7 +302,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
|
@ -408,7 +405,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
|
@ -420,7 +417,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
|
@ -521,7 +518,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(ImmutableList.of(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a0:agg",
|
||||
|
@ -570,7 +567,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
|
@ -586,7 +583,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
|
@ -598,7 +595,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQ
|
|||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE,
|
||||
false
|
||||
),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -39,10 +39,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
|||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
|
@ -151,11 +148,11 @@ public class QuantileSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
new ApproximateHistogramAggregatorFactory("a4:agg", "v0", null, null, null, null, false),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null, false),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null, false),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
),
|
||||
new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null, false)
|
||||
))
|
||||
|
@ -208,15 +205,47 @@ public class QuantileSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null, false),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null, false),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
new ApproximateHistogramFoldingAggregatorFactory(
|
||||
"a0:agg",
|
||||
"hist_m1",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
new ApproximateHistogramFoldingAggregatorFactory(
|
||||
"a2:agg",
|
||||
"hist_m1",
|
||||
200,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null, false),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
new ApproximateHistogramFoldingAggregatorFactory(
|
||||
"a4:agg",
|
||||
"hist_m1",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
equality("dim1", "abc", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory(
|
||||
"a5:agg",
|
||||
"hist_m1",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
|
@ -379,12 +408,21 @@ public class QuantileSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
|
||||
new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false)
|
||||
|
||||
))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(
|
||||
ImmutableList.of(
|
||||
new ApproximateHistogramFoldingAggregatorFactory(
|
||||
"a0:agg",
|
||||
"hist_m1",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false)
|
||||
)
|
||||
)
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a1:agg", 0.01f)
|
||||
|
@ -411,19 +449,35 @@ public class QuantileSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))
|
||||
.setAggregatorSpecs(
|
||||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false),
|
||||
selector("dim1", "nonexistent", null)
|
||||
new ApproximateHistogramFoldingAggregatorFactory(
|
||||
"a0:agg",
|
||||
"hist_m1",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false),
|
||||
selector("dim1", "nonexistent", null)
|
||||
new ApproximateHistogramAggregatorFactory(
|
||||
"a1:agg",
|
||||
"m1",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
),
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -60,8 +60,6 @@ import org.apache.druid.query.aggregation.post.ExpressionPostAggregator;
|
|||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||
|
@ -421,7 +419,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Intervals.ETERNITY))
|
||||
.columns("cnt", "dim1")
|
||||
.filters(selector("dim2", "nonexistent", null))
|
||||
.filters(equality("dim2", "nonexistent", ColumnType.STRING))
|
||||
.context(defaultScanQueryContext(context, resultSignature))
|
||||
.build()
|
||||
)
|
||||
|
@ -455,7 +453,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Intervals.ETERNITY))
|
||||
.columns("cnt", "dim1")
|
||||
.filters(selector("dim2", "nonexistent", null))
|
||||
.filters(equality("dim2", "nonexistent", ColumnType.STRING))
|
||||
.context(defaultScanQueryContext(context, resultSignature))
|
||||
.orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING)))
|
||||
.build()
|
||||
|
@ -771,7 +769,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
)
|
||||
)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(not(selector("j0.v", "xa", null)))
|
||||
.setDimFilter(not(equality("j0.v", "xa", ColumnType.STRING)))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0")))
|
||||
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
|
||||
|
@ -958,7 +956,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
new DoubleSumAggregatorFactory("a0:sum", "m2"),
|
||||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("a0:count"),
|
||||
not(selector("m2", null, null)),
|
||||
notNull("m2"),
|
||||
|
||||
// Not sure why the name is only set in SQL-compatible null mode. Seems strange.
|
||||
// May be due to JSON serialization: name is set on the serialized aggregator even
|
||||
|
@ -1540,14 +1538,13 @@ public class MSQSelectTest extends MSQTestBase
|
|||
)
|
||||
.setHavingSpec(
|
||||
having(
|
||||
bound(
|
||||
range(
|
||||
"a0",
|
||||
"1",
|
||||
ColumnType.LONG,
|
||||
1L,
|
||||
null,
|
||||
true,
|
||||
false,
|
||||
null,
|
||||
StringComparators.NUMERIC
|
||||
false
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -1859,7 +1856,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("a0"),
|
||||
new NotDimFilter(new SelectorDimFilter("dim3", null, null)),
|
||||
notNull("dim3"),
|
||||
"a0"
|
||||
)
|
||||
)
|
||||
|
@ -2073,7 +2070,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.setAggregatorSpecs(
|
||||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("a0"),
|
||||
new SelectorDimFilter("j0.d1", null, null),
|
||||
isNull("j0.d1"),
|
||||
"a0"
|
||||
)
|
||||
)
|
||||
|
|
|
@ -518,7 +518,7 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC))
|
||||
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
|
||||
.aggregators(
|
||||
new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
|
||||
new VarianceAggregatorFactory("a1:agg", "d1", "sample", "double"),
|
||||
|
@ -568,7 +568,7 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE3)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimFilter(selector("dim2", "a", null))
|
||||
.setDimFilter(equality("dim2", "a", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING))
|
||||
|
@ -576,35 +576,35 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
aggregators(
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a1:agg", "d1", "sample", "double"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a2:agg", "d1", "sample", "double"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a3:agg", "d1", "sample", "double"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a4:agg", "l1", "population", "long"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a5:agg", "l1", "sample", "long"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a6:agg", "l1", "sample", "long"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new VarianceAggregatorFactory("a7:agg", "l1", "sample", "long"),
|
||||
selector("dim1", "nonexistent", null)
|
||||
equality("dim1", "nonexistent", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -25,8 +25,6 @@ import org.apache.druid.guice.DruidInjectorBuilder;
|
|||
import org.apache.druid.guice.SleepModule;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.filter.BoundDimFilter;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.scan.ScanQuery.ResultFormat;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
|
@ -61,7 +59,7 @@ public class SleepSqlTest extends BaseCalciteQueryTest
|
|||
)
|
||||
)
|
||||
.columns("v0")
|
||||
.filters(new BoundDimFilter("m1", null, "2.0", null, true, null, null, StringComparators.NUMERIC))
|
||||
.filters(range("m1", ColumnType.DOUBLE, null, 2.0, false, true))
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.druid.segment.DimensionSelector;
|
|||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
|
@ -60,6 +61,7 @@ import org.apache.druid.utils.CloseableUtils;
|
|||
import org.apache.druid.utils.CollectionUtils;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -276,6 +278,15 @@ public class DruidSegmentReader extends IntermediateRowParsingReader<Map<String,
|
|||
return () -> selector.isNull() ? null : selector.getLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Object> makeArrayProcessor(
|
||||
BaseObjectColumnValueSelector<?> selector,
|
||||
@Nullable ColumnCapabilities columnCapabilities
|
||||
)
|
||||
{
|
||||
return selector::getObject;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Object> makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFact
|
|||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.segment.AutoTypeColumnSchema;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
|
@ -683,6 +684,118 @@ public class DruidSegmentReaderTest extends InitializedNullHandlingTest
|
|||
Assert.assertTrue("Sequence is not closed", isSequenceClosed.booleanValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayColumns() throws IOException
|
||||
{
|
||||
// make our own stuff here so that we don't pollute the shared spec, rows, and segment defined in setup and
|
||||
// break all the other tests
|
||||
DimensionsSpec dimensionsSpec = new DimensionsSpec(
|
||||
ImmutableList.of(
|
||||
StringDimensionSchema.create("strCol"),
|
||||
new DoubleDimensionSchema("dblCol"),
|
||||
new AutoTypeColumnSchema("arrayCol")
|
||||
)
|
||||
);
|
||||
List<AggregatorFactory> metrics = ImmutableList.of(
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new HyperUniquesAggregatorFactory("met_s", "strCol")
|
||||
);
|
||||
final List<InputRow> rows = ImmutableList.of(
|
||||
new MapBasedInputRow(
|
||||
DateTimes.of("2000"),
|
||||
ImmutableList.of("strCol", "dblCol", "arrayCol"),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("strCol", "foo")
|
||||
.put("dblCol", 1.23)
|
||||
.put("arrayCol", ImmutableList.of("a", "b", "c"))
|
||||
.build()
|
||||
),
|
||||
new MapBasedInputRow(
|
||||
DateTimes.of("2000T01"),
|
||||
ImmutableList.of("strCol", "dblCol", "arrayCol"),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("strCol", "bar")
|
||||
.put("dblCol", 4.56)
|
||||
.put("arrayCol", ImmutableList.of("x", "y", "z"))
|
||||
.build()
|
||||
)
|
||||
);
|
||||
|
||||
InputStats inputStats = new InputStatsImpl();
|
||||
final IncrementalIndex incrementalIndex =
|
||||
IndexBuilder.create()
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withDimensionsSpec(dimensionsSpec)
|
||||
.withMetrics(metrics.toArray(new AggregatorFactory[0]))
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(rows)
|
||||
.buildIncrementalIndex();
|
||||
|
||||
File segmentDirectory = temporaryFolder.newFolder();
|
||||
long segmentSize;
|
||||
try {
|
||||
TestHelper.getTestIndexMergerV9(
|
||||
OnHeapMemorySegmentWriteOutMediumFactory.instance()
|
||||
).persist(
|
||||
incrementalIndex,
|
||||
segmentDirectory,
|
||||
IndexSpec.DEFAULT,
|
||||
null
|
||||
);
|
||||
segmentSize = FileUtils.getFileSize(segmentDirectory);
|
||||
}
|
||||
finally {
|
||||
incrementalIndex.close();
|
||||
}
|
||||
InputEntity entity = new BytesCountingInputEntity(
|
||||
makeInputEntity(
|
||||
Intervals.of("2000/P1D"),
|
||||
segmentDirectory,
|
||||
ImmutableList.of("strCol", "dblCol", "arrayCol"),
|
||||
ImmutableList.of("cnt", "met_s")
|
||||
),
|
||||
inputStats
|
||||
);
|
||||
final DruidSegmentReader reader = new DruidSegmentReader(
|
||||
entity,
|
||||
indexIO,
|
||||
new TimestampSpec("__time", "millis", DateTimes.of("1971")),
|
||||
new DimensionsSpec(
|
||||
ImmutableList.of(
|
||||
StringDimensionSchema.create("strCol"),
|
||||
new DoubleDimensionSchema("dblCol"),
|
||||
new AutoTypeColumnSchema("arrayCol")
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all(),
|
||||
null,
|
||||
temporaryFolder.newFolder()
|
||||
);
|
||||
|
||||
List<InputRow> readRows = readRows(reader);
|
||||
|
||||
Assert.assertEquals(ImmutableList.of("strCol", "dblCol", "arrayCol"), readRows.get(0).getDimensions());
|
||||
Assert.assertEquals(DateTimes.of("2000T").getMillis(), readRows.get(0).getTimestampFromEpoch());
|
||||
Assert.assertEquals("foo", readRows.get(0).getRaw("strCol"));
|
||||
Assert.assertEquals(1.23, readRows.get(0).getRaw("dblCol"));
|
||||
Assert.assertArrayEquals(new Object[]{"a", "b", "c"}, (Object[]) readRows.get(0).getRaw("arrayCol"));
|
||||
Assert.assertEquals(1L, readRows.get(0).getRaw("cnt"));
|
||||
Assert.assertEquals(makeHLLC("foo"), readRows.get(0).getRaw("met_s"));
|
||||
|
||||
Assert.assertEquals(DateTimes.of("2000T1").getMillis(), readRows.get(1).getTimestampFromEpoch());
|
||||
Assert.assertEquals("bar", readRows.get(1).getRaw("strCol"));
|
||||
Assert.assertEquals(4.56, readRows.get(1).getRaw("dblCol"));
|
||||
Assert.assertArrayEquals(new Object[]{"x", "y", "z"}, (Object[]) readRows.get(1).getRaw("arrayCol"));
|
||||
Assert.assertEquals(1L, readRows.get(1).getRaw("cnt"));
|
||||
Assert.assertEquals(makeHLLC("bar"), readRows.get(1).getRaw("met_s"));
|
||||
|
||||
Assert.assertEquals(segmentSize, inputStats.getProcessedBytes());
|
||||
|
||||
}
|
||||
|
||||
private InputEntity makeInputEntity(final Interval interval)
|
||||
{
|
||||
return new BytesCountingInputEntity(
|
||||
|
|
|
@ -50,6 +50,14 @@ class BinLtExpr extends BinaryBooleanOpExprBase
|
|||
return Comparators.<String>naturalNullsFirst().compare(left, right) < 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean evalArray(ExprEval left, ExprEval right)
|
||||
{
|
||||
ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type());
|
||||
// type cannot be null here because ExprEval type is not nullable
|
||||
return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) < 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean evalLong(long left, long right)
|
||||
{
|
||||
|
@ -63,12 +71,6 @@ class BinLtExpr extends BinaryBooleanOpExprBase
|
|||
return Double.compare(left, right) < 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
|
||||
{
|
||||
|
@ -96,6 +98,14 @@ class BinLeqExpr extends BinaryBooleanOpExprBase
|
|||
return Comparators.<String>naturalNullsFirst().compare(left, right) <= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean evalArray(ExprEval left, ExprEval right)
|
||||
{
|
||||
ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type());
|
||||
// type cannot be null here because ExprEval type is not nullable
|
||||
return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) <= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean evalLong(long left, long right)
|
||||
{
|
||||
|
@ -109,12 +119,6 @@ class BinLeqExpr extends BinaryBooleanOpExprBase
|
|||
return Double.compare(left, right) <= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
|
||||
{
|
||||
|
@ -142,6 +146,14 @@ class BinGtExpr extends BinaryBooleanOpExprBase
|
|||
return Comparators.<String>naturalNullsFirst().compare(left, right) > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean evalArray(ExprEval left, ExprEval right)
|
||||
{
|
||||
ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type());
|
||||
// type cannot be null here because ExprEval type is not nullable
|
||||
return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean evalLong(long left, long right)
|
||||
{
|
||||
|
@ -155,12 +167,6 @@ class BinGtExpr extends BinaryBooleanOpExprBase
|
|||
return Double.compare(left, right) > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
|
||||
{
|
||||
|
@ -188,6 +194,14 @@ class BinGeqExpr extends BinaryBooleanOpExprBase
|
|||
return Comparators.<String>naturalNullsFirst().compare(left, right) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean evalArray(ExprEval left, ExprEval right)
|
||||
{
|
||||
ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type());
|
||||
// type cannot be null here because ExprEval type is not nullable
|
||||
return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean evalLong(long left, long right)
|
||||
{
|
||||
|
@ -201,12 +215,6 @@ class BinGeqExpr extends BinaryBooleanOpExprBase
|
|||
return Double.compare(left, right) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
|
||||
{
|
||||
|
@ -234,6 +242,14 @@ class BinEqExpr extends BinaryBooleanOpExprBase
|
|||
return Objects.equals(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean evalArray(ExprEval left, ExprEval right)
|
||||
{
|
||||
ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type());
|
||||
// type cannot be null here because ExprEval type is not nullable
|
||||
return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean evalLong(long left, long right)
|
||||
{
|
||||
|
@ -246,12 +262,6 @@ class BinEqExpr extends BinaryBooleanOpExprBase
|
|||
return left == right;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
|
||||
{
|
||||
|
@ -279,6 +289,14 @@ class BinNeqExpr extends BinaryBooleanOpExprBase
|
|||
return !Objects.equals(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean evalArray(ExprEval left, ExprEval right)
|
||||
{
|
||||
ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type());
|
||||
// type cannot be null here because ExprEval type is not nullable
|
||||
return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) != 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean evalLong(long left, long right)
|
||||
{
|
||||
|
@ -291,12 +309,6 @@ class BinNeqExpr extends BinaryBooleanOpExprBase
|
|||
return left != right;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
|
||||
{
|
||||
|
|
|
@ -195,6 +195,9 @@ abstract class BinaryBooleanOpExprBase extends BinaryOpExprBase
|
|||
case LONG:
|
||||
result = evalLong(leftVal.asLong(), rightVal.asLong());
|
||||
break;
|
||||
case ARRAY:
|
||||
result = evalArray(leftVal, rightVal);
|
||||
break;
|
||||
case DOUBLE:
|
||||
default:
|
||||
if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) {
|
||||
|
@ -203,21 +206,20 @@ abstract class BinaryBooleanOpExprBase extends BinaryOpExprBase
|
|||
result = evalDouble(leftVal.asDouble(), rightVal.asDouble());
|
||||
break;
|
||||
}
|
||||
if (!ExpressionProcessing.useStrictBooleans() && !type.is(ExprType.STRING)) {
|
||||
if (!ExpressionProcessing.useStrictBooleans() && !type.is(ExprType.STRING) && !type.isArray()) {
|
||||
return ExprEval.ofBoolean(result, type.getType());
|
||||
}
|
||||
return ExprEval.ofLongBoolean(result);
|
||||
}
|
||||
|
||||
protected boolean evalString(@Nullable String left, @Nullable String right)
|
||||
{
|
||||
throw new IllegalArgumentException("unsupported type " + ExprType.STRING);
|
||||
}
|
||||
protected abstract boolean evalString(@Nullable String left, @Nullable String right);
|
||||
|
||||
protected abstract boolean evalLong(long left, long right);
|
||||
|
||||
protected abstract boolean evalDouble(double left, double right);
|
||||
|
||||
protected abstract boolean evalArray(ExprEval left, ExprEval right);
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
|
@ -228,4 +230,13 @@ abstract class BinaryBooleanOpExprBase extends BinaryOpExprBase
|
|||
}
|
||||
return implicitCast;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
ExpressionType leftType = left.getOutputType(inspector);
|
||||
ExpressionType rightType = right.getOutputType(inspector);
|
||||
ExpressionType commonType = ExpressionTypeConversion.leastRestrictiveType(leftType, rightType);
|
||||
return inspector.canVectorize(left, right) && (commonType == null || commonType.isPrimitive());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,6 +58,12 @@ public class ExpressionTypeConversion
|
|||
if (Types.is(type, ExprType.STRING) && Types.is(otherType, ExprType.STRING)) {
|
||||
return ExpressionType.STRING;
|
||||
}
|
||||
// to preserve backwards compatibility, like with strings, we only use array type if both types are
|
||||
// arrays... this is pretty wack, but it is what it is. we might want to consider changing this
|
||||
// behavior in the future with a flag
|
||||
if (type.isArray() && otherType.isArray()) {
|
||||
return leastRestrictiveType(type, otherType);
|
||||
}
|
||||
|
||||
type = eval.value() != null ? type : otherType;
|
||||
otherType = otherEval.value() != null ? otherType : type;
|
||||
|
|
|
@ -255,7 +255,7 @@ public class InputBindings
|
|||
* Create {@link Expr.ObjectBinding} backed by map of {@link Supplier} to provide values for identifiers to evaluate
|
||||
* {@link Expr}
|
||||
*/
|
||||
public static Expr.ObjectBinding forInputSuppliers(final Map<String, InputSupplier> bindings)
|
||||
public static Expr.ObjectBinding forInputSuppliers(final Map<String, InputSupplier<?>> bindings)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.query.aggregation.cardinality.vector;
|
||||
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.segment.VectorColumnProcessorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
|
@ -66,6 +67,15 @@ public class CardinalityVectorProcessorFactory implements VectorColumnProcessorF
|
|||
return new LongCardinalityVectorProcessor(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CardinalityVectorProcessor makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
throw new UOE(
|
||||
"Cardinality aggregator does not support[%s] inputs",
|
||||
capabilities.toColumnType()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CardinalityVectorProcessor makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
|
|
|
@ -279,8 +279,12 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
} else if (getUpper() == null) {
|
||||
range = isLowerStrict() ? Range.greaterThan(getLower()) : Range.atLeast(getLower());
|
||||
} else {
|
||||
range = Range.range(getLower(), isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED,
|
||||
getUpper(), isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED);
|
||||
range = Range.range(
|
||||
getLower(),
|
||||
isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED,
|
||||
getUpper(),
|
||||
isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED
|
||||
);
|
||||
}
|
||||
retSet.add(range);
|
||||
return retSet;
|
||||
|
@ -323,7 +327,8 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
upperStrict,
|
||||
extractionFn,
|
||||
ordering,
|
||||
filterTuning);
|
||||
filterTuning
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -358,97 +363,212 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
return builder.appendFilterTuning(filterTuning).build();
|
||||
}
|
||||
|
||||
private DruidLongPredicate createLongPredicate()
|
||||
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
|
||||
{
|
||||
boolean hasLowerLongBound;
|
||||
boolean hasUpperLongBound;
|
||||
long lowerLongBound;
|
||||
long upperLongBound;
|
||||
boolean matchesNothing = false;
|
||||
return Suppliers.memoize(() -> {
|
||||
boolean hasLowerLongBound;
|
||||
boolean hasUpperLongBound;
|
||||
long lowerLongBound;
|
||||
long upperLongBound;
|
||||
boolean matchesNothing = false;
|
||||
|
||||
if (hasLowerBound()) {
|
||||
final Long lowerLong = GuavaUtils.tryParseLong(lower);
|
||||
if (lowerLong == null) {
|
||||
BigDecimal lowerBigDecimal = getBigDecimalLowerBoundFromFloatString(lower);
|
||||
if (lowerBigDecimal == null) {
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerLongBound = false;
|
||||
lowerLongBound = 0L;
|
||||
} else {
|
||||
try {
|
||||
lowerLongBound = lowerBigDecimal.longValueExact();
|
||||
hasLowerLongBound = true;
|
||||
}
|
||||
catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long
|
||||
if (hasLowerBound()) {
|
||||
final Long lowerLong = GuavaUtils.tryParseLong(lower);
|
||||
if (lowerLong == null) {
|
||||
BigDecimal lowerBigDecimal = getBigDecimalLowerBoundFromFloatString(lower);
|
||||
if (lowerBigDecimal == null) {
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerLongBound = false;
|
||||
lowerLongBound = 0L;
|
||||
if (lowerBigDecimal.compareTo(BigDecimal.ZERO) > 0) {
|
||||
// positive lower bound, > all longs, will match nothing
|
||||
matchesNothing = true;
|
||||
} else {
|
||||
try {
|
||||
lowerLongBound = lowerBigDecimal.longValueExact();
|
||||
hasLowerLongBound = true;
|
||||
}
|
||||
catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long
|
||||
hasLowerLongBound = false;
|
||||
lowerLongBound = 0L;
|
||||
if (lowerBigDecimal.compareTo(BigDecimal.ZERO) > 0) {
|
||||
// positive lower bound, > all longs, will match nothing
|
||||
matchesNothing = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
hasLowerLongBound = true;
|
||||
lowerLongBound = lowerLong;
|
||||
}
|
||||
} else {
|
||||
hasLowerLongBound = true;
|
||||
lowerLongBound = lowerLong;
|
||||
hasLowerLongBound = false;
|
||||
lowerLongBound = 0L;
|
||||
}
|
||||
} else {
|
||||
hasLowerLongBound = false;
|
||||
lowerLongBound = 0L;
|
||||
}
|
||||
|
||||
if (hasUpperBound()) {
|
||||
Long upperLong = GuavaUtils.tryParseLong(upper);
|
||||
if (upperLong == null) {
|
||||
BigDecimal upperBigDecimal = getBigDecimalUpperBoundFromFloatString(upper);
|
||||
if (upperBigDecimal == null) {
|
||||
if (hasUpperBound()) {
|
||||
Long upperLong = GuavaUtils.tryParseLong(upper);
|
||||
if (upperLong == null) {
|
||||
BigDecimal upperBigDecimal = getBigDecimalUpperBoundFromFloatString(upper);
|
||||
if (upperBigDecimal == null) {
|
||||
// Unparseable values fall before all actual numbers, so no numbers
|
||||
// can match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
} else {
|
||||
try {
|
||||
upperLongBound = upperBigDecimal.longValueExact();
|
||||
hasUpperLongBound = true;
|
||||
}
|
||||
catch (ArithmeticException ae) { // the BigDecimal can't be
|
||||
// contained in a long
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) {
|
||||
// negative upper bound, < all longs, will match nothing
|
||||
matchesNothing = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
hasUpperLongBound = true;
|
||||
upperLongBound = upperLong;
|
||||
}
|
||||
} else {
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of(
|
||||
hasLowerLongBound,
|
||||
lowerStrict,
|
||||
hasUpperLongBound,
|
||||
upperStrict
|
||||
);
|
||||
return RangeFilter.makeLongPredicate(rangeType, lowerLongBound, upperLongBound);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
final boolean hasLowerFloatBound;
|
||||
final boolean hasUpperFloatBound;
|
||||
final float lowerFloatBound;
|
||||
final float upperFloatBound;
|
||||
boolean matchesNothing = false;
|
||||
|
||||
if (hasLowerBound()) {
|
||||
final Float lowerFloat = Floats.tryParse(lower);
|
||||
if (lowerFloat == null) {
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerFloatBound = false;
|
||||
lowerFloatBound = 0L;
|
||||
} else {
|
||||
hasLowerFloatBound = true;
|
||||
lowerFloatBound = lowerFloat;
|
||||
}
|
||||
} else {
|
||||
hasLowerFloatBound = false;
|
||||
lowerFloatBound = 0L;
|
||||
}
|
||||
|
||||
if (hasUpperBound()) {
|
||||
Float upperFloat = Floats.tryParse(upper);
|
||||
if (upperFloat == null) {
|
||||
// Unparseable values fall before all actual numbers, so no numbers
|
||||
// can match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
hasUpperFloatBound = false;
|
||||
upperFloatBound = 0L;
|
||||
} else {
|
||||
try {
|
||||
upperLongBound = upperBigDecimal.longValueExact();
|
||||
hasUpperLongBound = true;
|
||||
}
|
||||
catch (ArithmeticException ae) { // the BigDecimal can't be
|
||||
// contained in a long
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) {
|
||||
// negative upper bound, < all longs, will match nothing
|
||||
matchesNothing = true;
|
||||
}
|
||||
}
|
||||
hasUpperFloatBound = true;
|
||||
upperFloatBound = upperFloat;
|
||||
}
|
||||
} else {
|
||||
hasUpperLongBound = true;
|
||||
upperLongBound = upperLong;
|
||||
hasUpperFloatBound = false;
|
||||
upperFloatBound = 0L;
|
||||
}
|
||||
} else {
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
return makeLongPredicateFromBounds(
|
||||
hasLowerLongBound,
|
||||
hasUpperLongBound,
|
||||
lowerStrict,
|
||||
upperStrict,
|
||||
lowerLongBound,
|
||||
upperLongBound);
|
||||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
|
||||
final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of(
|
||||
hasLowerFloatBound,
|
||||
lowerStrict,
|
||||
hasUpperFloatBound,
|
||||
upperStrict
|
||||
);
|
||||
final DruidDoublePredicate doublePredicate = RangeFilter.makeDoublePredicate(
|
||||
rangeType,
|
||||
lowerFloatBound,
|
||||
upperFloatBound
|
||||
);
|
||||
return doublePredicate::applyDouble;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
|
||||
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
||||
{
|
||||
Supplier<DruidLongPredicate> longPredicate = () -> createLongPredicate();
|
||||
return Suppliers.memoize(longPredicate);
|
||||
return Suppliers.memoize(() -> {
|
||||
final boolean hasLowerBound;
|
||||
final boolean hasUpperBound;
|
||||
final double lowerDoubleBound;
|
||||
final double upperDoubleBound;
|
||||
boolean matchesNothing = false;
|
||||
|
||||
if (hasLowerBound()) {
|
||||
final Double lowerDouble = Doubles.tryParse(lower);
|
||||
if (lowerDouble == null) {
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerBound = false;
|
||||
lowerDoubleBound = 0L;
|
||||
} else {
|
||||
hasLowerBound = true;
|
||||
lowerDoubleBound = lowerDouble;
|
||||
}
|
||||
} else {
|
||||
hasLowerBound = false;
|
||||
lowerDoubleBound = 0L;
|
||||
}
|
||||
|
||||
if (hasUpperBound()) {
|
||||
Double upperDouble = Doubles.tryParse(upper);
|
||||
if (upperDouble == null) {
|
||||
// Unparseable values fall before all actual numbers, so no numbers can
|
||||
// match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperBound = false;
|
||||
upperDoubleBound = 0L;
|
||||
} else {
|
||||
hasUpperBound = true;
|
||||
upperDoubleBound = upperDouble;
|
||||
}
|
||||
} else {
|
||||
hasUpperBound = false;
|
||||
upperDoubleBound = 0L;
|
||||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of(
|
||||
hasLowerBound,
|
||||
lowerStrict,
|
||||
hasUpperBound,
|
||||
upperStrict
|
||||
);
|
||||
return RangeFilter.makeDoublePredicate(rangeType, lowerDoubleBound, upperDoubleBound);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -486,225 +606,4 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
return convertedBD.setScale(0, RoundingMode.FLOOR);
|
||||
}
|
||||
}
|
||||
|
||||
private DruidFloatPredicate createDruidFloatPredicate()
|
||||
{
|
||||
final boolean hasLowerFloatBound;
|
||||
final boolean hasUpperFloatBound;
|
||||
final float lowerFloatBound;
|
||||
final float upperFloatBound;
|
||||
boolean matchesNothing = false;
|
||||
|
||||
if (hasLowerBound()) {
|
||||
final Float lowerFloat = Floats.tryParse(lower);
|
||||
if (lowerFloat == null) {
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerFloatBound = false;
|
||||
lowerFloatBound = 0L;
|
||||
} else {
|
||||
hasLowerFloatBound = true;
|
||||
lowerFloatBound = lowerFloat;
|
||||
}
|
||||
} else {
|
||||
hasLowerFloatBound = false;
|
||||
lowerFloatBound = 0L;
|
||||
}
|
||||
|
||||
if (hasUpperBound()) {
|
||||
Float upperFloat = Floats.tryParse(upper);
|
||||
if (upperFloat == null) {
|
||||
// Unparseable values fall before all actual numbers, so no numbers
|
||||
// can match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperFloatBound = false;
|
||||
upperFloatBound = 0L;
|
||||
} else {
|
||||
hasUpperFloatBound = true;
|
||||
upperFloatBound = upperFloat;
|
||||
}
|
||||
} else {
|
||||
hasUpperFloatBound = false;
|
||||
upperFloatBound = 0L;
|
||||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
return input -> {
|
||||
final DruidDoublePredicate druidDoublePredicate = makeDoublePredicateFromBounds(
|
||||
hasLowerFloatBound,
|
||||
hasUpperFloatBound,
|
||||
lowerStrict,
|
||||
upperStrict,
|
||||
(double) lowerFloatBound,
|
||||
(double) upperFloatBound);
|
||||
return druidDoublePredicate.applyDouble((double) input);
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
||||
{
|
||||
Supplier<DruidFloatPredicate> floatPredicate = () -> createDruidFloatPredicate();
|
||||
return Suppliers.memoize(floatPredicate);
|
||||
}
|
||||
|
||||
private DruidDoublePredicate createDruidDoublePredicate()
|
||||
{
|
||||
final boolean hasLowerBound;
|
||||
final boolean hasUpperBound;
|
||||
final double lowerDoubleBound;
|
||||
final double upperDoubleBound;
|
||||
boolean matchesNothing = false;
|
||||
|
||||
if (hasLowerBound()) {
|
||||
final Double lowerDouble = Doubles.tryParse(lower);
|
||||
if (lowerDouble == null) {
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerBound = false;
|
||||
lowerDoubleBound = 0L;
|
||||
} else {
|
||||
hasLowerBound = true;
|
||||
lowerDoubleBound = lowerDouble;
|
||||
}
|
||||
} else {
|
||||
hasLowerBound = false;
|
||||
lowerDoubleBound = 0L;
|
||||
}
|
||||
|
||||
if (hasUpperBound()) {
|
||||
Double upperDouble = Doubles.tryParse(upper);
|
||||
if (upperDouble == null) {
|
||||
// Unparseable values fall before all actual numbers, so no numbers can
|
||||
// match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperBound = false;
|
||||
upperDoubleBound = 0L;
|
||||
} else {
|
||||
hasUpperBound = true;
|
||||
upperDoubleBound = upperDouble;
|
||||
}
|
||||
} else {
|
||||
hasUpperBound = false;
|
||||
upperDoubleBound = 0L;
|
||||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
return makeDoublePredicateFromBounds(
|
||||
hasLowerBound,
|
||||
hasUpperBound,
|
||||
lowerStrict,
|
||||
upperStrict,
|
||||
lowerDoubleBound,
|
||||
upperDoubleBound);
|
||||
}
|
||||
}
|
||||
|
||||
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
||||
{
|
||||
Supplier<DruidDoublePredicate> doublePredicate = () -> createDruidDoublePredicate();
|
||||
return Suppliers.memoize(doublePredicate);
|
||||
}
|
||||
|
||||
private static DruidLongPredicate makeLongPredicateFromBounds(
|
||||
final boolean hasLowerLongBound,
|
||||
final boolean hasUpperLongBound,
|
||||
final boolean lowerStrict,
|
||||
final boolean upperStrict,
|
||||
final long lowerLongBound,
|
||||
final long upperLongBound)
|
||||
{
|
||||
if (hasLowerLongBound && hasUpperLongBound) {
|
||||
if (upperStrict && lowerStrict) {
|
||||
return input -> input > lowerLongBound && input < upperLongBound;
|
||||
} else if (lowerStrict) {
|
||||
return input -> input > lowerLongBound && input <= upperLongBound;
|
||||
} else if (upperStrict) {
|
||||
return input -> input >= lowerLongBound && input < upperLongBound;
|
||||
} else {
|
||||
return input -> input >= lowerLongBound && input <= upperLongBound;
|
||||
}
|
||||
} else if (hasUpperLongBound) {
|
||||
if (upperStrict) {
|
||||
return input -> input < upperLongBound;
|
||||
} else {
|
||||
return input -> input <= upperLongBound;
|
||||
}
|
||||
} else if (hasLowerLongBound) {
|
||||
if (lowerStrict) {
|
||||
return input -> input > lowerLongBound;
|
||||
} else {
|
||||
return input -> input >= lowerLongBound;
|
||||
}
|
||||
} else {
|
||||
return DruidLongPredicate.ALWAYS_TRUE;
|
||||
}
|
||||
}
|
||||
|
||||
private static DruidDoublePredicate makeDoublePredicateFromBounds(
|
||||
final boolean hasLowerDoubleBound,
|
||||
final boolean hasUpperDoubleBound,
|
||||
final boolean lowerStrict,
|
||||
final boolean upperStrict,
|
||||
final double lowerDoubleBound,
|
||||
final double upperDoubleBound)
|
||||
{
|
||||
if (hasLowerDoubleBound && hasUpperDoubleBound) {
|
||||
if (upperStrict && lowerStrict) {
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return ((lowerComparing > 0)) && (upperComparing > 0);
|
||||
};
|
||||
} else if (lowerStrict) {
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return (lowerComparing > 0) && (upperComparing >= 0);
|
||||
};
|
||||
} else if (upperStrict) {
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return (lowerComparing >= 0) && (upperComparing > 0);
|
||||
};
|
||||
} else {
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return (lowerComparing >= 0) && (upperComparing >= 0);
|
||||
};
|
||||
}
|
||||
} else if (hasUpperDoubleBound) {
|
||||
if (upperStrict) {
|
||||
return input -> {
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return upperComparing > 0;
|
||||
};
|
||||
} else {
|
||||
return input -> {
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return upperComparing >= 0;
|
||||
};
|
||||
}
|
||||
} else if (hasLowerDoubleBound) {
|
||||
if (lowerStrict) {
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
return lowerComparing > 0;
|
||||
};
|
||||
} else {
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
return lowerComparing >= 0;
|
||||
};
|
||||
}
|
||||
} else {
|
||||
return DruidDoublePredicate.ALWAYS_TRUE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,7 +48,10 @@ import java.util.Set;
|
|||
@JsonSubTypes.Type(name = "like", value = LikeDimFilter.class),
|
||||
@JsonSubTypes.Type(name = "expression", value = ExpressionDimFilter.class),
|
||||
@JsonSubTypes.Type(name = "true", value = TrueDimFilter.class),
|
||||
@JsonSubTypes.Type(name = "false", value = FalseDimFilter.class)
|
||||
@JsonSubTypes.Type(name = "false", value = FalseDimFilter.class),
|
||||
@JsonSubTypes.Type(name = "null", value = NullFilter.class),
|
||||
@JsonSubTypes.Type(name = "equals", value = EqualityFilter.class),
|
||||
@JsonSubTypes.Type(name = "range", value = RangeFilter.class)
|
||||
})
|
||||
public interface DimFilter extends Cacheable
|
||||
{
|
||||
|
|
|
@ -54,6 +54,10 @@ public class DimFilterUtils
|
|||
static final byte TRUE_CACHE_ID = 0xF;
|
||||
static final byte FALSE_CACHE_ID = 0x11;
|
||||
public static final byte BLOOM_DIM_FILTER_CACHE_ID = 0x10;
|
||||
static final byte NULL_CACHE_ID = 0x12;
|
||||
static final byte EQUALS_CACHE_ID = 0x13;
|
||||
static final byte RANGE_CACHE_ID = 0x14;
|
||||
|
||||
|
||||
public static final byte STRING_SEPARATOR = (byte) 0xFF;
|
||||
|
||||
|
|
|
@ -30,6 +30,8 @@ public interface DruidFloatPredicate
|
|||
{
|
||||
DruidFloatPredicate ALWAYS_FALSE = input -> false;
|
||||
|
||||
DruidFloatPredicate ALWAYS_TRUE = input -> true;
|
||||
|
||||
DruidFloatPredicate MATCH_NULL_ONLY = new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,11 @@ package org.apache.druid.query.filter;
|
|||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
@SubclassesMustOverrideEqualsAndHashCode
|
||||
public interface DruidPredicateFactory
|
||||
|
@ -33,6 +38,11 @@ public interface DruidPredicateFactory
|
|||
|
||||
DruidDoublePredicate makeDoublePredicate();
|
||||
|
||||
default Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> inputType)
|
||||
{
|
||||
throw new UOE("Predicate does not support ARRAY types");
|
||||
}
|
||||
|
||||
/**
|
||||
* Object predicate is currently only used by vectorized matchers for non-string object selectors. This currently
|
||||
* means it will be used only if we encounter COMPLEX types, but will also include array types once they are more
|
||||
|
|
|
@ -0,0 +1,581 @@
|
|||
/*
|
||||
* 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.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.TreeRangeSet;
|
||||
import org.apache.druid.error.InvalidInput;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
||||
import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnProcessorFactory;
|
||||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.TypeStrategy;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.filter.PredicateValueMatcherFactory;
|
||||
import org.apache.druid.segment.filter.ValueMatchers;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
import org.apache.druid.segment.index.semantic.ValueIndexes;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
public class EqualityFilter extends AbstractOptimizableDimFilter implements Filter
|
||||
{
|
||||
private final String column;
|
||||
private final ColumnType matchValueType;
|
||||
private final Object matchValue;
|
||||
@Nullable
|
||||
private final ExtractionFn extractionFn;
|
||||
@Nullable
|
||||
private final FilterTuning filterTuning;
|
||||
private final DruidPredicateFactory predicateFactory;
|
||||
|
||||
@JsonCreator
|
||||
public EqualityFilter(
|
||||
@JsonProperty("column") String column,
|
||||
@JsonProperty("matchValueType") ColumnType matchValueType,
|
||||
@JsonProperty("matchValue") Object matchValue,
|
||||
@JsonProperty("extractionFn") @Nullable ExtractionFn extractionFn,
|
||||
@JsonProperty("filterTuning") @Nullable FilterTuning filterTuning
|
||||
)
|
||||
{
|
||||
if (column == null) {
|
||||
throw InvalidInput.exception("Invalid equality filter, column cannot be null");
|
||||
}
|
||||
this.column = column;
|
||||
if (matchValueType == null) {
|
||||
throw InvalidInput.exception("Invalid equality filter on column [%s], matchValueType cannot be null", column);
|
||||
}
|
||||
this.matchValueType = matchValueType;
|
||||
if (matchValue == null) {
|
||||
throw InvalidInput.exception("Invalid equality filter on column [%s], matchValue cannot be null", column);
|
||||
}
|
||||
this.matchValue = matchValue;
|
||||
// remove once SQL planner no longer uses extractionFn
|
||||
this.extractionFn = extractionFn;
|
||||
this.filterTuning = filterTuning;
|
||||
this.predicateFactory = new EqualityPredicateFactory(matchValue, matchValueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final TypeStrategy<Object> typeStrategy = matchValueType.getStrategy();
|
||||
final int size = typeStrategy.estimateSizeBytes(matchValue);
|
||||
final ByteBuffer valueBuffer = ByteBuffer.allocate(size);
|
||||
typeStrategy.write(valueBuffer, matchValue, size);
|
||||
return new CacheKeyBuilder(DimFilterUtils.EQUALS_CACHE_ID)
|
||||
.appendByte(DimFilterUtils.STRING_SEPARATOR)
|
||||
.appendString(column)
|
||||
.appendByte(DimFilterUtils.STRING_SEPARATOR)
|
||||
.appendString(matchValueType.asTypeString())
|
||||
.appendByte(DimFilterUtils.STRING_SEPARATOR)
|
||||
.appendByteArray(valueBuffer.array())
|
||||
.appendByte(DimFilterUtils.STRING_SEPARATOR)
|
||||
.appendByteArray(extractionFn == null ? new byte[0] : extractionFn.getCacheKey())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimFilter optimize()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
if (extractionFn == null) {
|
||||
return this;
|
||||
} else {
|
||||
return new DimensionPredicateFilter(column, predicateFactory, extractionFn, filterTuning);
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getColumn()
|
||||
{
|
||||
return column;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ColumnType getMatchValueType()
|
||||
{
|
||||
return matchValueType;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Object getMatchValue()
|
||||
{
|
||||
return matchValue;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public ExtractionFn getExtractionFn()
|
||||
{
|
||||
return extractionFn;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public FilterTuning getFilterTuning()
|
||||
{
|
||||
return filterTuning;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
DimFilter.DimFilterToStringBuilder bob =
|
||||
new DimFilter.DimFilterToStringBuilder().appendDimension(column, extractionFn)
|
||||
.append(" = ")
|
||||
.append(matchValue);
|
||||
|
||||
if (!ColumnType.STRING.equals(matchValueType)) {
|
||||
bob.append(" (" + matchValueType.asTypeString() + ")");
|
||||
}
|
||||
return bob.appendFilterTuning(filterTuning).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
EqualityFilter that = (EqualityFilter) o;
|
||||
if (!column.equals(that.column)) {
|
||||
return false;
|
||||
}
|
||||
if (!Objects.equals(matchValueType, that.matchValueType)) {
|
||||
return false;
|
||||
}
|
||||
if (!Objects.equals(extractionFn, that.extractionFn)) {
|
||||
return false;
|
||||
}
|
||||
if (!Objects.equals(filterTuning, that.filterTuning)) {
|
||||
return false;
|
||||
}
|
||||
if (matchValueType.isArray()) {
|
||||
// just use predicate to see if the values are the same
|
||||
final ExprEval<?> thatValue = ExprEval.ofType(
|
||||
ExpressionType.fromColumnType(that.matchValueType),
|
||||
that.matchValue
|
||||
);
|
||||
final Predicate<Object[]> arrayPredicate = predicateFactory.makeArrayPredicate(matchValueType);
|
||||
return arrayPredicate.apply(thatValue.asArray());
|
||||
} else {
|
||||
return Objects.equals(matchValue, that.matchValue);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(column, matchValueType, matchValue, extractionFn, filterTuning);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RangeSet<String> getDimensionRangeSet(String dimension)
|
||||
{
|
||||
if (!Objects.equals(getColumn(), dimension) || getExtractionFn() != null) {
|
||||
return null;
|
||||
}
|
||||
RangeSet<String> retSet = TreeRangeSet.create();
|
||||
retSet.add(Range.singleton(String.valueOf(matchValue)));
|
||||
return retSet;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
|
||||
{
|
||||
if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column);
|
||||
if (indexSupplier == null) {
|
||||
return new AllFalseBitmapColumnIndex(selector);
|
||||
}
|
||||
|
||||
final ValueIndexes valueIndexes = indexSupplier.as(ValueIndexes.class);
|
||||
if (valueIndexes != null) {
|
||||
return valueIndexes.forValue(matchValue, matchValueType);
|
||||
}
|
||||
|
||||
if (matchValueType.isPrimitive()) {
|
||||
final StringValueSetIndexes stringValueSetIndexes = indexSupplier.as(StringValueSetIndexes.class);
|
||||
if (stringValueSetIndexes != null) {
|
||||
|
||||
return stringValueSetIndexes.forValue(String.valueOf(matchValue));
|
||||
}
|
||||
}
|
||||
// column exists, but has no indexes we can use
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||
{
|
||||
return ColumnProcessors.makeProcessor(
|
||||
column,
|
||||
new TypedConstantValueMatcherFactory(matchValue, matchValueType, predicateFactory),
|
||||
factory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
|
||||
{
|
||||
final ColumnCapabilities capabilities = factory.getColumnCapabilities(column);
|
||||
|
||||
if (matchValueType.isPrimitive() && (capabilities == null || capabilities.isPrimitive())) {
|
||||
return ColumnProcessors.makeVectorProcessor(
|
||||
column,
|
||||
VectorValueMatcherColumnProcessorFactory.instance(),
|
||||
factory
|
||||
).makeMatcher(matchValue, matchValueType);
|
||||
}
|
||||
return ColumnProcessors.makeVectorProcessor(
|
||||
column,
|
||||
VectorValueMatcherColumnProcessorFactory.instance(),
|
||||
factory
|
||||
).makeMatcher(new EqualityPredicateFactory(matchValue, matchValueType));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
|
||||
{
|
||||
return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorizeMatcher(ColumnInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getRequiredColumns()
|
||||
{
|
||||
return ImmutableSet.of(column);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsRequiredColumnRewrite()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter rewriteRequiredColumns(Map<String, String> columnRewrites)
|
||||
{
|
||||
String rewriteDimensionTo = columnRewrites.get(column);
|
||||
|
||||
if (rewriteDimensionTo == null) {
|
||||
throw new IAE(
|
||||
"Received a non-applicable rewrite: %s, filter's dimension: %s",
|
||||
columnRewrites,
|
||||
columnRewrites
|
||||
);
|
||||
}
|
||||
|
||||
return new EqualityFilter(
|
||||
rewriteDimensionTo,
|
||||
matchValueType,
|
||||
matchValue,
|
||||
extractionFn,
|
||||
filterTuning
|
||||
);
|
||||
}
|
||||
|
||||
private static class EqualityPredicateFactory implements DruidPredicateFactory
|
||||
{
|
||||
private final ExprEval<?> matchValue;
|
||||
private final ColumnType matchValueType;
|
||||
private final Supplier<Predicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidLongPredicate> longPredicateSupplier;
|
||||
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
|
||||
private final Supplier<DruidDoublePredicate> doublePredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, Predicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<Predicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
private final Supplier<Predicate<Object>> objectPredicateSupplier;
|
||||
|
||||
public EqualityPredicateFactory(Object matchValue, ColumnType matchValueType)
|
||||
{
|
||||
this.matchValue = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||
this.matchValueType = matchValueType;
|
||||
this.stringPredicateSupplier = makeStringPredicateSupplier();
|
||||
this.longPredicateSupplier = makeLongPredicateSupplier();
|
||||
this.floatPredicateSupplier = makeFloatPredicateSupplier();
|
||||
this.doublePredicateSupplier = makeDoublePredicateSupplier();
|
||||
this.objectPredicateSupplier = makeObjectPredicateSupplier();
|
||||
this.arrayPredicates = new ConcurrentHashMap<>();
|
||||
this.typeDetectingArrayPredicateSupplier = makeTypeDetectingArrayPredicate();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
{
|
||||
return stringPredicateSupplier.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return longPredicateSupplier.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return floatPredicateSupplier.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return doublePredicateSupplier.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
if (arrayType == null) {
|
||||
// fall back to per row detection if input array type is unknown
|
||||
return typeDetectingArrayPredicateSupplier.get();
|
||||
}
|
||||
|
||||
return arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return objectPredicateSupplier.get();
|
||||
}
|
||||
|
||||
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> Predicates.equalTo(matchValue.castTo(ExpressionType.STRING).asString()));
|
||||
}
|
||||
|
||||
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
final Long valueAsLong = (Long) matchValue.castTo(ExpressionType.LONG).valueOrDefault();
|
||||
|
||||
if (valueAsLong == null) {
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = valueAsLong;
|
||||
return input -> input == unboxedLong;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
final Double doubleValue = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault();
|
||||
|
||||
if (doubleValue == null) {
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// Compare with floatToIntBits instead of == to canonicalize NaNs.
|
||||
final int floatBits = Float.floatToIntBits(doubleValue.floatValue());
|
||||
return input -> Float.floatToIntBits(input) == floatBits;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
final Double aDouble = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault();
|
||||
|
||||
if (aDouble == null) {
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// Compare with doubleToLongBits instead of == to canonicalize NaNs.
|
||||
final long bits = Double.doubleToLongBits(aDouble);
|
||||
return input -> Double.doubleToLongBits(input) == bits;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object>> makeObjectPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
if (matchValueType.equals(ColumnType.NESTED_DATA)) {
|
||||
return input -> Objects.equals(StructuredData.unwrap(input), StructuredData.unwrap(matchValue.value()));
|
||||
}
|
||||
return Predicates.equalTo(matchValue.valueOrDefault());
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
{
|
||||
return Suppliers.memoize(() -> input -> {
|
||||
final ExprEval<?> eval = ExprEval.bestEffortOf(input);
|
||||
final Comparator<Object[]> arrayComparator = eval.type().getNullableStrategy();
|
||||
final Object[] matchArray = matchValue.castTo(eval.type()).asArray();
|
||||
return arrayComparator.compare(input, matchArray) == 0;
|
||||
});
|
||||
}
|
||||
private Predicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
|
||||
final Comparator<Object[]> arrayComparator = arrayType.getNullableStrategy();
|
||||
final Object[] matchArray = matchValue.castTo(expressionType).asArray();
|
||||
return input -> arrayComparator.compare(input, matchArray) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
EqualityPredicateFactory that = (EqualityPredicateFactory) o;
|
||||
return Objects.equals(matchValue, that.matchValue) && Objects.equals(matchValueType, that.matchValueType);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(matchValue, matchValueType);
|
||||
}
|
||||
}
|
||||
|
||||
private static class TypedConstantValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
|
||||
{
|
||||
private final ExprEval<?> matchValue;
|
||||
private final PredicateValueMatcherFactory predicateMatcherFactory;
|
||||
|
||||
public TypedConstantValueMatcherFactory(
|
||||
Object matchValue,
|
||||
ColumnType matchValueType,
|
||||
DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
this.matchValue = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||
this.predicateMatcherFactory = new PredicateValueMatcherFactory(predicateFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType defaultType()
|
||||
{
|
||||
return ColumnType.UNKNOWN_COMPLEX;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
return ValueMatchers.makeStringValueMatcher(
|
||||
selector,
|
||||
matchValue.castTo(ExpressionType.STRING).asString(),
|
||||
multiValue
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeFloatValueMatcher(selector, (float) matchValue.castTo(ExpressionType.DOUBLE).asDouble());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeDoubleValueMatcher(selector, matchValue.castTo(ExpressionType.DOUBLE).asDouble());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeLongValueMatcher(selector, matchValue.castTo(ExpressionType.LONG).asLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeArrayProcessor(
|
||||
BaseObjectColumnValueSelector<?> selector,
|
||||
@Nullable ColumnCapabilities columnCapabilities
|
||||
)
|
||||
{
|
||||
return predicateMatcherFactory.makeArrayProcessor(selector, columnCapabilities);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
return predicateMatcherFactory.makeComplexProcessor(selector);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -26,7 +26,7 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
|||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -57,11 +57,11 @@ import org.apache.druid.segment.ColumnProcessors;
|
|||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.column.Utf8ValueSetIndex;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -292,20 +292,20 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
|
||||
if (indexSupplier == null) {
|
||||
// column doesn't exist, match against null
|
||||
return Filters.makeNullIndex(
|
||||
return Filters.makeMissingColumnNullIndex(
|
||||
predicateFactory.makeStringPredicate().apply(null),
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
final Utf8ValueSetIndex utf8ValueSetIndex = indexSupplier.as(Utf8ValueSetIndex.class);
|
||||
if (utf8ValueSetIndex != null) {
|
||||
return utf8ValueSetIndex.forSortedValuesUtf8(valuesUtf8);
|
||||
final Utf8ValueSetIndexes utf8ValueSetIndexes = indexSupplier.as(Utf8ValueSetIndexes.class);
|
||||
if (utf8ValueSetIndexes != null) {
|
||||
return utf8ValueSetIndexes.forSortedValuesUtf8(valuesUtf8);
|
||||
}
|
||||
|
||||
final StringValueSetIndex stringValueSetIndex = indexSupplier.as(StringValueSetIndex.class);
|
||||
if (stringValueSetIndex != null) {
|
||||
return stringValueSetIndex.forSortedValues(values);
|
||||
final StringValueSetIndexes stringValueSetIndexes = indexSupplier.as(StringValueSetIndexes.class);
|
||||
if (stringValueSetIndexes != null) {
|
||||
return stringValueSetIndexes.forSortedValues(values);
|
||||
}
|
||||
}
|
||||
return Filters.makePredicateIndex(
|
||||
|
|
|
@ -0,0 +1,316 @@
|
|||
/*
|
||||
* 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.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.TreeRangeSet;
|
||||
import org.apache.druid.error.InvalidInput;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
||||
import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.NullValueIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
public class NullFilter extends AbstractOptimizableDimFilter implements Filter
|
||||
{
|
||||
public static NullFilter forColumn(String column)
|
||||
{
|
||||
return new NullFilter(column, null, null);
|
||||
}
|
||||
|
||||
private final String column;
|
||||
@Nullable
|
||||
private final ExtractionFn extractionFn;
|
||||
@Nullable
|
||||
private final FilterTuning filterTuning;
|
||||
|
||||
@JsonCreator
|
||||
public NullFilter(
|
||||
@JsonProperty("column") String column,
|
||||
@JsonProperty("extractionFn") @Nullable ExtractionFn extractionFn,
|
||||
@JsonProperty("filterTuning") @Nullable FilterTuning filterTuning
|
||||
)
|
||||
{
|
||||
if (column == null) {
|
||||
throw InvalidInput.exception("Invalid null filter, column cannot be null");
|
||||
}
|
||||
this.column = column;
|
||||
// remove once SQL planner no longer uses extractionFn
|
||||
this.extractionFn = extractionFn;
|
||||
this.filterTuning = filterTuning;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getColumn()
|
||||
{
|
||||
return column;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public ExtractionFn getExtractionFn()
|
||||
{
|
||||
return extractionFn;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public FilterTuning getFilterTuning()
|
||||
{
|
||||
return filterTuning;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new CacheKeyBuilder(DimFilterUtils.NULL_CACHE_ID)
|
||||
.appendByte(DimFilterUtils.STRING_SEPARATOR)
|
||||
.appendString(column)
|
||||
.appendByte(DimFilterUtils.STRING_SEPARATOR)
|
||||
.appendByteArray(extractionFn == null ? new byte[0] : extractionFn.getCacheKey())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimFilter optimize()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
if (extractionFn == null) {
|
||||
return this;
|
||||
} else {
|
||||
return new DimensionPredicateFilter(column, NullPredicateFactory.INSTANCE, extractionFn, filterTuning);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public RangeSet<String> getDimensionRangeSet(String dimension)
|
||||
{
|
||||
RangeSet<String> retSet = TreeRangeSet.create();
|
||||
// Nulls are less than empty String in segments
|
||||
retSet.add(Range.lessThan(""));
|
||||
return retSet;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
|
||||
{
|
||||
if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) {
|
||||
return null;
|
||||
}
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column);
|
||||
if (indexSupplier == null) {
|
||||
return new AllTrueBitmapColumnIndex(selector);
|
||||
}
|
||||
final NullValueIndex nullValueIndex = indexSupplier.as(NullValueIndex.class);
|
||||
if (nullValueIndex == null) {
|
||||
return null;
|
||||
}
|
||||
return nullValueIndex.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||
{
|
||||
return Filters.makeValueMatcher(factory, column, NullPredicateFactory.INSTANCE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
|
||||
{
|
||||
return ColumnProcessors.makeVectorProcessor(
|
||||
column,
|
||||
VectorValueMatcherColumnProcessorFactory.instance(),
|
||||
factory
|
||||
).makeMatcher(NullPredicateFactory.INSTANCE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
|
||||
{
|
||||
return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorizeMatcher(ColumnInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getRequiredColumns()
|
||||
{
|
||||
return ImmutableSet.of(column);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsRequiredColumnRewrite()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter rewriteRequiredColumns(Map<String, String> columnRewrites)
|
||||
{
|
||||
String rewriteDimensionTo = columnRewrites.get(column);
|
||||
|
||||
if (rewriteDimensionTo == null) {
|
||||
throw new IAE(
|
||||
"Received a non-applicable rewrite: %s, filter's dimension: %s",
|
||||
columnRewrites,
|
||||
columnRewrites
|
||||
);
|
||||
}
|
||||
return new NullFilter(rewriteDimensionTo, extractionFn, filterTuning);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
NullFilter that = (NullFilter) o;
|
||||
return Objects.equals(column, that.column) &&
|
||||
Objects.equals(extractionFn, that.extractionFn) &&
|
||||
Objects.equals(filterTuning, that.filterTuning);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(column, extractionFn, filterTuning);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return new DimFilterToStringBuilder().appendDimension(column, extractionFn)
|
||||
.append(" IS NULL")
|
||||
.appendFilterTuning(filterTuning)
|
||||
.build();
|
||||
}
|
||||
|
||||
private static class NullPredicateFactory implements DruidPredicateFactory
|
||||
{
|
||||
public static final NullPredicateFactory INSTANCE = new NullPredicateFactory();
|
||||
|
||||
private NullPredicateFactory()
|
||||
{
|
||||
// no instantiation
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
{
|
||||
return Predicates.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return DruidLongPredicate.MATCH_NULL_ONLY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return DruidFloatPredicate.MATCH_NULL_ONLY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return DruidDoublePredicate.MATCH_NULL_ONLY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
return Predicates.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return Predicates.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return super.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj)
|
||||
{
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "NullPredicateFactory{}";
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* 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.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class ArrayVectorValueMatcher implements VectorValueMatcherFactory
|
||||
{
|
||||
protected final TypeSignature<ValueType> columnType;
|
||||
protected final VectorObjectSelector selector;
|
||||
|
||||
public ArrayVectorValueMatcher(
|
||||
TypeSignature<ValueType> columnType,
|
||||
VectorObjectSelector selector
|
||||
)
|
||||
{
|
||||
this.columnType = columnType;
|
||||
this.selector = selector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(@Nullable String value)
|
||||
{
|
||||
throw new UnsupportedOperationException(
|
||||
"Vectorized matcher cannot make string matcher for ARRAY types"
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
throw new UnsupportedOperationException(
|
||||
"Vectorized matcher cannot make object matcher for ARRAY types"
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<Object[]> predicate = predicateFactory.makeArrayPredicate(columnType);
|
||||
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
|
||||
|
||||
@Override
|
||||
public ReadableVectorMatch match(final ReadableVectorMatch mask)
|
||||
{
|
||||
final Object[] vector = selector.getObjectVector();
|
||||
final int[] selection = match.getSelection();
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
Object o = vector[rowNum];
|
||||
if (o == null || o instanceof Object[]) {
|
||||
if (predicate.apply((Object[]) o)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else {
|
||||
if (predicate.apply(new Object[]{o})) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -19,9 +19,12 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidDoublePredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -48,8 +51,22 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
return BooleanVectorValueMatcher.of(selector, false);
|
||||
}
|
||||
|
||||
final double matchValDouble = matchVal;
|
||||
return makeDoubleMatcher(matchVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
||||
ExprEval<?> cast = eval.castTo(ExpressionType.DOUBLE);
|
||||
if (cast.isNumericNull()) {
|
||||
return makeNullValueMatcher(selector);
|
||||
}
|
||||
return makeDoubleMatcher(cast.asDouble());
|
||||
}
|
||||
|
||||
private BaseVectorValueMatcher makeDoubleMatcher(double matchValDouble)
|
||||
{
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
|
||||
|
@ -74,12 +91,12 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
|
@ -111,7 +128,6 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -19,9 +19,12 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidFloatPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -50,6 +53,22 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
|
||||
final float matchValFloat = matchVal;
|
||||
|
||||
return makeFloatMatcher(matchValFloat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
||||
ExprEval<?> cast = eval.castTo(ExpressionType.DOUBLE);
|
||||
if (cast.isNumericNull()) {
|
||||
return makeNullValueMatcher(selector);
|
||||
}
|
||||
return makeFloatMatcher((float) cast.asDouble());
|
||||
}
|
||||
|
||||
private BaseVectorValueMatcher makeFloatMatcher(float matchValFloat)
|
||||
{
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
|
||||
|
@ -75,7 +94,6 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
@ -112,7 +130,6 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -19,9 +19,12 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -50,6 +53,22 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
|
||||
final long matchValLong = matchVal;
|
||||
|
||||
return makeLongMatcher(matchValLong);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
||||
ExprEval<?> cast = eval.castTo(ExpressionType.LONG);
|
||||
if (cast.isNumericNull()) {
|
||||
return makeNullValueMatcher(selector);
|
||||
}
|
||||
return makeLongMatcher(cast.asLong());
|
||||
}
|
||||
|
||||
private BaseVectorValueMatcher makeLongMatcher(long matchValLong)
|
||||
{
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
|
||||
|
@ -75,7 +94,6 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
@ -112,7 +130,6 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -21,8 +21,11 @@ package org.apache.druid.query.filter.vector;
|
|||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
|
||||
|
@ -89,7 +92,6 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
@ -98,6 +100,14 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
||||
ExprEval<?> cast = eval.castTo(ExpressionType.STRING);
|
||||
return makeMatcher(cast.asString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
|
@ -159,7 +169,6 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
@ -199,7 +208,6 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.query.filter.vector;
|
|||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -52,6 +53,12 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
return BooleanVectorValueMatcher.of(selector, value == null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
return BooleanVectorValueMatcher.of(selector, value == null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
|
@ -77,7 +84,6 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -21,8 +21,11 @@ package org.apache.druid.query.filter.vector;
|
|||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.filter.ValueMatchers;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
|
||||
|
@ -97,7 +100,6 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
@ -106,6 +108,14 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
||||
ExprEval<?> cast = eval.castTo(ExpressionType.STRING);
|
||||
return makeMatcher(cast.asString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
|
@ -157,7 +167,6 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
@ -183,7 +192,6 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -20,7 +20,10 @@
|
|||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -61,12 +64,19 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
||||
{
|
||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
||||
ExprEval<?> cast = eval.castTo(ExpressionType.STRING);
|
||||
return makeMatcher(cast.asString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
|
@ -92,7 +102,6 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
}
|
||||
|
||||
match.setSelectionSize(numRows);
|
||||
assert match.isValid(mask);
|
||||
return match;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -86,6 +86,12 @@ public class VectorValueMatcherColumnProcessorFactory implements VectorColumnPro
|
|||
return new LongVectorValueMatcher(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcherFactory makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
return new ArrayVectorValueMatcher(capabilities, selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcherFactory makeObjectProcessor(
|
||||
final ColumnCapabilities capabilities,
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -28,6 +29,8 @@ public interface VectorValueMatcherFactory
|
|||
{
|
||||
VectorValueMatcher makeMatcher(@Nullable String value);
|
||||
|
||||
VectorValueMatcher makeMatcher(Object value, ColumnType type);
|
||||
|
||||
VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory);
|
||||
|
||||
default VectorValueMatcher makeNullValueMatcher(VectorValueSelector selector)
|
||||
|
|
|
@ -106,6 +106,14 @@ public class GroupByVectorColumnProcessorFactory implements VectorColumnProcesso
|
|||
return new NullableLongGroupByVectorColumnSelector(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GroupByVectorColumnSelector makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector)
|
||||
{
|
||||
throw new UnsupportedOperationException(
|
||||
"Vectorized groupBys on ARRAY columns are not yet implemented"
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GroupByVectorColumnSelector makeObjectProcessor(
|
||||
final ColumnCapabilities capabilities,
|
||||
|
@ -113,6 +121,11 @@ public class GroupByVectorColumnProcessorFactory implements VectorColumnProcesso
|
|||
)
|
||||
{
|
||||
if (capabilities.is(ValueType.STRING)) {
|
||||
if (capabilities.hasMultipleValues().isTrue()) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Vectorized groupBys on multi-value dictionary-encoded dimensions are not yet implemented"
|
||||
);
|
||||
}
|
||||
return new DictionaryBuildingSingleValueStringGroupByVectorColumnSelector(selector);
|
||||
}
|
||||
return NilGroupByVectorColumnSelector.INSTANCE;
|
||||
|
|
|
@ -44,12 +44,12 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.ColumnTypeFactory;
|
||||
import org.apache.druid.segment.column.ComplexColumn;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.joda.time.DateTime;
|
||||
|
|
|
@ -526,7 +526,6 @@ public class DefaultFramedOnHeapAggregatable implements FramedOnHeapAggregatable
|
|||
.setDictionaryEncoded(false)
|
||||
.setHasMultipleValues(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setFilterable(false)
|
||||
.setType(factory.getIntermediateType());
|
||||
}
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -39,12 +39,12 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.NumericColumn;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.virtual.VirtualizedColumnInspector;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.data.input.impl.DimensionSchema;
|
|||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
|
@ -223,10 +224,13 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
)
|
||||
{
|
||||
final int dimIndex = desc.getIndex();
|
||||
if (fieldIndexers.size() == 0 && isConstant && !hasNestedData) {
|
||||
return DimensionSelector.constant(null, spec.getExtractionFn());
|
||||
}
|
||||
final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
|
||||
if (rootLiteralSelector != null) {
|
||||
final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
final ColumnType rootType = root.getTypes().getSingleType();
|
||||
final ColumnType rootType = root.isSingleType() ? root.getTypes().getSingleType() : getLogicalType();
|
||||
if (rootType.isArray()) {
|
||||
throw new UOE(
|
||||
"makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector",
|
||||
|
@ -234,17 +238,31 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
rootType
|
||||
);
|
||||
}
|
||||
if (spec.getExtractionFn() == null) {
|
||||
return new BaseSingleValueDimensionSelector()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
protected String getValue()
|
||||
{
|
||||
return Evals.asString(rootLiteralSelector.getObject());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
return new BaseSingleValueDimensionSelector()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
protected String getValue()
|
||||
{
|
||||
final Object o = rootLiteralSelector.getObject();
|
||||
if (o == null) {
|
||||
return null;
|
||||
}
|
||||
return o.toString();
|
||||
final String s = Evals.asString(rootLiteralSelector.getObject());
|
||||
return spec.getExtractionFn().apply(s);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -457,14 +475,14 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
int dimIndex
|
||||
)
|
||||
{
|
||||
if (fieldIndexers.size() > 1) {
|
||||
if (fieldIndexers.size() > 1 || hasNestedData) {
|
||||
return null;
|
||||
}
|
||||
final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
if (root == null || !root.isSingleType()) {
|
||||
if (root == null) {
|
||||
return null;
|
||||
}
|
||||
final Object defaultValue = getDefaultValueForType(root.getTypes().getSingleType());
|
||||
final Object defaultValue = getDefaultValueForType(getLogicalType());
|
||||
return new ColumnValueSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -20,8 +20,11 @@
|
|||
package org.apache.druid.segment;
|
||||
|
||||
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Class that encapsulates knowledge about how to create "column processors", which are... objects that process columns
|
||||
* and want to have type-specific logic. Used by {@link ColumnProcessors#makeProcessor}.
|
||||
|
@ -76,6 +79,14 @@ public interface ColumnProcessorFactory<T>
|
|||
*/
|
||||
T makeLongProcessor(BaseLongColumnValueSelector selector);
|
||||
|
||||
/**
|
||||
*
|
||||
* @param selector array selector
|
||||
* @param columnCapabilities information about the underlying column to match. Null here just means the capabilities
|
||||
* are unknown, and not necessarily indicative that the column doesn't exist
|
||||
*/
|
||||
T makeArrayProcessor(BaseObjectColumnValueSelector<?> selector, @Nullable ColumnCapabilities columnCapabilities);
|
||||
|
||||
/**
|
||||
* Create a processor for a complex column.
|
||||
*
|
||||
|
|
|
@ -286,6 +286,11 @@ public class ColumnProcessors
|
|||
return processorFactory.makeFloatProcessor(valueSelectorFunction.apply(selectorFactory));
|
||||
case DOUBLE:
|
||||
return processorFactory.makeDoubleProcessor(valueSelectorFunction.apply(selectorFactory));
|
||||
case ARRAY:
|
||||
return processorFactory.makeArrayProcessor(
|
||||
valueSelectorFunction.apply(selectorFactory),
|
||||
capabilities
|
||||
);
|
||||
case COMPLEX:
|
||||
return processorFactory.makeComplexProcessor(valueSelectorFunction.apply(selectorFactory));
|
||||
default:
|
||||
|
@ -359,6 +364,8 @@ public class ColumnProcessors
|
|||
return processorFactory.makeFloatProcessor(capabilities, valueSelectorFn.apply(selectorFactory));
|
||||
case DOUBLE:
|
||||
return processorFactory.makeDoubleProcessor(capabilities, valueSelectorFn.apply(selectorFactory));
|
||||
case ARRAY:
|
||||
return processorFactory.makeArrayProcessor(capabilities, objectSelectorFn.apply(selectorFactory));
|
||||
case COMPLEX:
|
||||
return processorFactory.makeObjectProcessor(capabilities, objectSelectorFn.apply(selectorFactory));
|
||||
default:
|
||||
|
|
|
@ -80,10 +80,9 @@ public class ColumnSelectorColumnIndexSelector implements ColumnIndexSelector
|
|||
indexSupplier = virtualColumns.getIndexSupplier(column, columnSelector);
|
||||
} else {
|
||||
final ColumnHolder columnHolder = columnSelector.getColumnHolder(column);
|
||||
// for missing columns and columns with types that do not support filtering,
|
||||
// treat the column as if it were full of nulls. This allows callers to fabricate an 'all true' or 'all false'
|
||||
// for missing columns we return null here. This allows callers to fabricate an 'all true' or 'all false'
|
||||
// index so that filters which match the values can still use "indexes".
|
||||
if (columnHolder == null || !columnHolder.getCapabilities().isFilterable()) {
|
||||
if (columnHolder == null) {
|
||||
return null;
|
||||
}
|
||||
indexSupplier = columnHolder.getIndexSupplier();
|
||||
|
|
|
@ -24,9 +24,9 @@ import org.apache.druid.query.BitmapResultFactory;
|
|||
import org.apache.druid.query.DefaultBitmapResultFactory;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.filter.AndFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
|
|
|
@ -27,10 +27,10 @@ import org.apache.druid.query.filter.Filter;
|
|||
import org.apache.druid.query.filter.RowOffsetMatcherFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.data.Offset;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
|
||||
public final class FilteredOffset extends Offset
|
||||
|
|
|
@ -30,11 +30,11 @@ import org.apache.druid.segment.column.ColumnHolder;
|
|||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedValueIndex;
|
||||
import org.apache.druid.segment.data.BitmapValues;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.apache.druid.segment.data.ImmutableBitmapValues;
|
||||
import org.apache.druid.segment.data.IndexedIterable;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex;
|
||||
import org.apache.druid.segment.nested.NestedCommonFormatColumn;
|
||||
import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
|
||||
import org.apache.druid.segment.nested.SortedValueDictionary;
|
||||
|
|
|
@ -33,9 +33,9 @@ import org.apache.druid.segment.column.ColumnCapabilities;
|
|||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.NumericColumn;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.vector.VectorCursor;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
|
|
@ -83,6 +83,8 @@ public interface VectorColumnProcessorFactory<T>
|
|||
*/
|
||||
T makeLongProcessor(ColumnCapabilities capabilities, VectorValueSelector selector);
|
||||
|
||||
T makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector);
|
||||
|
||||
/**
|
||||
* Called when {@link ColumnCapabilities#getType()} is COMPLEX. May also be called for STRING typed columns in
|
||||
* cases where the dictionary does not exist or is not expected to be useful.
|
||||
|
|
|
@ -136,7 +136,6 @@ public class CapabilitiesBasedFormat implements ColumnFormat
|
|||
if (merged.hasSpatialIndexes() != otherSnapshot.hasSpatialIndexes()) {
|
||||
merged.setHasSpatialIndexes(merged.hasSpatialIndexes() || otherSnapshot.hasSpatialIndexes());
|
||||
}
|
||||
merged.setFilterable(merged.isFilterable() && otherSnapshot.isFilterable());
|
||||
|
||||
return new CapabilitiesBasedFormat(merged);
|
||||
}
|
||||
|
|
|
@ -96,13 +96,6 @@ public class ColumnBuilder
|
|||
return this;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public ColumnBuilder setFilterable(boolean filterable)
|
||||
{
|
||||
this.capabilitiesBuilder.setFilterable(filterable);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setComplexColumnSupplier(Supplier<? extends ComplexColumn> columnSupplier)
|
||||
{
|
||||
checkColumnSupplierNotSet();
|
||||
|
|
|
@ -74,12 +74,6 @@ public interface ColumnCapabilities extends TypeSignature<ValueType>
|
|||
*/
|
||||
boolean hasSpatialIndexes();
|
||||
|
||||
/**
|
||||
* All Druid primitive columns support filtering, maybe with or without indexes, but by default complex columns
|
||||
* do not support direct filtering, unless provided by through a custom implementation.
|
||||
*/
|
||||
boolean isFilterable();
|
||||
|
||||
/**
|
||||
* Does this column contain null values? If so, callers, especially for primitive numeric columns, will need to check
|
||||
* for null value rows and act accordingly
|
||||
|
|
|
@ -79,7 +79,6 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
capabilities.dictionaryValuesSorted = other.areDictionaryValuesSorted();
|
||||
capabilities.dictionaryValuesUnique = other.areDictionaryValuesUnique();
|
||||
capabilities.hasNulls = other.hasNulls();
|
||||
capabilities.filterable = other.isFilterable();
|
||||
}
|
||||
return capabilities;
|
||||
}
|
||||
|
@ -179,8 +178,6 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
@JsonIgnore
|
||||
private Capable dictionaryValuesUnique = Capable.UNKNOWN;
|
||||
@JsonIgnore
|
||||
private boolean filterable;
|
||||
@JsonIgnore
|
||||
private Capable hasNulls = Capable.UNKNOWN;
|
||||
|
||||
@Nullable
|
||||
|
@ -314,16 +311,4 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
this.hasNulls = hasNulls;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFilterable()
|
||||
{
|
||||
return (type != null && (isPrimitive() || isArray())) || filterable;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setFilterable(boolean filterable)
|
||||
{
|
||||
this.filterable = filterable;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,16 @@
|
|||
|
||||
package org.apache.druid.segment.column;
|
||||
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes;
|
||||
import org.apache.druid.segment.index.semantic.NumericRangeIndexes;
|
||||
|
||||
public interface ColumnConfig
|
||||
{
|
||||
ColumnConfig DEFAULT = new ColumnConfig() {};
|
||||
ColumnConfig DEFAULT = new ColumnConfig()
|
||||
{
|
||||
};
|
||||
|
||||
ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig()
|
||||
{
|
||||
|
@ -41,7 +48,7 @@ public interface ColumnConfig
|
|||
|
||||
/**
|
||||
* If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap
|
||||
* index operations required to perform to use a {@link LexicographicalRangeIndex} or {@link NumericRangeIndex},
|
||||
* index operations required to perform to use {@link LexicographicalRangeIndexes} or {@link NumericRangeIndexes},
|
||||
* then for any {@link ColumnIndexSupplier} which chooses to participate in this config it will skip computing the
|
||||
* index, indicated by a return value of null from the 'forRange' methods, to force the filter to be processed
|
||||
* with a scan using a {@link org.apache.druid.query.filter.ValueMatcher} instead.
|
||||
|
@ -74,7 +81,7 @@ public interface ColumnConfig
|
|||
|
||||
/**
|
||||
* If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap
|
||||
* index operations required to perform to use a {@link DruidPredicateIndex} then for any {@link ColumnIndexSupplier}
|
||||
* index operations required to perform to use {@link DruidPredicateIndexes} then for any {@link ColumnIndexSupplier}
|
||||
* which chooses to participate in this config it will skip computing the index, in favor of doing a full scan and
|
||||
* using a {@link org.apache.druid.query.filter.ValueMatcher} instead. This is indicated returning null from
|
||||
* {@link ColumnIndexSupplier#as(Class)} even though it would have otherwise been able to create a
|
||||
|
|
|
@ -39,9 +39,9 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
|||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -44,12 +44,15 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.LexicographicalRangeIndex;
|
||||
import org.apache.druid.segment.column.NullValueIndex;
|
||||
import org.apache.druid.segment.column.NumericRangeIndex;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes;
|
||||
import org.apache.druid.segment.index.semantic.NullValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.NumericRangeIndexes;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -77,20 +80,54 @@ public class BoundFilter implements Filter
|
|||
if (!Filters.checkFilterTuningUseIndex(boundDimFilter.getDimension(), selector, filterTuning)) {
|
||||
return null;
|
||||
}
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension());
|
||||
if (indexSupplier == null) {
|
||||
// missing column -> match all rows if the predicate matches null; match no rows otherwise
|
||||
return getPredicateFactory().makeStringPredicate().apply(null)
|
||||
? new AllTrueBitmapColumnIndex(selector)
|
||||
: new AllFalseBitmapColumnIndex(selector);
|
||||
}
|
||||
|
||||
if (supportStringShortCircuit()) {
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension());
|
||||
if (indexSupplier == null) {
|
||||
return Filters.makeNullIndex(doesMatchNull(), selector);
|
||||
}
|
||||
final LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class);
|
||||
if (rangeIndex != null) {
|
||||
final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange(
|
||||
final LexicographicalRangeIndexes rangeIndexes = indexSupplier.as(LexicographicalRangeIndexes.class);
|
||||
if (rangeIndexes != null) {
|
||||
final BitmapColumnIndex rangeBitmaps = rangeIndexes.forRange(
|
||||
boundDimFilter.getLower(),
|
||||
boundDimFilter.isLowerStrict(),
|
||||
boundDimFilter.getUpper(),
|
||||
boundDimFilter.isUpperStrict()
|
||||
);
|
||||
if (rangeBitmaps != null) {
|
||||
if (rangeBitmaps == null) {
|
||||
return null;
|
||||
}
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (doesMatchNull()) {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
} else {
|
||||
return rangeBitmaps;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (supportNumericShortCircuit()) {
|
||||
final NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class);
|
||||
if (rangeIndexes != null) {
|
||||
final Number lower = boundDimFilter.hasLowerBound() ? Doubles.tryParse(boundDimFilter.getLower()) : null;
|
||||
final Number upper = boundDimFilter.hasUpperBound() ? Doubles.tryParse(boundDimFilter.getUpper()) : null;
|
||||
// valid number bounds are required to use the range index, otherwise we need to fall back to the predicate
|
||||
// index to get consistent behavior with the value matcher. in a better world this might be a much earlier
|
||||
// validation error, but.. the bound filter doesn't live in that world
|
||||
final boolean lowerValid = !(boundDimFilter.hasLowerBound() && lower == null);
|
||||
final boolean upperValid = !(boundDimFilter.hasUpperBound() && upper == null);
|
||||
if (lowerValid && upperValid) {
|
||||
final BitmapColumnIndex rangeBitmaps = rangeIndexes.forRange(
|
||||
lower,
|
||||
boundDimFilter.isLowerStrict(),
|
||||
upper,
|
||||
boundDimFilter.isUpperStrict()
|
||||
);
|
||||
if (rangeBitmaps == null) {
|
||||
return null;
|
||||
}
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (doesMatchNull()) {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
|
@ -100,41 +137,13 @@ public class BoundFilter implements Filter
|
|||
}
|
||||
}
|
||||
}
|
||||
if (supportNumericShortCircuit()) {
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension());
|
||||
if (indexSupplier == null) {
|
||||
return Filters.makeNullIndex(doesMatchNull(), selector);
|
||||
}
|
||||
final NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
|
||||
if (rangeIndex != null) {
|
||||
final Number lower = boundDimFilter.hasLowerBound() ? Doubles.tryParse(boundDimFilter.getLower()) : null;
|
||||
final Number upper = boundDimFilter.hasUpperBound() ? Doubles.tryParse(boundDimFilter.getUpper()) : null;
|
||||
// valid number bounds are required to use the range index, otherwise we need to fall back to the predicate
|
||||
// index to get consistent behavior with the value matcher. in a better world this might be a much earlier
|
||||
// validation error, but.. the bound filter doesn't live in that world
|
||||
final boolean lowerValid = !(boundDimFilter.hasLowerBound() && lower == null);
|
||||
final boolean upperValid = !(boundDimFilter.hasUpperBound() && upper == null);
|
||||
if (lowerValid && upperValid) {
|
||||
final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange(
|
||||
lower,
|
||||
boundDimFilter.isLowerStrict(),
|
||||
upper,
|
||||
boundDimFilter.isUpperStrict()
|
||||
);
|
||||
if (rangeBitmaps != null) {
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (doesMatchNull()) {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
} else {
|
||||
return rangeBitmaps;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// fall back to predicate based index if it is available
|
||||
return Filters.makePredicateIndex(boundDimFilter.getDimension(), selector, getPredicateFactory());
|
||||
final DruidPredicateIndexes predicateIndexes = indexSupplier.as(DruidPredicateIndexes.class);
|
||||
if (predicateIndexes != null) {
|
||||
return predicateIndexes.forPredicate(getPredicateFactory());
|
||||
}
|
||||
// index doesn't exist
|
||||
return null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -150,7 +159,7 @@ public class BoundFilter implements Filter
|
|||
if (nulls == null) {
|
||||
return null;
|
||||
}
|
||||
nullBitmap = nulls.forNull();
|
||||
nullBitmap = nulls.get();
|
||||
|
||||
return new BitmapColumnIndex()
|
||||
{
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
|
@ -34,9 +35,10 @@ import org.apache.druid.segment.ColumnProcessors;
|
|||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
|
@ -233,6 +235,26 @@ public class ColumnComparisonFilter implements Filter
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeArrayProcessor(
|
||||
BaseObjectColumnValueSelector<?> selector,
|
||||
@Nullable ColumnCapabilities columnCapabilities
|
||||
)
|
||||
{
|
||||
return () -> {
|
||||
final Object o = selector.getObject();
|
||||
if (o instanceof Object[]) {
|
||||
final Object[] arr = (Object[]) o;
|
||||
final String[] s = new String[arr.length];
|
||||
for (int i = 0; i < arr.length; i++) {
|
||||
s[i] = Evals.asString(arr[i]);
|
||||
}
|
||||
return s;
|
||||
}
|
||||
return NULL_VALUE;
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -46,10 +46,12 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.virtual.ExpressionSelectors;
|
||||
import org.apache.druid.segment.virtual.ExpressionVectorSelectors;
|
||||
|
@ -122,6 +124,11 @@ public class ExpressionFilter implements Filter
|
|||
ColumnCapabilitiesImpl.createSimpleSingleValueStringColumnCapabilities(),
|
||||
ExpressionVectorSelectors.makeVectorObjectSelector(factory, theExpr)
|
||||
).makeMatcher(predicateFactory);
|
||||
case ARRAY:
|
||||
return VectorValueMatcherColumnProcessorFactory.instance().makeObjectProcessor(
|
||||
ColumnCapabilitiesImpl.createDefault().setType(ExpressionType.toColumnType(outputType)).setHasNulls(true),
|
||||
ExpressionVectorSelectors.makeVectorObjectSelector(factory, theExpr)
|
||||
).makeMatcher(predicateFactory);
|
||||
default:
|
||||
if (ExpressionType.NESTED_DATA.equals(outputType)) {
|
||||
return VectorValueMatcherColumnProcessorFactory.instance().makeObjectProcessor(
|
||||
|
@ -187,7 +194,7 @@ public class ExpressionFilter implements Filter
|
|||
final Expr.BindingAnalysis details = bindingDetails.get();
|
||||
if (details.getRequiredBindings().isEmpty()) {
|
||||
// Constant expression.
|
||||
return Filters.makeNullIndex(
|
||||
return Filters.makeMissingColumnNullIndex(
|
||||
expr.get().eval(InputBindings.nilBindings()).asBoolean(),
|
||||
selector
|
||||
);
|
||||
|
@ -199,10 +206,7 @@ public class ExpressionFilter implements Filter
|
|||
// we use a default 'all false' capabilities here because if the column has a bitmap index, but the capabilities
|
||||
// are null, it means that the column is missing and should take the single valued path, while truly unknown
|
||||
// things will not have a bitmap index available
|
||||
final ColumnCapabilities capabilities = selector.getColumnCapabilitiesWithDefault(
|
||||
column,
|
||||
ColumnCapabilitiesImpl.createDefault()
|
||||
);
|
||||
final ColumnCapabilities capabilities = selector.getColumnCapabilities(column);
|
||||
if (ExpressionSelectors.canMapOverDictionary(details, capabilities)) {
|
||||
if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) {
|
||||
return null;
|
||||
|
@ -210,7 +214,7 @@ public class ExpressionFilter implements Filter
|
|||
return Filters.makePredicateIndex(
|
||||
column,
|
||||
selector,
|
||||
getBitmapPredicateFactory()
|
||||
getBitmapPredicateFactory(capabilities)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -338,7 +342,7 @@ public class ExpressionFilter implements Filter
|
|||
* {@link DruidPredicateFactory} which evaluates the expression using the value as input, used for building predicate
|
||||
* indexes where the raw column values will be checked against this predicate
|
||||
*/
|
||||
private DruidPredicateFactory getBitmapPredicateFactory()
|
||||
private DruidPredicateFactory getBitmapPredicateFactory(@Nullable ColumnCapabilities inputCapabilites)
|
||||
{
|
||||
return new DruidPredicateFactory()
|
||||
{
|
||||
|
@ -414,6 +418,19 @@ public class ExpressionFilter implements Filter
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
if (inputCapabilites == null) {
|
||||
return input -> expr.get()
|
||||
.eval(InputBindings.forInputSupplier(ExpressionType.STRING_ARRAY, () -> input))
|
||||
.asBoolean();
|
||||
}
|
||||
return input -> expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.fromColumnType(inputCapabilites), () -> input)
|
||||
).asBoolean();
|
||||
}
|
||||
|
||||
// The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
|
||||
// DruidPredicateFactory currently doesn't really need equals or hashcode since 'toString' method that is actually
|
||||
// called when testing equality of DimensionPredicateFilter, so it's the truly required method, but that seems
|
||||
|
|
|
@ -27,8 +27,8 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
|||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -35,16 +35,16 @@ import org.apache.druid.query.filter.ValueMatcher;
|
|||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.DruidPredicateIndex;
|
||||
import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException;
|
||||
import org.apache.druid.segment.filter.cnf.CalciteCnfHelper;
|
||||
import org.apache.druid.segment.filter.cnf.HiveCnfHelper;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -90,29 +90,6 @@ public class Filters
|
|||
return dimFilter == null ? null : dimFilter.toOptimizedFilter();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a ValueMatcher that compares row values to the provided string.
|
||||
* <p>
|
||||
* An implementation of this method should be able to handle dimensions of various types.
|
||||
*
|
||||
* @param columnSelectorFactory Selector for columns.
|
||||
* @param columnName The column to filter.
|
||||
* @param value The value to match against, represented as a String.
|
||||
*
|
||||
* @return An object that matches row values on the provided value.
|
||||
*/
|
||||
public static ValueMatcher makeValueMatcher(
|
||||
final ColumnSelectorFactory columnSelectorFactory,
|
||||
final String columnName,
|
||||
final String value
|
||||
)
|
||||
{
|
||||
return ColumnProcessors.makeProcessor(
|
||||
columnName,
|
||||
new ConstantValueMatcherFactory(value),
|
||||
columnSelectorFactory
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a ValueMatcher that applies a predicate to row values.
|
||||
|
@ -154,9 +131,9 @@ public class Filters
|
|||
Preconditions.checkNotNull(predicateFactory, "predicateFactory");
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column);
|
||||
if (indexSupplier != null) {
|
||||
final DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
|
||||
if (predicateIndex != null) {
|
||||
return predicateIndex.forPredicate(predicateFactory);
|
||||
final DruidPredicateIndexes predicateIndexes = indexSupplier.as(DruidPredicateIndexes.class);
|
||||
if (predicateIndexes != null) {
|
||||
return predicateIndexes.forPredicate(predicateFactory);
|
||||
}
|
||||
// index doesn't exist
|
||||
return null;
|
||||
|
@ -167,7 +144,7 @@ public class Filters
|
|||
: new AllFalseBitmapColumnIndex(selector);
|
||||
}
|
||||
|
||||
public static BitmapColumnIndex makeNullIndex(boolean matchesNull, final ColumnIndexSelector selector)
|
||||
public static BitmapColumnIndex makeMissingColumnNullIndex(boolean matchesNull, final ColumnIndexSelector selector)
|
||||
{
|
||||
return matchesNull ? new AllTrueBitmapColumnIndex(selector) : new AllFalseBitmapColumnIndex(selector);
|
||||
}
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.druid.query.filter.JavaScriptDimFilter;
|
|||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
|
|
@ -34,12 +34,12 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.LexicographicalRangeIndex;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -82,23 +82,23 @@ public class LikeFilter implements Filter
|
|||
: new AllFalseBitmapColumnIndex(selector);
|
||||
}
|
||||
if (isSimpleEquals()) {
|
||||
StringValueSetIndex valueIndex = indexSupplier.as(StringValueSetIndex.class);
|
||||
if (valueIndex != null) {
|
||||
return valueIndex.forValue(
|
||||
StringValueSetIndexes valueIndexes = indexSupplier.as(StringValueSetIndexes.class);
|
||||
if (valueIndexes != null) {
|
||||
return valueIndexes.forValue(
|
||||
NullHandling.emptyToNullIfNeeded(likeMatcher.getPrefix())
|
||||
);
|
||||
}
|
||||
}
|
||||
if (isSimplePrefix()) {
|
||||
final LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class);
|
||||
if (rangeIndex != null) {
|
||||
final LexicographicalRangeIndexes rangeIndexes = indexSupplier.as(LexicographicalRangeIndexes.class);
|
||||
if (rangeIndexes != null) {
|
||||
final String lower = NullHandling.nullToEmptyIfNeeded(likeMatcher.getPrefix());
|
||||
final String upper = NullHandling.nullToEmptyIfNeeded(likeMatcher.getPrefix()) + Character.MAX_VALUE;
|
||||
|
||||
if (likeMatcher.getSuffixMatch() == LikeDimFilter.LikeMatcher.SuffixMatch.MATCH_ANY) {
|
||||
return rangeIndex.forRange(lower, false, upper, false);
|
||||
return rangeIndexes.forRange(lower, false, upper, false);
|
||||
} else {
|
||||
return rangeIndex.forRange(lower, false, upper, false, likeMatcher::matchesSuffixOnly);
|
||||
return rangeIndexes.forRange(lower, false, upper, false, likeMatcher::matchesSuffixOnly);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,8 +32,8 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
|||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -39,9 +39,9 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
|||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.segment.filter;
|
|||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.Rows;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.query.filter.DruidDoublePredicate;
|
||||
import org.apache.druid.query.filter.DruidFloatPredicate;
|
||||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
|
@ -35,8 +36,10 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
|||
import org.apache.druid.segment.ColumnProcessorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.NilColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -46,7 +49,7 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
{
|
||||
private final DruidPredicateFactory predicateFactory;
|
||||
|
||||
PredicateValueMatcherFactory(DruidPredicateFactory predicateFactory)
|
||||
public PredicateValueMatcherFactory(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
this.predicateFactory = predicateFactory;
|
||||
}
|
||||
|
@ -83,6 +86,45 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
return ValueMatchers.makeLongValueMatcher(selector, predicateFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeArrayProcessor(
|
||||
BaseObjectColumnValueSelector<?> selector,
|
||||
@Nullable ColumnCapabilities columnCapabilities
|
||||
)
|
||||
{
|
||||
if (selector instanceof NilColumnValueSelector) {
|
||||
// Column does not exist, or is unfilterable. Treat it as all nulls.
|
||||
return BooleanValueMatcher.of(predicateFactory.makeArrayPredicate(columnCapabilities).apply(null));
|
||||
} else {
|
||||
// use the object predicate
|
||||
final Predicate<Object[]> predicate = predicateFactory.makeArrayPredicate(columnCapabilities);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
Object o = selector.getObject();
|
||||
if (o == null || o instanceof Object[]) {
|
||||
return predicate.apply((Object[]) o);
|
||||
}
|
||||
if (o instanceof List) {
|
||||
ExprEval<?> oEval = ExprEval.bestEffortArray((List<?>) o);
|
||||
return predicate.apply(oEval.asArray());
|
||||
}
|
||||
// upcast non-array to a single element array to behave consistently with expressions.. idk if this is cool
|
||||
return predicate.apply(new Object[]{o});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
|
@ -117,6 +159,7 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
private DruidLongPredicate longPredicate;
|
||||
private DruidFloatPredicate floatPredicate;
|
||||
private DruidDoublePredicate doublePredicate;
|
||||
private Predicate<Object[]> arrayPredicate;
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
|
@ -134,6 +177,8 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
} else if (rowValue instanceof Number) {
|
||||
// Double or some other non-int, non-long, non-float number.
|
||||
return getDoublePredicate().applyDouble((double) rowValue);
|
||||
} else if (rowValue instanceof Object[]) {
|
||||
return getArrayPredicate().apply((Object[]) rowValue);
|
||||
} else {
|
||||
// Other types. Cast to list of strings and evaluate them as strings.
|
||||
// Boolean values are handled here as well since it is not a known type in Druid.
|
||||
|
@ -196,6 +241,14 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
|
||||
return doublePredicate;
|
||||
}
|
||||
|
||||
private Predicate<Object[]> getArrayPredicate()
|
||||
{
|
||||
if (arrayPredicate == null) {
|
||||
arrayPredicate = predicateFactory.makeArrayPredicate(null);
|
||||
}
|
||||
return arrayPredicate;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,10 +33,10 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.NullValueIndex;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.NullValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -87,28 +87,32 @@ public class SelectorFilter implements Filter
|
|||
final boolean isNull = NullHandling.isNullOrEquivalent(value);
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(dimension);
|
||||
if (indexSupplier == null) {
|
||||
return Filters.makeNullIndex(isNull, selector);
|
||||
return Filters.makeMissingColumnNullIndex(isNull, selector);
|
||||
}
|
||||
if (isNull) {
|
||||
final NullValueIndex nullValueIndex = indexSupplier.as(NullValueIndex.class);
|
||||
if (nullValueIndex == null) {
|
||||
return null;
|
||||
}
|
||||
return nullValueIndex.forNull();
|
||||
return nullValueIndex.get();
|
||||
} else {
|
||||
final StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
|
||||
if (valueSetIndex == null) {
|
||||
final StringValueSetIndexes valueSetIndexes = indexSupplier.as(StringValueSetIndexes.class);
|
||||
if (valueSetIndexes == null) {
|
||||
// column exists, but has no index
|
||||
return null;
|
||||
}
|
||||
return valueSetIndex.forValue(value);
|
||||
return valueSetIndexes.forValue(value);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||
{
|
||||
return Filters.makeValueMatcher(factory, dimension, value);
|
||||
return ColumnProcessors.makeProcessor(
|
||||
dimension,
|
||||
new StringConstantValueMatcherFactory(value),
|
||||
factory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -36,13 +36,13 @@ import org.apache.druid.query.filter.FilterTuning;
|
|||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SpatialIndex;
|
||||
import org.apache.druid.segment.incremental.SpatialDimensionRowTransformer;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.SpatialIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.druid.segment.BaseLongColumnValueSelector;
|
|||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnProcessorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -35,12 +36,12 @@ import javax.annotation.Nullable;
|
|||
/**
|
||||
* Creates {@link ValueMatcher} that match constants.
|
||||
*/
|
||||
public class ConstantValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
|
||||
public class StringConstantValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
|
||||
{
|
||||
@Nullable
|
||||
private final String matchValue;
|
||||
|
||||
ConstantValueMatcherFactory(@Nullable String matchValue)
|
||||
StringConstantValueMatcherFactory(@Nullable String matchValue)
|
||||
{
|
||||
this.matchValue = NullHandling.emptyToNullIfNeeded(matchValue);
|
||||
}
|
||||
|
@ -76,6 +77,18 @@ public class ConstantValueMatcherFactory implements ColumnProcessorFactory<Value
|
|||
return ValueMatchers.makeLongValueMatcher(selector, matchValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeArrayProcessor(
|
||||
BaseObjectColumnValueSelector<?> selector,
|
||||
@Nullable ColumnCapabilities columnCapabilities
|
||||
)
|
||||
{
|
||||
// this is gonna fail because SelectorPredicateFactory does not implement array predicate...
|
||||
return new PredicateValueMatcherFactory(
|
||||
new SelectorPredicateFactory(matchValue)
|
||||
).makeArrayProcessor(selector, columnCapabilities);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
|
@ -27,8 +27,8 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
|||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -39,7 +39,7 @@ import javax.annotation.Nullable;
|
|||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link ConstantValueMatcherFactory}
|
||||
* Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link StringConstantValueMatcherFactory}
|
||||
* and {@link PredicateValueMatcherFactory}.
|
||||
*/
|
||||
public class ValueMatchers
|
||||
|
@ -114,8 +114,22 @@ public class ValueMatchers
|
|||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
return makeFloatValueMatcher(selector, matchVal);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a float-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
*/
|
||||
public static ValueMatcher makeFloatValueMatcher(
|
||||
final BaseFloatColumnValueSelector selector,
|
||||
final float value
|
||||
)
|
||||
{
|
||||
// Use "floatToIntBits" to canonicalize NaN values.
|
||||
final int matchValIntBits = Float.floatToIntBits(matchVal);
|
||||
final int matchValIntBits = Float.floatToIntBits(value);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
|
@ -141,7 +155,11 @@ public class ValueMatchers
|
|||
if (matchVal == null) {
|
||||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
final long matchValLong = matchVal;
|
||||
return makeLongValueMatcher(selector, matchVal);
|
||||
}
|
||||
|
||||
public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, long value)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
|
@ -150,7 +168,7 @@ public class ValueMatchers
|
|||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return selector.getLong() == matchValLong;
|
||||
return selector.getLong() == value;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -187,6 +205,7 @@ public class ValueMatchers
|
|||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a predicate-based {@link ValueMatcher} for a float-typed selector.
|
||||
*
|
||||
|
@ -235,8 +254,16 @@ public class ValueMatchers
|
|||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
return makeDoubleValueMatcher(selector, matchVal);
|
||||
}
|
||||
|
||||
public static ValueMatcher makeDoubleValueMatcher(
|
||||
final BaseDoubleColumnValueSelector selector,
|
||||
final double value
|
||||
)
|
||||
{
|
||||
// Use "doubleToLongBits" to canonicalize NaN values.
|
||||
final long matchValLongBits = Double.doubleToLongBits(matchVal);
|
||||
final long matchValLongBits = Double.doubleToLongBits(value);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -17,10 +17,12 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
|
||||
public class AllFalseBitmapColumnIndex implements BitmapColumnIndex
|
||||
{
|
|
@ -17,10 +17,12 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
|
||||
public class AllTrueBitmapColumnIndex implements BitmapColumnIndex
|
||||
{
|
|
@ -17,9 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
|
||||
public interface BitmapColumnIndex
|
||||
{
|
|
@ -17,11 +17,12 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
@ -17,19 +17,23 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
public final class IndexedStringDruidPredicateIndex<TDictionary extends Indexed<String>> implements DruidPredicateIndex
|
||||
public final class IndexedStringDruidPredicateIndexes<TDictionary extends Indexed<String>> implements
|
||||
DruidPredicateIndexes
|
||||
{
|
||||
private final BitmapFactory bitmapFactory;
|
||||
private final TDictionary dictionary;
|
||||
|
@ -37,7 +41,7 @@ public final class IndexedStringDruidPredicateIndex<TDictionary extends Indexed<
|
|||
private final ColumnConfig columnConfig;
|
||||
private final int numRows;
|
||||
|
||||
public IndexedStringDruidPredicateIndex(
|
||||
public IndexedStringDruidPredicateIndexes(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
Indexed<ImmutableBitmap> bitmaps,
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
|
@ -29,15 +29,18 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.IntListUtils;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
public final class IndexedUtf8LexicographicalRangeIndex<TDictionary extends Indexed<ByteBuffer>>
|
||||
implements LexicographicalRangeIndex
|
||||
public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends Indexed<ByteBuffer>>
|
||||
implements LexicographicalRangeIndexes
|
||||
{
|
||||
private final BitmapFactory bitmapFactory;
|
||||
private final TDictionary dictionary;
|
||||
|
@ -47,7 +50,7 @@ public final class IndexedUtf8LexicographicalRangeIndex<TDictionary extends Inde
|
|||
private final ColumnConfig columnConfig;
|
||||
private final int numRows;
|
||||
|
||||
public IndexedUtf8LexicographicalRangeIndex(
|
||||
public IndexedUtf8LexicographicalRangeIndexes(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
Indexed<ImmutableBitmap> bitmaps,
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
@ -27,8 +27,15 @@ import org.apache.druid.collections.bitmap.BitmapFactory;
|
|||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.java.util.common.ByteBufferUtils;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes;
|
||||
import org.apache.druid.segment.index.semantic.ValueIndexes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -37,8 +44,8 @@ import java.util.Iterator;
|
|||
import java.util.NoSuchElementException;
|
||||
import java.util.SortedSet;
|
||||
|
||||
public final class IndexedUtf8ValueSetIndex<TDictionary extends Indexed<ByteBuffer>>
|
||||
implements StringValueSetIndex, Utf8ValueSetIndex
|
||||
public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffer>>
|
||||
implements StringValueSetIndexes, Utf8ValueSetIndexes, ValueIndexes
|
||||
{
|
||||
// This determines the cut-off point to switch the merging algorithm from doing binary-search per element in the value
|
||||
// set to doing a sorted merge algorithm between value set and dictionary. The ratio here represents the ratio b/w
|
||||
|
@ -53,7 +60,7 @@ public final class IndexedUtf8ValueSetIndex<TDictionary extends Indexed<ByteBuff
|
|||
private final TDictionary dictionary;
|
||||
private final Indexed<ImmutableBitmap> bitmaps;
|
||||
|
||||
public IndexedUtf8ValueSetIndex(
|
||||
public IndexedUtf8ValueIndexes(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
Indexed<ImmutableBitmap> bitmaps
|
||||
|
@ -68,6 +75,7 @@ public final class IndexedUtf8ValueSetIndex<TDictionary extends Indexed<ByteBuff
|
|||
@Override
|
||||
public BitmapColumnIndex forValue(@Nullable String value)
|
||||
{
|
||||
final ByteBuffer utf8 = StringUtils.toUtf8ByteBuffer(value);
|
||||
return new SimpleBitmapColumnIndex()
|
||||
{
|
||||
@Override
|
||||
|
@ -85,20 +93,33 @@ public final class IndexedUtf8ValueSetIndex<TDictionary extends Indexed<ByteBuff
|
|||
|
||||
private ImmutableBitmap getBitmapForValue()
|
||||
{
|
||||
final ByteBuffer valueUtf8 = value == null ? null : ByteBuffer.wrap(StringUtils.toUtf8(value));
|
||||
final int idx = dictionary.indexOf(valueUtf8);
|
||||
final int idx = dictionary.indexOf(utf8);
|
||||
return getBitmap(idx);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public BitmapColumnIndex forValue(Object value, TypeSignature<ValueType> valueType)
|
||||
{
|
||||
if (valueType.isPrimitive()) {
|
||||
return forValue(
|
||||
ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value)
|
||||
.castTo(ExpressionType.STRING)
|
||||
.asString()
|
||||
);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return getBitmapColumnIndexForSortedIterableUtf8(
|
||||
Iterables.transform(
|
||||
values,
|
||||
input -> input != null ? ByteBuffer.wrap(StringUtils.toUtf8(input)) : null
|
||||
StringUtils::toUtf8ByteBuffer
|
||||
),
|
||||
values.size()
|
||||
);
|
||||
|
@ -196,6 +217,15 @@ public final class IndexedUtf8ValueSetIndex<TDictionary extends Indexed<ByteBuff
|
|||
|
||||
// if the size of in-filter values is less than the threshold percentage of dictionary size, then use binary search
|
||||
// based lookup per value. The algorithm works well for smaller number of values.
|
||||
return getSimpleImmutableBitmapIterableIndexFromIterator(valuesUtf8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterates over the value set, using binary search to look up each element. The algorithm works well for smaller
|
||||
* number of values, and must be used if the values are not sorted in the same manner as {@link #dictionary}
|
||||
*/
|
||||
private SimpleImmutableBitmapIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable<ByteBuffer> valuesUtf8)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
|
@ -17,7 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
|
||||
/**
|
||||
* {@link BitmapColumnIndex} with Druids "default" {@link ColumnIndexCapabilities}.
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
|
@ -17,9 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index.semantic;
|
||||
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
@ -28,8 +29,8 @@ import javax.annotation.Nullable;
|
|||
* to directly retrieve bitmaps via dictionary ids, as well as access to lower level details of such a column like
|
||||
* value lookup and value cardinality.
|
||||
*
|
||||
* Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndex},
|
||||
* {@link LexicographicalRangeIndex}, {@link NumericRangeIndex}, or {@link DruidPredicateIndex}
|
||||
* Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndexes},
|
||||
* {@link LexicographicalRangeIndexes}, {@link NumericRangeIndexes}, or {@link DruidPredicateIndexes}
|
||||
*/
|
||||
public interface DictionaryEncodedStringValueIndex extends DictionaryEncodedValueIndex
|
||||
{
|
|
@ -17,9 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index.semantic;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
|
||||
/**
|
||||
* This exposes a 'raw' view into bitmap value indexes for {@link DictionaryEncodedColumn}. This allows callers
|
||||
|
@ -29,8 +30,8 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
|||
* either already know what value the dictionary id represents, not care at all, or have some other means to know
|
||||
* exactly which bitmaps to retrieve.
|
||||
*
|
||||
* Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndex},
|
||||
* {@link LexicographicalRangeIndex}, {@link NumericRangeIndex}, or {@link DruidPredicateIndex}.
|
||||
* Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndexes},
|
||||
* {@link LexicographicalRangeIndexes}, {@link NumericRangeIndexes}, or {@link DruidPredicateIndexes}.
|
||||
*/
|
||||
public interface DictionaryEncodedValueIndex
|
||||
{
|
|
@ -17,16 +17,17 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index.semantic;
|
||||
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Uses a {@link DruidPredicateFactory} to construct a {@link BitmapColumnIndex}
|
||||
*/
|
||||
public interface DruidPredicateIndex
|
||||
public interface DruidPredicateIndexes
|
||||
{
|
||||
/**
|
||||
* Get a {@link BitmapColumnIndex} corresponding to all the rows that match the supplied {@link DruidPredicateFactory}
|
|
@ -17,9 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index.semantic;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
@ -28,7 +29,7 @@ import javax.annotation.Nullable;
|
|||
* allowing short-circuit processing of string value ranges. This index does not match null values, union the results
|
||||
* of this index with {@link NullValueIndex} if null values should be considered part of the value range.
|
||||
*/
|
||||
public interface LexicographicalRangeIndex
|
||||
public interface LexicographicalRangeIndexes
|
||||
{
|
||||
/**
|
||||
* Get a {@link BitmapColumnIndex} corresponding to the values supplied in the specified range. If supplied starting
|
|
@ -17,12 +17,14 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.column;
|
||||
package org.apache.druid.segment.index.semantic;
|
||||
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
/**
|
||||
* Provides index for all null rows in a column, to use with IS/IS NOT NULL filters
|
||||
*/
|
||||
public interface NullValueIndex
|
||||
{
|
||||
BitmapColumnIndex forNull();
|
||||
BitmapColumnIndex get();
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue