add native 'array contains element' filter (#15366)

* add native arrayContainsElement filter to use array column element indexes
This commit is contained in:
Clint Wylie 2023-11-29 03:33:00 -08:00 committed by GitHub
parent 0a56c87e93
commit 64fcb32bcf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 2338 additions and 950 deletions

View File

@ -413,7 +413,24 @@ public class SqlBenchmark
"SELECT APPROX_COUNT_DISTINCT_BUILTIN(dimZipf) FROM foo",
"SELECT APPROX_COUNT_DISTINCT_DS_HLL(dimZipf) FROM foo",
"SELECT APPROX_COUNT_DISTINCT_DS_HLL_UTF8(dimZipf) FROM foo",
"SELECT APPROX_COUNT_DISTINCT_DS_THETA(dimZipf) FROM foo"
"SELECT APPROX_COUNT_DISTINCT_DS_THETA(dimZipf) FROM foo",
// 32: LATEST aggregator long
"SELECT LATEST(long1) FROM foo",
// 33: LATEST aggregator double
"SELECT LATEST(double4) FROM foo",
// 34: LATEST aggregator double
"SELECT LATEST(float3) FROM foo",
// 35: LATEST aggregator double
"SELECT LATEST(float3), LATEST(long1), LATEST(double4) FROM foo",
// 36,37: filter numeric nulls
"SELECT SUM(long5) FROM foo WHERE long5 IS NOT NULL",
"SELECT string2, SUM(long5) FROM foo WHERE long5 IS NOT NULL GROUP BY 1",
// 38: EARLIEST aggregator long
"SELECT EARLIEST(long1) FROM foo",
// 39: EARLIEST aggregator double
"SELECT EARLIEST(double4) FROM foo",
// 40: EARLIEST aggregator float
"SELECT EARLIEST(float3) FROM foo"
);
@Param({"5000000"})

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
@ -31,10 +32,12 @@ import org.apache.druid.math.expr.ExpressionProcessing;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.server.security.AuthConfig;
@ -197,23 +200,8 @@ public class SqlExpressionBenchmark
"SELECT TIME_SHIFT(MILLIS_TO_TIMESTAMP(long4), 'PT1H', 1), string2, SUM(long1 * double4) FROM foo GROUP BY 1,2 ORDER BY 3",
// 37: time shift + expr agg (group by), uniform distribution high cardinality
"SELECT TIME_SHIFT(MILLIS_TO_TIMESTAMP(long5), 'PT1H', 1), string2, SUM(long1 * double4) FROM foo GROUP BY 1,2 ORDER BY 3",
// 38: LATEST aggregator long
"SELECT LATEST(long1) FROM foo",
// 39: LATEST aggregator double
"SELECT LATEST(double4) FROM foo",
// 40: LATEST aggregator double
"SELECT LATEST(float3) FROM foo",
// 41: LATEST aggregator double
"SELECT LATEST(float3), LATEST(long1), LATEST(double4) FROM foo",
// 42,43: filter numeric nulls
"SELECT SUM(long5) FROM foo WHERE long5 IS NOT NULL",
"SELECT string2, SUM(long5) FROM foo WHERE long5 IS NOT NULL GROUP BY 1",
// 44: EARLIEST aggregator long
"SELECT EARLIEST(long1) FROM foo",
// 45: EARLIEST aggregator double
"SELECT EARLIEST(double4) FROM foo",
// 46: EARLIEST aggregator float
"SELECT EARLIEST(float3) FROM foo"
// 38: array filtering
"SELECT string1, long1 FROM foo WHERE ARRAY_CONTAINS(\"multi-string3\", 100) GROUP BY 1,2"
);
@Param({"5000000"})
@ -225,6 +213,12 @@ public class SqlExpressionBenchmark
})
private String vectorize;
@Param({
"explicit",
"auto"
})
private String schema;
@Param({
// non-expression reference
"0",
@ -266,16 +260,7 @@ public class SqlExpressionBenchmark
"35",
"36",
"37",
"38",
"39",
"40",
"41",
"42",
"43",
"44",
"45",
"46",
"47"
"38"
})
private String query;
@ -300,8 +285,21 @@ public class SqlExpressionBenchmark
final PlannerConfig plannerConfig = new PlannerConfig();
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
log.info("Starting benchmark setup using cacheDir[%s], rows[%,d], schema[%s].", segmentGenerator.getCacheDir(), rowsPerSegment, schema);
final QueryableIndex index;
if ("auto".equals(schema)) {
index = segmentGenerator.generate(
dataSegment,
schemaInfo,
DimensionsSpec.builder().useSchemaDiscovery(true).build(),
TransformSpec.NONE,
IndexSpec.DEFAULT,
Granularities.NONE,
rowsPerSegment
);
} else {
index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
}
final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(
closer,

View File

@ -34,6 +34,7 @@ import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.InputStats;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.SplitHintSpec;
import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.SplittableInputSource;
import org.apache.druid.guice.IndexingServiceInputSourceModule;
import org.apache.druid.java.util.common.CloseableIterators;
@ -179,7 +180,10 @@ public class GeneratorInputSource extends AbstractInputSource implements Splitta
public InputRow next()
{
rowCount++;
return generator.nextRow();
return MapInputRowParser.parse(
inputRowSchema,
generator.nextRaw(inputRowSchema.getTimestampSpec().getTimestampColumn())
);
}
});
}

View File

