mirror of https://github.com/apache/druid.git
add native 'array contains element' filter (#15366)
* add native arrayContainsElement filter to use array column element indexes
This commit is contained in:
parent
0a56c87e93
commit
64fcb32bcf
|
@ -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"})
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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())
|
||||
);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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),
|
||||
|
|
|
@ -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")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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) {
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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(
|
||||
|
|
Loading…
Reference in New Issue