@ -25,8 +25,12 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.InputSourceReader;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.IndexingServiceInputSourceModule;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
@ -128,11 +132,20 @@ public class GeneratorInputSourceTest
timestampIncrement
);
InputSourceReader reader = inputSource.fixedFormatReader(null, null);
InputRowSchema rowSchema = new InputRowSchema(
new TimestampSpec(null, null, null),
DimensionsSpec.builder().useSchemaDiscovery(true).build(),
null
);
InputSourceReader reader = inputSource.fixedFormatReader(
rowSchema,
null
);
CloseableIterator<InputRow> iterator = reader.read();
InputRow first = iterator.next();
InputRow generatorFirst = generator.nextRow();
InputRow generatorFirst = MapInputRowParser.parse(rowSchema, generator.nextRaw(rowSchema.getTimestampSpec().getTimestampColumn()));
Assert.assertEquals(generatorFirst, first);
Assert.assertTrue(iterator.hasNext());
int i;
@ -157,7 +170,7 @@ public class GeneratorInputSourceTest
);
Assert.assertEquals(2, inputSource.estimateNumSplits(null, null));
Assert.assertEquals(false, inputSource.needsFormat());
Assert.assertFalse(inputSource.needsFormat());
Assert.assertEquals(2, inputSource.createSplits(null, null).count());
Assert.assertEquals(
new Long(2048L),

View File

@ -653,6 +653,13 @@ public abstract class ExprEval<T>
@Nullable
public static ExprEval<?> castForEqualityComparison(ExprEval<?> valueToCompare, ExpressionType typeToCompareWith)
{
if (valueToCompare.isArray() && !typeToCompareWith.isArray()) {
final Object[] array = valueToCompare.asArray();
// cannot cast array to scalar if array length is greater than 1
if (array != null && array.length > 1) {
return null;
}
}
ExprEval<?> cast = valueToCompare.castTo(typeToCompareWith);
if (ExpressionType.LONG.equals(typeToCompareWith) && valueToCompare.asDouble() != cast.asDouble()) {
// make sure the DOUBLE value when cast to LONG is the same before and after the cast

View File

@ -3327,11 +3327,11 @@ public interface Function extends NamedFunction
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List<Expr> args)
{
ExpressionType type = ExpressionType.LONG;
ExpressionType type = null;
for (Expr arg : args) {
type = ExpressionTypeConversion.function(type, arg.getOutputType(inspector));
type = ExpressionTypeConversion.leastRestrictiveType(type, arg.getOutputType(inspector));
}
return ExpressionType.asArrayType(type);
return type == null ? null : ExpressionTypeFactory.getInstance().ofArray(type);
}
/**

View File

@ -0,0 +1,530 @@
/*
* 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.RangeSet;
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.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.ColumnInspector;
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.NullableTypeStrategy;
import org.apache.druid.segment.column.TypeSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex;
import org.apache.druid.segment.index.BitmapColumnIndex;
import org.apache.druid.segment.index.semantic.ArrayElementIndexes;
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.Arrays;
import java.util.Comparator;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
/**
* Check to see if an array contains a specific element. This filter is not an exact replica of SQL ARRAY_CONTAINS
* or the native array_contains expression, which when given something like ARRAY_CONTAINS(arrayColumn, ARRAY[1,2,3])
* will check that arrayColumn contains all elements of the match value. To model this functionality, use an
* {@link AndDimFilter} with an element filter for each element to match.
*/
public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter implements Filter
{
private final String column;
private final ColumnType elementMatchValueType;
@Nullable
private final Object elementMatchValue;
private final ExprEval<?> elementMatchValueEval;
@Nullable
private final FilterTuning filterTuning;
private final DruidPredicateFactory predicateFactory;
@JsonCreator
public ArrayContainsElementFilter(
@JsonProperty("column") String column,
@JsonProperty("elementMatchValueType") ColumnType elementMatchValueType,
@JsonProperty("elementMatchValue") @Nullable Object elementMatchValue,
@JsonProperty("filterTuning") @Nullable FilterTuning filterTuning
)
{
if (column == null) {
throw InvalidInput.exception("Invalid array_contains filter, column cannot be null");
}
this.column = column;
if (elementMatchValueType == null) {
throw InvalidInput.exception("Invalid array_contains filter on column [%s], elementMatchValueType cannot be null", column);
}
this.elementMatchValueType = elementMatchValueType;
this.elementMatchValue = elementMatchValue;
this.elementMatchValueEval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(elementMatchValueType), elementMatchValue);
this.filterTuning = filterTuning;
this.predicateFactory = new ArrayContainsPredicateFactory(elementMatchValueEval);
}
@Override
public byte[] getCacheKey()
{
final NullableTypeStrategy<Object> typeStrategy = elementMatchValueEval.type().getNullableStrategy();
final int size = typeStrategy.estimateSizeBytes(elementMatchValueEval.value());
final ByteBuffer valueBuffer = ByteBuffer.allocate(size);
typeStrategy.write(valueBuffer, elementMatchValueEval.value(), size);
return new CacheKeyBuilder(DimFilterUtils.ARRAY_CONTAINS_CACHE_ID)
.appendByte(DimFilterUtils.STRING_SEPARATOR)
.appendString(column)
.appendByte(DimFilterUtils.STRING_SEPARATOR)
.appendString(elementMatchValueType.asTypeString())
.appendByte(DimFilterUtils.STRING_SEPARATOR)
.appendByteArray(valueBuffer.array())
.build();
}
@Override
public DimFilter optimize()
{
return this;
}
@Override
public Filter toFilter()
{
return this;
}
@JsonProperty
public String getColumn()
{
return column;
}
@JsonProperty
public ColumnType getElementMatchValueType()
{
return elementMatchValueType;
}
@JsonProperty
public Object getElementMatchValue()
{
return elementMatchValue;
}
@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_NULL)
public FilterTuning getFilterTuning()
{
return filterTuning;
}
@Override
public String toString()
{
DimFilter.DimFilterToStringBuilder bob =
new DimFilter.DimFilterToStringBuilder().append("array_contains_element(")
.appendDimension(column, null)
.append(", ")
.append(
elementMatchValueType.isArray()
? Arrays.deepToString(elementMatchValueEval.asArray())
: elementMatchValueEval.value()
)
.append(")");
if (!ColumnType.STRING.equals(elementMatchValueType)) {
bob.append(" (" + elementMatchValueType.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;
}
ArrayContainsElementFilter that = (ArrayContainsElementFilter) o;
if (!column.equals(that.column)) {
return false;
}
if (!Objects.equals(elementMatchValueType, that.elementMatchValueType)) {
return false;
}
if (!Objects.equals(filterTuning, that.filterTuning)) {
return false;
}
if (elementMatchValueType.isArray()) {
return Arrays.deepEquals(elementMatchValueEval.asArray(), that.elementMatchValueEval.asArray());
} else {
return Objects.equals(elementMatchValueEval.value(), that.elementMatchValueEval.value());
}
}
@Override
public int hashCode()
{
return Objects.hash(column, elementMatchValueType, elementMatchValueEval.value(), filterTuning);
}
@Override
public RangeSet<String> getDimensionRangeSet(String dimension)
{
return null;
}
@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 AllUnknownBitmapColumnIndex(selector);
}
final ArrayElementIndexes elementIndexes = indexSupplier.as(ArrayElementIndexes.class);
if (elementIndexes != null) {
return elementIndexes.containsValue(elementMatchValueEval.value(), elementMatchValueType);
}
if (elementMatchValueEval.valueOrDefault() != null && selector.getColumnCapabilities(column) != null && !selector.getColumnCapabilities(column).isArray()) {
// column is not an array, behave like a normal equality filter
return EqualityFilter.getEqualityIndex(column, elementMatchValueEval, elementMatchValueType, selector);
}
// column exists, but has no indexes we can use
return null;
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
return ColumnProcessors.makeProcessor(
column,
new TypedConstantElementValueMatcherFactory(elementMatchValueEval, predicateFactory),
factory
);
}
@Override
public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
{
final ColumnCapabilities capabilities = factory.getColumnCapabilities(column);
if (elementMatchValueEval.valueOrDefault() != null && elementMatchValueType.isPrimitive() && (capabilities == null || capabilities.isPrimitive())) {
return ColumnProcessors.makeVectorProcessor(
column,
VectorValueMatcherColumnProcessorFactory.instance(),
factory
).makeMatcher(elementMatchValueEval.value(), elementMatchValueType);
}
return ColumnProcessors.makeVectorProcessor(
column,
VectorValueMatcherColumnProcessorFactory.instance(),
factory
).makeMatcher(predicateFactory);
}
@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 ArrayContainsElementFilter(
rewriteDimensionTo,
elementMatchValueType,
elementMatchValue,
filterTuning
);
}
private static class ArrayContainsPredicateFactory implements DruidPredicateFactory
{
private final ExprEval<?> elementMatchValue;
private final EqualityFilter.EqualityPredicateFactory equalityPredicateFactory;
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 ArrayContainsPredicateFactory(ExprEval<?> elementMatchValue)
{
this.elementMatchValue = elementMatchValue;
this.equalityPredicateFactory = new EqualityFilter.EqualityPredicateFactory(elementMatchValue);
// if element match value is an array, scalar matches can never be true
final Object matchVal = elementMatchValue.valueOrDefault();
if (matchVal == null || (elementMatchValue.isArray() && elementMatchValue.asArray().length > 1)) {
this.stringPredicateSupplier = Predicates::alwaysFalse;
this.longPredicateSupplier = () -> DruidLongPredicate.ALWAYS_FALSE;
this.doublePredicateSupplier = () -> DruidDoublePredicate.ALWAYS_FALSE;
this.floatPredicateSupplier = () -> DruidFloatPredicate.ALWAYS_FALSE;
} else {
this.stringPredicateSupplier = equalityPredicateFactory::makeStringPredicate;
this.longPredicateSupplier = equalityPredicateFactory::makeLongPredicate;
this.doublePredicateSupplier = equalityPredicateFactory::makeDoublePredicate;
this.floatPredicateSupplier = equalityPredicateFactory::makeFloatPredicate;
}
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 new FallbackPredicate<>(computeArrayPredicate(arrayType), ExpressionType.fromColumnTypeStrict(arrayType));
}
@Override
public Predicate<Object> makeObjectPredicate()
{
return objectPredicateSupplier.get();
}
private Supplier<Predicate<Object>> makeObjectPredicateSupplier()
{
return Suppliers.memoize(() -> input -> {
if (input == null) {
return false;
}
final ExprEval<?> inputEval = ExprEval.bestEffortOf(StructuredData.unwrap(input));
final Predicate<Object[]> matcher = new FallbackPredicate<>(
computeArrayPredicate(ExpressionType.toColumnType(inputEval.asArrayType())),
inputEval.asArrayType()
);
return matcher.apply(inputEval.asArray());
});
}
private Predicate<Object[]> computeArrayPredicate(TypeSignature<ValueType> arrayType)
{
return arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType));
}
private Supplier<Predicate<Object[]>> makeTypeDetectingArrayPredicate()
{
return Suppliers.memoize(() -> input -> {
if (input == null) {
return false;
}
// just use object predicate logic
final Predicate<Object> objectPredicate = objectPredicateSupplier.get();
return objectPredicate.apply(input);
});
}
private Predicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
{
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
final Comparator elementComparator = arrayType.getElementType().getNullableStrategy();
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(
elementMatchValue,
(ExpressionType) expressionType.getElementType()
);
if (castForComparison == null) {
return Predicates.alwaysFalse();
}
final Object matchVal = castForComparison.value();
return input -> {
if (input == null) {
return false;
}
boolean anyMatch = false;
for (Object elem : input) {
anyMatch = anyMatch || elementComparator.compare(elem, matchVal) == 0;
}
return anyMatch;
};
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ArrayContainsPredicateFactory that = (ArrayContainsPredicateFactory) o;
if (!Objects.equals(elementMatchValue.type(), that.elementMatchValue.type())) {
return false;
}
if (elementMatchValue.isArray()) {
return Arrays.deepEquals(elementMatchValue.asArray(), that.elementMatchValue.asArray());
}
return Objects.equals(elementMatchValue.value(), that.elementMatchValue.value());
}
@Override
public int hashCode()
{
return Objects.hash(elementMatchValue);
}
}
/**
* {@link EqualityFilter.TypedConstantValueMatcherFactory} with special handling for scalar processors in the case
* matchValue is null or an array (which is not possible in equality filter, but is allowed by this filter).
* Uses {@link ArrayContainsPredicateFactory} for the base predicate factory so that it performs element matching
* instead of standard equality matching.
*/
private static class TypedConstantElementValueMatcherFactory extends EqualityFilter.TypedConstantValueMatcherFactory
{
public TypedConstantElementValueMatcherFactory(
ExprEval<?> matchValue,
DruidPredicateFactory predicateFactory
)
{
super(matchValue, predicateFactory);
}
@Override
public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
{
if (matchValue.valueOrDefault() == null || matchValue.isArray()) {
return predicateMatcherFactory.makeDimensionProcessor(selector, multiValue);
}
return super.makeDimensionProcessor(selector, multiValue);
}
@Override
public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
{
if (matchValue.valueOrDefault() == null || matchValue.isArray()) {
return predicateMatcherFactory.makeFloatProcessor(selector);
}
return super.makeFloatProcessor(selector);
}
@Override
public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
{
if (matchValue.valueOrDefault() == null || matchValue.isArray()) {
return predicateMatcherFactory.makeDoubleProcessor(selector);
}
return super.makeDoubleProcessor(selector);
}
@Override
public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
{
if (matchValue.valueOrDefault() == null || matchValue.isArray()) {
return predicateMatcherFactory.makeLongProcessor(selector);
}
return super.makeLongProcessor(selector);
}
}
}

View File

@ -53,7 +53,8 @@ import java.util.Set;
@JsonSubTypes.Type(name = "equals", value = EqualityFilter.class),
@JsonSubTypes.Type(name = "range", value = RangeFilter.class),
@JsonSubTypes.Type(name = "isfalse", value = IsFalseDimFilter.class),
@JsonSubTypes.Type(name = "istrue", value = IsTrueDimFilter.class)
@JsonSubTypes.Type(name = "istrue", value = IsTrueDimFilter.class),
@JsonSubTypes.Type(name = "arrayContainsElement", value = ArrayContainsElementFilter.class)
})
public interface DimFilter extends Cacheable
{

View File

@ -60,8 +60,8 @@ public class DimFilterUtils
static final byte NULL_CACHE_ID = 0x12;
static final byte EQUALS_CACHE_ID = 0x13;
static final byte RANGE_CACHE_ID = 0x14;
static final byte IS_FILTER_BOOLEAN_FILTER_CACHE_ID = 0x15;
static final byte ARRAY_CONTAINS_CACHE_ID = 0x16;
public static final byte STRING_SEPARATOR = (byte) 0xFF;

View File

@ -169,7 +169,11 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
DimFilter.DimFilterToStringBuilder bob =
new DimFilter.DimFilterToStringBuilder().appendDimension(column, null)
.append(" = ")
.append(matchValueEval.value());
.append(
matchValueEval.isArray()
? Arrays.deepToString(matchValueEval.asArray())
: matchValueEval.value()
);
if (!ColumnType.STRING.equals(matchValueType)) {
bob.append(" (" + matchValueType.asTypeString() + ")");
@ -231,28 +235,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) {
return null;
}
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column);
if (indexSupplier == null) {
return new AllUnknownBitmapColumnIndex(selector);
}
final ValueIndexes valueIndexes = indexSupplier.as(ValueIndexes.class);
if (valueIndexes != null) {
// matchValueEval.value() cannot be null here due to check in the constructor
//noinspection DataFlowIssue
return valueIndexes.forValue(matchValueEval.value(), matchValueType);
}
if (matchValueType.isPrimitive()) {
final StringValueSetIndexes stringValueSetIndexes = indexSupplier.as(StringValueSetIndexes.class);
if (stringValueSetIndexes != null) {
return stringValueSetIndexes.forValue(matchValueEval.asString());
}
}
// column exists, but has no indexes we can use
return null;
return getEqualityIndex(column, matchValueEval, matchValueType, selector);
}
@Override
@ -329,7 +312,37 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
);
}
private static class EqualityPredicateFactory implements DruidPredicateFactory
public static BitmapColumnIndex getEqualityIndex(
String column,
ExprEval<?> matchValueEval,
ColumnType matchValueType,
ColumnIndexSelector selector
)
{
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column);
if (indexSupplier == null) {
return new AllUnknownBitmapColumnIndex(selector);
}
final ValueIndexes valueIndexes = indexSupplier.as(ValueIndexes.class);
if (valueIndexes != null) {
// matchValueEval.value() cannot be null here due to check in the constructor
//noinspection DataFlowIssue
return valueIndexes.forValue(matchValueEval.value(), matchValueType);
}
if (matchValueType.isPrimitive()) {
final StringValueSetIndexes stringValueSetIndexes = indexSupplier.as(StringValueSetIndexes.class);
if (stringValueSetIndexes != null) {
return stringValueSetIndexes.forValue(matchValueEval.asString());
}
}
// column exists, but has no indexes we can use
return null;
}
public static class EqualityPredicateFactory implements DruidPredicateFactory
{
private final ExprEval<?> matchValue;
private final Supplier<Predicate<String>> stringPredicateSupplier;
@ -472,6 +485,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
return arrayComparator.compare(input, matchArray) == 0;
});
}
private Predicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
{
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
@ -512,10 +526,10 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
}
}
private static class TypedConstantValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
public static class TypedConstantValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
{
private final ExprEval<?> matchValue;
private final PredicateValueMatcherFactory predicateMatcherFactory;
protected final ExprEval<?> matchValue;
protected final PredicateValueMatcherFactory predicateMatcherFactory;
public TypedConstantValueMatcherFactory(
ExprEval<?> matchValue,

View File

@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IndexSizeExceededException;
@ -102,6 +103,21 @@ public class DataGenerator
return new MapBasedInputRow(nextTimestamp(), dimensionNames, event);
}
public Map<String, Object> nextRaw()
{
return nextRaw(TimestampSpec.DEFAULT_COLUMN);
}
public Map<String, Object> nextRaw(String timestampColumn)
{
Map<String, Object> event = new HashMap<>();
for (ColumnValueGenerator generator : columnGenerators) {
event.put(generator.getSchema().getName(), generator.generateRowValue());
}
event.put(timestampColumn, nextTimestamp());
return event;
}
/**
* Reset this generator to start from the begining of the interval with a new seed.
*

View File

@ -384,19 +384,19 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
final int id = dictionary.indexOf(ids) + arrayOffset;
final int localId = dictionary.indexOf(ids);
if (includeUnknown) {
if (id < 0) {
if (localId < 0) {
return bitmapResultFactory.wrapDimensionValue(nullValueBitmap);
}
return bitmapResultFactory.unionDimensionValueBitmaps(
ImmutableList.of(getBitmap(id), nullValueBitmap)
ImmutableList.of(getBitmap(localId + arrayOffset), nullValueBitmap)
);
}
if (id < 0) {
if (localId < 0) {
return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap());
}
return bitmapResultFactory.wrapDimensionValue(getBitmap(id));
return bitmapResultFactory.wrapDimensionValue(getBitmap(localId + arrayOffset));
}
};
}
@ -404,20 +404,24 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
private class VariantArrayElementIndexes implements ArrayElementIndexes
{
@Nullable
@Override
public BitmapColumnIndex containsValue(@Nullable Object value, TypeSignature<ValueType> elementValueType)
{
// this column doesn't store nested arrays, bail out if checking if we contain an array
if (elementValueType.isArray()) {
return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap);
}
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(elementValueType), value);
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(
eval,
ExpressionType.fromColumnTypeStrict(logicalType.getElementType())
ExpressionType.fromColumnTypeStrict(logicalType.isArray() ? logicalType.getElementType() : logicalType)
);
if (castForComparison == null) {
return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap);
}
Indexed elements;
final Indexed elements;
final int elementOffset;
switch (logicalType.getElementType().getType()) {
case STRING:

View File

@ -277,6 +277,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
}
builder.setType(logicalType);
builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false);
builder.setColumnFormat(new NestedCommonFormatColumn.Format(
logicalType,
capabilitiesBuilder.hasNulls().isTrue(),

View File

@ -309,6 +309,13 @@ public class FunctionTest extends InitializedNullHandlingTest
assertArrayExpr("array(1, 2, 3, 'bar')", new Long[]{1L, 2L, 3L, null});
assertArrayExpr("array(1.0)", new Double[]{1.0});
assertArrayExpr("array('foo', 'bar')", new String[]{"foo", "bar"});
assertArrayExpr(
"array(a, b)",
new Object[]{
new Object[]{"foo", "bar", "baz", "foobar"},
new Object[]{"1", "2", "3", "4", "5"}
}
);
}
@Test

View File

@ -329,6 +329,11 @@ public class OutputTypeTest extends InitializedNullHandlingTest
{
assertOutputType("array(1, 2, 3)", inspector, ExpressionType.LONG_ARRAY);
assertOutputType("array(1, 2, 3.0)", inspector, ExpressionType.DOUBLE_ARRAY);
assertOutputType(
"array(a, b)",
inspector,
ExpressionTypeFactory.getInstance().ofArray(ExpressionType.STRING_ARRAY)
);
assertOutputType("array_length(a)", inspector, ExpressionType.LONG);
assertOutputType("array_length(b)", inspector, ExpressionType.LONG);

View File

@ -0,0 +1,732 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment.filter;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
import org.apache.druid.guice.NestedDataModule;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.query.filter.ArrayContainsElementFilter;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.filter.FilterTuning;
import org.apache.druid.query.filter.NotDimFilter;
import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import org.junit.experimental.runners.Enclosed;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.Closeable;
import java.util.Arrays;
@RunWith(Enclosed.class)
public class ArrayContainsElementFilterTests
{
@RunWith(Parameterized.class)
public static class ArrayContainsElementFilterTest extends BaseFilterTest
{
public ArrayContainsElementFilterTest(
String testName,
IndexBuilder indexBuilder,
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
boolean cnf,
boolean optimize
)
{
super(testName, DEFAULT_ROWS, indexBuilder, finisher, cnf, optimize);
}
@AfterClass
public static void tearDown() throws Exception
{
BaseFilterTest.tearDown(ArrayContainsElementFilterTest.class.getName());
}
@Test
public void testArrayStringColumn()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
/*
dim0 .. arrayString
"0", .. ["a", "b", "c"]
"1", .. []
"2", .. null
"3", .. ["a", "b", "c"]
"4", .. ["c", "d"]
"5", .. [null]
*/
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING,
"a",
null
),
ImmutableList.of("0", "3")
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING,
"a",
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "4", "5")
: ImmutableList.of("1", "2", "4", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING,
"c",
null
),
ImmutableList.of("0", "3", "4")
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING,
"c",
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "5")
: ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING,
null,
null
),
ImmutableList.of("5")
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING,
null,
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "3", "4")
: ImmutableList.of("0", "1", "2", "3", "4")
);
}
@Test
public void testArrayLongColumn()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
/*
dim0 .. arrayLong
"0", .. [1L, 2L, 3L]
"1", .. []
"2", .. [1L, 2L, 3L]
"3", .. null
"4", .. [null]
"5", .. [123L, 345L]
*/
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.LONG,
2L,
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.LONG,
2L,
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "4", "5")
: ImmutableList.of("1", "3", "4", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.LONG,
null,
null
),
ImmutableList.of("4")
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.LONG,
null,
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "5")
: ImmutableList.of("0", "1", "2", "3", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.DOUBLE,
2.0,
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.STRING,
"2",
null
),
ImmutableList.of("0", "2")
);
}
@Test
public void testArrayDoubleColumn()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
/*
dim0 .. arrayDouble
"0", .. [1.1, 2.2, 3.3]
"1", .. [1.1, 2.2, 3.3]
"2", .. [null]
"3", .. []
"4", .. [-1.1, -333.3]
"5", .. null
*/
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayDouble",
ColumnType.DOUBLE,
2.2,
null
),
ImmutableList.of("0", "1")
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayDouble",
ColumnType.DOUBLE,
2.2,
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("2", "3", "4")
: ImmutableList.of("2", "3", "4", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayDouble",
ColumnType.STRING,
"2.2",
null
),
ImmutableList.of("0", "1")
);
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayDouble",
ColumnType.DOUBLE,
null,
null
),
ImmutableList.of("2")
);
}
@Test
public void testArrayStringColumnContainsArrays()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
// these are not nested arrays, expect no matches
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "3", "4", "5")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
}
@Test
public void testArrayLongColumnContainsArrays()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
// these are not nested arrays, expect no matches
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(1L, 2L, 3L),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(1L, 2L, 3L),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "4", "5")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
}
@Test
public void testArrayDoubleColumnContainsArrays()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
// these are not nested arrays, expect no matches
assertFilterMatches(
new ArrayContainsElementFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.3),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new ArrayContainsElementFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.3),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "3", "4")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
}
@Test
public void testScalarColumnContains()
{
assertFilterMatches(
new ArrayContainsElementFilter("s0", ColumnType.STRING, "a", null),
ImmutableList.of("1", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter("s0", ColumnType.STRING, "b", null),
ImmutableList.of("2")
);
assertFilterMatches(
new ArrayContainsElementFilter("s0", ColumnType.STRING, "c", null),
ImmutableList.of("4")
);
assertFilterMatches(
new ArrayContainsElementFilter("s0", ColumnType.STRING, "noexist", null),
ImmutableList.of()
);
assertFilterMatches(
new ArrayContainsElementFilter("s0", ColumnType.STRING_ARRAY, ImmutableList.of("c"), null),
ImmutableList.of("4")
);
assertFilterMatches(
new ArrayContainsElementFilter("s0", ColumnType.STRING_ARRAY, ImmutableList.of("a", "c"), null),
ImmutableList.of()
);
assertFilterMatches(
new ArrayContainsElementFilter("d0", ColumnType.DOUBLE, 10.1, null),
ImmutableList.of("1")
);
assertFilterMatches(
new ArrayContainsElementFilter("d0", ColumnType.DOUBLE, 120.0245, null),
ImmutableList.of("3")
);
assertFilterMatches(
new ArrayContainsElementFilter("d0", ColumnType.DOUBLE, 765.432, null),
ImmutableList.of("5")
);
assertFilterMatches(
new ArrayContainsElementFilter("d0", ColumnType.DOUBLE, 765.431, null),
ImmutableList.of()
);
assertFilterMatches(
new ArrayContainsElementFilter("d0", ColumnType.DOUBLE_ARRAY, new Object[]{10.1}, null),
ImmutableList.of("1")
);
assertFilterMatches(
new ArrayContainsElementFilter("d0", ColumnType.DOUBLE_ARRAY, new Object[]{10.1, 120.0245}, null),
ImmutableList.of()
);
assertFilterMatches(
new ArrayContainsElementFilter("l0", ColumnType.LONG, 100L, null),
ImmutableList.of("1")
);
assertFilterMatches(
new ArrayContainsElementFilter("l0", ColumnType.LONG, 40L, null),
ImmutableList.of("2")
);
assertFilterMatches(
new ArrayContainsElementFilter("l0", ColumnType.LONG, 9001L, null),
ImmutableList.of("4")
);
assertFilterMatches(
new ArrayContainsElementFilter("l0", ColumnType.LONG, 9000L, null),
ImmutableList.of()
);
assertFilterMatches(
new ArrayContainsElementFilter("l0", ColumnType.LONG_ARRAY, ImmutableList.of(9001L), null),
ImmutableList.of("4")
);
assertFilterMatches(
new ArrayContainsElementFilter("l0", ColumnType.LONG_ARRAY, ImmutableList.of(40L, 9001L), null),
ImmutableList.of()
);
}
@Test
public void testArrayContainsNestedArray()
{
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
assertFilterMatchesSkipVectorize(
new ArrayContainsElementFilter("nestedArrayLong", ColumnType.LONG_ARRAY, new Object[]{1L, 2L, 3L}, null),
ImmutableList.of("0", "2")
);
assertFilterMatchesSkipVectorize(
new ArrayContainsElementFilter("nestedArrayLong", ColumnType.LONG_ARRAY, new Object[]{1L, 2L}, null),
ImmutableList.of()
);
}
@Test
public void testArrayContainsMvd()
{
assertFilterMatches(
new ArrayContainsElementFilter("dim2", ColumnType.STRING, "a", null),
ImmutableList.of("0", "3")
);
if (isAutoSchema()) {
assertFilterMatches(
NotDimFilter.of(new ArrayContainsElementFilter("dim2", ColumnType.STRING, "a", null)),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "2", "4")
: ImmutableList.of("1", "2", "4", "5")
);
// [""] becomes [null] in default value mode
assertFilterMatches(
new ArrayContainsElementFilter("dim2", ColumnType.STRING, null, null),
NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("2")
);
} else {
// multi-value dimension treats [] as null, so in sql compatible mode row 1 ends up as not matching
assertFilterMatches(
NotDimFilter.of(new ArrayContainsElementFilter("dim2", ColumnType.STRING, "a", null)),
NullHandling.sqlCompatible()
? ImmutableList.of("2", "4")
: ImmutableList.of("1", "2", "4", "5")
);
assertFilterMatches(
new ArrayContainsElementFilter("dim2", ColumnType.STRING, null, null),
ImmutableList.of()
);
}
}
}
public static class ArrayContainsElementFilterNonParameterizedTests extends InitializedNullHandlingTest
{
@Test
public void testSerde() throws JsonProcessingException
{
ObjectMapper mapper = new DefaultObjectMapper();
ArrayContainsElementFilter filter = new ArrayContainsElementFilter("x", ColumnType.STRING, "hello", null);
String s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.LONG, 1L, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.LONG, 1, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.DOUBLE, 111.111, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.FLOAT, 1234.0f, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.STRING_ARRAY, new Object[]{"a", "b", null, "c"}, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.STRING_ARRAY, Arrays.asList("a", "b", null, "c"), null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.LONG_ARRAY, new Object[]{1L, null, 2L, 3L}, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.LONG_ARRAY, Arrays.asList(1L, null, 2L, 3L), null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.DOUBLE_ARRAY, new Object[]{1.1, 2.1, null, 3.1}, null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter("x", ColumnType.DOUBLE_ARRAY, Arrays.asList(1.1, 2.1, null, 3.1), null);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
filter = new ArrayContainsElementFilter(
"x",
ColumnType.NESTED_DATA,
ImmutableMap.of("x", ImmutableList.of(1, 2, 3)),
null
);
s = mapper.writeValueAsString(filter);
Assert.assertEquals(filter, mapper.readValue(s, ArrayContainsElementFilter.class));
}
@Test
public void testRewrite()
{
ArrayContainsElementFilter filter = new ArrayContainsElementFilter("x", ColumnType.STRING, "hello", null);
Filter rewrite = filter.rewriteRequiredColumns(ImmutableMap.of("x", "y"));
ArrayContainsElementFilter expected = new ArrayContainsElementFilter("y", ColumnType.STRING, "hello", null);
Assert.assertEquals(expected, rewrite);
}
@Test
public void testGetCacheKey()
{
ArrayContainsElementFilter f1 = new ArrayContainsElementFilter("x", ColumnType.STRING, "hello", null);
ArrayContainsElementFilter f1_2 = new ArrayContainsElementFilter("x", ColumnType.STRING, "hello", null);
ArrayContainsElementFilter f2 = new ArrayContainsElementFilter("x", ColumnType.STRING, "world", null);
ArrayContainsElementFilter f3 = new ArrayContainsElementFilter(
"x",
ColumnType.STRING,
"hello",
new FilterTuning(true, null, null)
);
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
f1 = new ArrayContainsElementFilter("x", ColumnType.LONG, 1L, null);
f1_2 = new ArrayContainsElementFilter("x", ColumnType.LONG, 1, null);
f2 = new ArrayContainsElementFilter("x", ColumnType.LONG, 2L, null);
f3 = new ArrayContainsElementFilter("x", ColumnType.LONG, 1L, new FilterTuning(true, null, null));
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
f1 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE, 1.1, null);
f1_2 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE, 1.1, null);
f2 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE, 2.2, null);
f3 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE, 1.1, new FilterTuning(true, null, null));
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
f1 = new ArrayContainsElementFilter("x", ColumnType.FLOAT, 1.1f, null);
f1_2 = new ArrayContainsElementFilter("x", ColumnType.FLOAT, 1.1f, null);
f2 = new ArrayContainsElementFilter("x", ColumnType.FLOAT, 2.2f, null);
f3 = new ArrayContainsElementFilter("x", ColumnType.FLOAT, 1.1f, new FilterTuning(true, null, null));
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
f1 = new ArrayContainsElementFilter("x", ColumnType.STRING_ARRAY, new Object[]{"a", "b", null, "c"}, null);
f1_2 = new ArrayContainsElementFilter("x", ColumnType.STRING_ARRAY, Arrays.asList("a", "b", null, "c"), null);
f2 = new ArrayContainsElementFilter("x", ColumnType.STRING_ARRAY, new Object[]{"a", "b", "c"}, null);
f3 = new ArrayContainsElementFilter(
"x",
ColumnType.STRING_ARRAY,
new Object[]{"a", "b", null, "c"},
new FilterTuning(true, null, null)
);
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
f1 = new ArrayContainsElementFilter("x", ColumnType.LONG_ARRAY, new Object[]{100L, 200L, null, 300L}, null);
f1_2 = new ArrayContainsElementFilter("x", ColumnType.LONG_ARRAY, Arrays.asList(100L, 200L, null, 300L), null);
f2 = new ArrayContainsElementFilter("x", ColumnType.LONG_ARRAY, new Object[]{100L, null, 200L, 300L}, null);
f3 = new ArrayContainsElementFilter(
"x",
ColumnType.LONG_ARRAY,
new Object[]{100L, 200L, null, 300L},
new FilterTuning(true, null, null)
);
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
f1 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE_ARRAY, new Object[]{1.001, null, 20.0002, 300.0003}, null);
f1_2 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE_ARRAY, Arrays.asList(1.001, null, 20.0002, 300.0003), null);
f2 = new ArrayContainsElementFilter("x", ColumnType.DOUBLE_ARRAY, new Object[]{1.001, 20.0002, 300.0003, null}, null);
f3 = new ArrayContainsElementFilter(
"x",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.001, null, 20.0002, 300.0003},
new FilterTuning(true, null, null)
);
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
NestedDataModule.registerHandlersAndSerde();
f1 = new ArrayContainsElementFilter("x", ColumnType.NESTED_DATA, ImmutableMap.of("x", ImmutableList.of(1, 2, 3)), null);
f1_2 = new ArrayContainsElementFilter(
"x",
ColumnType.NESTED_DATA,
ImmutableMap.of("x", ImmutableList.of(1, 2, 3)),
null
);
f2 = new ArrayContainsElementFilter(
"x",
ColumnType.NESTED_DATA,
ImmutableMap.of("x", ImmutableList.of(1, 2, 3, 4)),
null
);
f3 = new ArrayContainsElementFilter(
"x",
ColumnType.NESTED_DATA,
ImmutableMap.of("x", ImmutableList.of(1, 2, 3)),
new FilterTuning(true, null, null)
);
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
}
@Test
public void testInvalidParameters()
{
Throwable t = Assert.assertThrows(
DruidException.class,
() -> new ArrayContainsElementFilter(null, ColumnType.STRING, null, null)
);
Assert.assertEquals("Invalid array_contains filter, column cannot be null", t.getMessage());
t = Assert.assertThrows(
DruidException.class,
() -> new ArrayContainsElementFilter("dim0", null, null, null)
);
Assert.assertEquals(
"Invalid array_contains filter on column [dim0], elementMatchValueType cannot be null",
t.getMessage()
);
}
@Test
public void test_equals()
{
EqualsVerifier.forClass(ArrayContainsElementFilter.class).usingGetClass()
.withNonnullFields(
"column",
"elementMatchValueType",
"elementMatchValueEval",
"elementMatchValue",
"predicateFactory",
"cachedOptimizedFilter"
)
.withPrefabValues(ColumnType.class, ColumnType.STRING, ColumnType.DOUBLE)
.withIgnoredFields("predicateFactory", "cachedOptimizedFilter", "elementMatchValue")
.verify();
}
}
}

View File

@ -141,6 +141,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
new ExpressionVirtualColumn("vd0", "d0", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vf0", "f0", ColumnType.FLOAT, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vl0", "l0", ColumnType.LONG, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("nestedArrayLong", "array(arrayLong)", ColumnType.ofArray(ColumnType.LONG_ARRAY), TestExprMacroTable.INSTANCE),
new ListFilteredVirtualColumn("allow-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), true),
new ListFilteredVirtualColumn("deny-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), false),
new ListFilteredVirtualColumn("allow-dim2", DefaultDimensionSpec.of("dim2"), ImmutableSet.of("a"), true),

View File

@ -46,6 +46,7 @@ import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import org.junit.experimental.runners.Enclosed;
import org.junit.runner.RunWith;
@ -722,265 +723,264 @@ public class EqualityFilterTests
@Test
public void testArrays()
{
if (isAutoSchema()) {
// only auto schema supports array columns... skip other segment types
/*
dim0 .. arrayString arrayLong arrayDouble
"0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3]
"1", .. [], [], [1.1, 2.2, 3.3]
"2", .. null, [1L, 2L, 3L], [null]
"3", .. ["a", "b", "c"], null, []
"4", .. ["c", "d"], [null], [-1.1, -333.3]
"5", .. [null], [123L, 345L], null
*/
// only auto schema supports array columns... skip other segment types
Assume.assumeTrue(isAutoSchema());
/*
dim0 .. arrayString arrayLong arrayDouble
"0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3]
"1", .. [], [], [1.1, 2.2, 3.3]
"2", .. null, [1L, 2L, 3L], [null]
"3", .. ["a", "b", "c"], null, []
"4", .. ["c", "d"], [null], [-1.1, -333.3]
"5", .. [null], [123L, 345L], null
*/
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
),
ImmutableList.of("0", "3")
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "4", "5")
: ImmutableList.of("1", "2", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{"a", "b", "c"},
null
),
ImmutableList.of("0", "3")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of(),
null
),
ImmutableList.of("1")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("5")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{null, null},
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{null, null},
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "3", "4", "5")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
),
ImmutableList.of("0", "3")
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "4", "5")
: ImmutableList.of("1", "2", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{"a", "b", "c"},
null
),
ImmutableList.of("0", "3")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
ImmutableList.of(),
null
),
ImmutableList.of("1")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("5")
);
assertFilterMatches(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{null, null},
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayString",
ColumnType.STRING_ARRAY,
new Object[]{null, null},
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "3", "4", "5")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(1L, 2L, 3L),
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(1L, 2L, 3L),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "4", "5")
: ImmutableList.of("1", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{1L, 2L, 3L},
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(),
null
),
ImmutableList.of("1")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("4")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{null, null},
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{null, null},
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "4", "5")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(1L, 2L, 3L),
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(1L, 2L, 3L),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("1", "4", "5")
: ImmutableList.of("1", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{1L, 2L, 3L},
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
ImmutableList.of(),
null
),
ImmutableList.of("1")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("4")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{null, null},
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayLong",
ColumnType.LONG_ARRAY,
new Object[]{null, null},
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "4", "5")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
// test loss of precision matching long arrays with double array match values
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.0, 2.0, 3.0},
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.1, 2.2, 3.3},
null
),
ImmutableList.of()
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.DOUBLE_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("4")
);
// test loss of precision matching long arrays with double array match values
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.0, 2.0, 3.0},
null
),
ImmutableList.of("0", "2")
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.1, 2.2, 3.3},
null
),
ImmutableList.of()
);
assertFilterMatches(
new EqualityFilter(
"arrayLong",
ColumnType.DOUBLE_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("4")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.3),
null
),
ImmutableList.of("0", "1")
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.3),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("2", "3", "4")
: ImmutableList.of("2", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.1, 2.2, 3.3},
null
),
ImmutableList.of("0", "1")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(),
null
),
ImmutableList.of("3")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("2")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.4),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.4),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "3", "4")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
}
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.3),
null
),
ImmutableList.of("0", "1")
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.3),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("2", "3", "4")
: ImmutableList.of("2", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
new Object[]{1.1, 2.2, 3.3},
null
),
ImmutableList.of("0", "1")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(),
null
),
ImmutableList.of("3")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
new Object[]{null},
null
),
ImmutableList.of("2")
);
assertFilterMatches(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.4),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"arrayDouble",
ColumnType.DOUBLE_ARRAY,
ImmutableList.of(1.1, 2.2, 3.4),
null
)
),
NullHandling.sqlCompatible()
? ImmutableList.of("0", "1", "2", "3", "4")
: ImmutableList.of("0", "1", "2", "3", "4", "5")
);
}
@Test
@ -996,68 +996,67 @@ public class EqualityFilterTests
"5", .. [100, 200, 300]
*/
if (isAutoSchema()) {
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"variant",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
)
),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
Assume.assumeTrue(isAutoSchema());
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
),
ImmutableList.of()
);
assertFilterMatches(
NotDimFilter.of(
new EqualityFilter(
"variant",
ColumnType.STRING_ARRAY,
ImmutableList.of("a", "b", "c"),
null
)
),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.STRING,
"abc",
null
),
ImmutableList.of("0")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.STRING,
"abc",
null
),
ImmutableList.of("0")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.LONG,
100L,
null
),
ImmutableList.of("1", "2")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.LONG,
100L,
null
),
ImmutableList.of("1", "2")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.STRING,
"100",
null
),
ImmutableList.of("1", "2")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.STRING,
"100",
null
),
ImmutableList.of("1", "2")
);
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.LONG_ARRAY,
Arrays.asList(100, 200, 300),
null
),
ImmutableList.of("5")
);
}
assertFilterMatches(
new EqualityFilter(
"variant",
ColumnType.LONG_ARRAY,
Arrays.asList(100, 200, 300),
null
),
ImmutableList.of("5")
);
}
}

View File

@ -22,8 +22,10 @@ package org.apache.druid.segment.generator;
import com.google.common.hash.Hashing;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.NestedDataModule;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.ISE;
@ -52,7 +54,6 @@ import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -183,14 +184,16 @@ public class SegmentGenerator implements Closeable
final List<QueryableIndex> indexes = new ArrayList<>();
Transformer transformer = transformSpec.toTransformer();
InputRowSchema rowSchema = new InputRowSchema(
new TimestampSpec(null, null, null),
dimensionsSpec,
null
);
for (int i = 0; i < numRows; i++) {
final InputRow row = transformer.transform(dataGenerator.nextRow());
Map<String, Object> evaluated = new HashMap<>();
for (String dimension : dimensionsSpec.getDimensionNames()) {
evaluated.put(dimension, row.getRaw(dimension));
}
MapBasedInputRow transformedRow = new MapBasedInputRow(row.getTimestamp(), dimensionsSpec.getDimensionNames(), evaluated);
Map<String, Object> raw = dataGenerator.nextRaw();
InputRow inputRow = MapInputRowParser.parse(rowSchema, raw);
InputRow transformedRow = transformer.transform(inputRow);
rows.add(transformedRow);
if ((i + 1) % 20000 == 0) {

View File

@ -461,9 +461,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest
}
Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i));
if (expectedType.getSingleType() != null) {
Assert.assertFalse(arrayElementIndexes.containsValue(null, expectedType.getSingleType()).computeBitmapResult(resultFactory,
false
).get(i));
Assert.assertFalse(arrayElementIndexes.containsValue(null, expectedType.getSingleType()).computeBitmapResult(resultFactory, false).get(i));
}
}

View File

@ -31,8 +31,10 @@ import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.query.filter.AndDimFilter;
import org.apache.druid.query.filter.ArrayContainsElementFilter;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.EqualityFilter;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
@ -139,6 +141,43 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper
}
}
}
// if the input is a direct array column, we can use sweet array filter
if (leftExpr.isDirectColumnAccess() && isArray(leftExpr)) {
Expr expr = plannerContext.parseExpression(rightExpr.getExpression());
// To convert this expression filter into an And of ArrayContainsElement filters, we need to extract all array
// elements. For now, we can optimize only when rightExpr is a literal because there is no way to extract the
// array elements by traversing the Expr. Note that all implementations of Expr are defined as package-private
// classes in a different package.
if (expr.isLiteral()) {
// Evaluate the expression to get out the array elements.
// We can safely pass a nil ObjectBinding if the expression is literal.
ExprEval<?> exprEval = expr.eval(InputBindings.nilBindings());
if (exprEval.isArray()) {
final Object[] arrayElements = exprEval.asArray();
final List<DimFilter> filters = new ArrayList<>(arrayElements.length);
final ColumnType elementType = ExpressionType.toColumnType(ExpressionType.elementType(exprEval.type()));
for (final Object val : arrayElements) {
filters.add(
new ArrayContainsElementFilter(
leftExpr.getSimpleExtraction().getColumn(),
elementType,
val,
null
)
);
}
return filters.size() == 1 ? filters.get(0) : new AndDimFilter(filters);
} else {
return new ArrayContainsElementFilter(
leftExpr.getSimpleExtraction().getColumn(),
ExpressionType.toColumnType(exprEval.type()),
exprEval.valueOrDefault(),
null
);
}
}
}
return toExpressionFilter(plannerContext, getDruidFunctionName(), druidExpressions);
}

View File

@ -53,6 +53,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
import org.apache.druid.query.filter.ArrayContainsElementFilter;
import org.apache.druid.query.filter.ExpressionDimFilter;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.LikeDimFilter;
@ -1091,7 +1092,11 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.dataSource(DATA_SOURCE_ARRAYS)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(
expressionFilter("array_contains(\"arrayStringNulls\",array('a','b'))")
and(
new ArrayContainsElementFilter("arrayStringNulls", ColumnType.STRING, "a", null),
new ArrayContainsElementFilter("arrayStringNulls", ColumnType.STRING, "b", null)
)
)
.columns("arrayStringNulls")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
@ -1117,7 +1122,10 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.dataSource(DATA_SOURCE_ARRAYS)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(
expressionFilter("array_contains(\"arrayLongNulls\",array(1,null))")
and(
new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, 1L, null),
new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, null, null)
)
)
.columns("arrayLongNulls")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
@ -1142,7 +1150,10 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.dataSource(DATA_SOURCE_ARRAYS)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(
expressionFilter("array_contains(\"arrayDoubleNulls\",array(1.1,null))")
and(
new ArrayContainsElementFilter("arrayDoubleNulls", ColumnType.DOUBLE, 1.1, null),
new ArrayContainsElementFilter("arrayDoubleNulls", ColumnType.DOUBLE, null, null)
)
)
.columns("arrayDoubleNulls")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
@ -6980,7 +6991,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(
expressionFilter("array_contains(\"arrayLongNulls\",array(2))")
new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, 2L, null)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)

View File

@ -49,6 +49,7 @@ import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.filter.ArrayContainsElementFilter;
import org.apache.druid.query.filter.EqualityFilter;
import org.apache.druid.query.filter.ExpressionDimFilter;
import org.apache.druid.query.filter.InDimFilter;
@ -1506,10 +1507,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
)
.setDimFilter(
new ExpressionDimFilter(
"array_contains(\"arrayLongNulls\",1)",
queryFramework().macroTable()
)
new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, 1, null)
)
.setAggregatorSpecs(
aggregators(
@ -1564,7 +1562,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.setDimFilter(
or(
expressionFilter("array_contains(\"arrayLongNulls\",1)"),
new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, 1L, null),
expressionFilter("array_overlap(\"arrayLongNulls\",array(2,3))")
)
)
@ -1775,10 +1773,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
)
.setDimFilter(
new ExpressionDimFilter(
"array_contains(\"arrayStringNulls\",'b')",
queryFramework().macroTable()
)
new ArrayContainsElementFilter("arrayStringNulls", ColumnType.STRING, "b", null)
)
.setAggregatorSpecs(
aggregators(
@ -1994,10 +1989,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
)
.setDimFilter(
new ExpressionDimFilter(
"array_contains(\"arrayDoubleNulls\",2.2)",
queryFramework().macroTable()
)
new ArrayContainsElementFilter("arrayDoubleNulls", ColumnType.DOUBLE, 2.2, null)
)
.setAggregatorSpecs(
aggregators(