mirror of https://github.com/apache/druid.git
fix issues with equality and range filters matching double values to long typed inputs (#14654)
* fix issues with equality and range filters matching double values to long typed inputs * adjust to ensure we never homogenize null, [], and [null] into [null] for expressions on real array columns
This commit is contained in:
parent
9a9038c7ae
commit
d406bafdfc
|
@ -644,6 +644,37 @@ public abstract class ExprEval<T>
|
||||||
return rv;
|
return rv;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cast an {@link ExprEval} to some {@link ExpressionType} that the value will be compared with. If the value is not
|
||||||
|
* appropriate to use for comparison after casting, this method returns null. For example, the
|
||||||
|
* {@link ExpressionType#DOUBLE} value 1.1 when cast to {@link ExpressionType#LONG} becomes 1L, which is no longer
|
||||||
|
* appropriate to use for value equality comparisons, while 1.0 is valid.
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public static ExprEval<?> castForEqualityComparison(ExprEval<?> valueToCompare, ExpressionType typeToCompareWith)
|
||||||
|
{
|
||||||
|
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
|
||||||
|
// this lets us match 1.0 to 1, but not 1.1
|
||||||
|
return null;
|
||||||
|
} else if (ExpressionType.LONG_ARRAY.equals(typeToCompareWith)) {
|
||||||
|
// if comparison array is double typed, make sure the values are the same when cast to long
|
||||||
|
// this lets us match [1.0, 2.0, 3.0] to [1, 2, 3], but not [1.1, 2.2, 3.3]
|
||||||
|
final ExprEval<?> doubleCast = valueToCompare.castTo(ExpressionType.DOUBLE_ARRAY);
|
||||||
|
final ExprEval<?> castDoubleCast = cast.castTo(ExpressionType.DOUBLE_ARRAY);
|
||||||
|
if (ExpressionType.DOUBLE_ARRAY.getStrategy().compare(doubleCast.value(), castDoubleCast.value()) != 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// did value become null during cast but was not initially null?
|
||||||
|
if (valueToCompare.value() != null && cast.value() == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return cast;
|
||||||
|
}
|
||||||
|
|
||||||
// Cached String values
|
// Cached String values
|
||||||
private boolean stringValueCached = false;
|
private boolean stringValueCached = false;
|
||||||
@Nullable
|
@Nullable
|
||||||
|
|
|
@ -247,7 +247,7 @@ public class ExpressionTypeConversion
|
||||||
{
|
{
|
||||||
final ExpressionType functionType = ExpressionTypeConversion.function(type, other);
|
final ExpressionType functionType = ExpressionTypeConversion.function(type, other);
|
||||||
// any number is long
|
// any number is long
|
||||||
return functionType != null && functionType.isNumeric() ? ExpressionType.LONG : functionType;
|
return Types.isNumeric(functionType) ? ExpressionType.LONG : functionType;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
|
@ -53,6 +53,7 @@ import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.TypeSignature;
|
import org.apache.druid.segment.column.TypeSignature;
|
||||||
import org.apache.druid.segment.column.TypeStrategy;
|
import org.apache.druid.segment.column.TypeStrategy;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
|
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||||
import org.apache.druid.segment.filter.Filters;
|
import org.apache.druid.segment.filter.Filters;
|
||||||
import org.apache.druid.segment.filter.PredicateValueMatcherFactory;
|
import org.apache.druid.segment.filter.PredicateValueMatcherFactory;
|
||||||
import org.apache.druid.segment.filter.ValueMatchers;
|
import org.apache.druid.segment.filter.ValueMatchers;
|
||||||
|
@ -384,7 +385,10 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
return typeDetectingArrayPredicateSupplier.get();
|
return typeDetectingArrayPredicateSupplier.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
return arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType));
|
return new FallbackPredicate<>(
|
||||||
|
arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType)),
|
||||||
|
ExpressionType.fromColumnTypeStrict(arrayType)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -395,19 +399,24 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
|
|
||||||
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
||||||
{
|
{
|
||||||
return Suppliers.memoize(() -> Predicates.equalTo(matchValue.castTo(ExpressionType.STRING).asString()));
|
return Suppliers.memoize(() -> {
|
||||||
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.STRING);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return Predicates.alwaysFalse();
|
||||||
|
}
|
||||||
|
return Predicates.equalTo(castForComparison.asString());
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
|
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
|
||||||
{
|
{
|
||||||
return Suppliers.memoize(() -> {
|
return Suppliers.memoize(() -> {
|
||||||
final Long valueAsLong = (Long) matchValue.castTo(ExpressionType.LONG).valueOrDefault();
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.LONG);
|
||||||
|
if (castForComparison == null) {
|
||||||
if (valueAsLong == null) {
|
|
||||||
return DruidLongPredicate.ALWAYS_FALSE;
|
return DruidLongPredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
// store the primitive, so we don't unbox for every comparison
|
// store the primitive, so we don't unbox for every comparison
|
||||||
final long unboxedLong = valueAsLong;
|
final long unboxedLong = castForComparison.asLong();
|
||||||
return input -> input == unboxedLong;
|
return input -> input == unboxedLong;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -416,13 +425,12 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
||||||
{
|
{
|
||||||
return Suppliers.memoize(() -> {
|
return Suppliers.memoize(() -> {
|
||||||
final Double doubleValue = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault();
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||||
|
if (castForComparison == null) {
|
||||||
if (doubleValue == null) {
|
|
||||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
// Compare with floatToIntBits instead of == to canonicalize NaNs.
|
// Compare with floatToIntBits instead of == to canonicalize NaNs.
|
||||||
final int floatBits = Float.floatToIntBits(doubleValue.floatValue());
|
final int floatBits = Float.floatToIntBits((float) castForComparison.asDouble());
|
||||||
return input -> Float.floatToIntBits(input) == floatBits;
|
return input -> Float.floatToIntBits(input) == floatBits;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -431,13 +439,12 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
||||||
{
|
{
|
||||||
return Suppliers.memoize(() -> {
|
return Suppliers.memoize(() -> {
|
||||||
final Double aDouble = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault();
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||||
|
if (castForComparison == null) {
|
||||||
if (aDouble == null) {
|
|
||||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
// Compare with doubleToLongBits instead of == to canonicalize NaNs.
|
// Compare with doubleToLongBits instead of == to canonicalize NaNs.
|
||||||
final long bits = Double.doubleToLongBits(aDouble);
|
final long bits = Double.doubleToLongBits(castForComparison.asDouble());
|
||||||
return input -> Double.doubleToLongBits(input) == bits;
|
return input -> Double.doubleToLongBits(input) == bits;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -458,7 +465,11 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
return Suppliers.memoize(() -> input -> {
|
return Suppliers.memoize(() -> input -> {
|
||||||
final ExprEval<?> eval = ExprEval.bestEffortOf(input);
|
final ExprEval<?> eval = ExprEval.bestEffortOf(input);
|
||||||
final Comparator<Object[]> arrayComparator = eval.type().getNullableStrategy();
|
final Comparator<Object[]> arrayComparator = eval.type().getNullableStrategy();
|
||||||
final Object[] matchArray = matchValue.castTo(eval.type()).asArray();
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, eval.type());
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final Object[] matchArray = castForComparison.asArray();
|
||||||
return arrayComparator.compare(input, matchArray) == 0;
|
return arrayComparator.compare(input, matchArray) == 0;
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -466,7 +477,12 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
{
|
{
|
||||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
|
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
|
||||||
final Comparator<Object[]> arrayComparator = arrayType.getNullableStrategy();
|
final Comparator<Object[]> arrayComparator = arrayType.getNullableStrategy();
|
||||||
final Object[] matchArray = matchValue.castTo(expressionType).asArray();
|
|
||||||
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, expressionType);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return Predicates.alwaysFalse();
|
||||||
|
}
|
||||||
|
final Object[] matchArray = castForComparison.asArray();
|
||||||
return input -> arrayComparator.compare(input, matchArray) == 0;
|
return input -> arrayComparator.compare(input, matchArray) == 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -520,29 +536,41 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||||
{
|
{
|
||||||
return ValueMatchers.makeStringValueMatcher(
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.STRING);
|
||||||
selector,
|
if (castForComparison == null) {
|
||||||
matchValue.castTo(ExpressionType.STRING).asString(),
|
return BooleanValueMatcher.of(false);
|
||||||
multiValue
|
}
|
||||||
);
|
return ValueMatchers.makeStringValueMatcher(selector, castForComparison.asString(), multiValue);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
|
public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
|
||||||
{
|
{
|
||||||
return ValueMatchers.makeFloatValueMatcher(selector, (float) matchValue.castTo(ExpressionType.DOUBLE).asDouble());
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
return ValueMatchers.makeFloatValueMatcher(selector, (float) castForComparison.asDouble());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
|
public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
|
||||||
{
|
{
|
||||||
return ValueMatchers.makeDoubleValueMatcher(selector, matchValue.castTo(ExpressionType.DOUBLE).asDouble());
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
return ValueMatchers.makeDoubleValueMatcher(selector, castForComparison.asDouble());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
|
public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
|
||||||
{
|
{
|
||||||
return ValueMatchers.makeLongValueMatcher(selector, matchValue.castTo(ExpressionType.LONG).asLong());
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.LONG);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
return ValueMatchers.makeLongValueMatcher(selector, castForComparison.asLong());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,66 @@
|
||||||
|
/*
|
||||||
|
* 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.google.common.base.Predicate;
|
||||||
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link Predicate} that wraps another predicate and can catch {@link ClassCastException} from some
|
||||||
|
* {@link java.util.Comparator} and degrades into casting input to an expected {@link ExpressionType} once an exception
|
||||||
|
* is encountered. Useful when processing data that might be mixed types, despite what the column capabilities might
|
||||||
|
* claim the type is, such as variant 'auto' types. This class is not thread-safe.
|
||||||
|
*/
|
||||||
|
public class FallbackPredicate<T> implements Predicate<T>
|
||||||
|
{
|
||||||
|
private final Predicate<T> delegate;
|
||||||
|
private final ExpressionType expectedType;
|
||||||
|
private boolean needsCast = false;
|
||||||
|
|
||||||
|
public FallbackPredicate(Predicate<T> delegate, ExpressionType expectedType)
|
||||||
|
{
|
||||||
|
this.delegate = delegate;
|
||||||
|
this.expectedType = expectedType;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean apply(@Nullable T input)
|
||||||
|
{
|
||||||
|
if (needsCast) {
|
||||||
|
return castApply(input);
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
return delegate.apply(input);
|
||||||
|
}
|
||||||
|
catch (ClassCastException caster) {
|
||||||
|
needsCast = true;
|
||||||
|
return castApply(input);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean castApply(@Nullable T input)
|
||||||
|
{
|
||||||
|
final ExprEval<T> castEval = ExprEval.bestEffortOf(input).castTo(expectedType);
|
||||||
|
return delegate.apply(castEval.value());
|
||||||
|
}
|
||||||
|
}
|
|
@ -34,6 +34,7 @@ import com.google.common.collect.TreeRangeSet;
|
||||||
import org.apache.druid.error.InvalidInput;
|
import org.apache.druid.error.InvalidInput;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
import org.apache.druid.math.expr.Evals;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExpressionType;
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||||
|
@ -48,6 +49,7 @@ import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.TypeSignature;
|
import org.apache.druid.segment.column.TypeSignature;
|
||||||
import org.apache.druid.segment.column.TypeStrategy;
|
import org.apache.druid.segment.column.TypeStrategy;
|
||||||
|
import org.apache.druid.segment.column.Types;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
import org.apache.druid.segment.filter.Filters;
|
import org.apache.druid.segment.filter.Filters;
|
||||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||||
|
@ -514,10 +516,18 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
if (hasLowerBound()) {
|
if (hasLowerBound()) {
|
||||||
ExprEval<?> lowerCast = lowerEval.castTo(ExpressionType.LONG);
|
ExprEval<?> lowerCast = lowerEval.castTo(ExpressionType.LONG);
|
||||||
if (lowerCast.isNumericNull()) {
|
if (lowerCast.isNumericNull()) {
|
||||||
hasLowerBound = false;
|
// lower value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||||
lowerBound = Long.MIN_VALUE;
|
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||||
|
// but just in case...
|
||||||
|
return DruidLongPredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
lowerBound = lowerCast.asLong();
|
if (lowerOpen) {
|
||||||
|
// lower bound is open, so take the floor of the value so that x > 1.1 can match 2 but not 1
|
||||||
|
lowerBound = (long) Math.floor(lowerEval.asDouble());
|
||||||
|
} else {
|
||||||
|
// lower bound is closed, tkae the ceil of the value so that x >= 1.1 can match 2 but not 1
|
||||||
|
lowerBound = (long) Math.ceil(lowerEval.asDouble());
|
||||||
|
}
|
||||||
hasLowerBound = true;
|
hasLowerBound = true;
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
@ -529,10 +539,18 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
ExprEval<?> upperCast = upperEval.castTo(ExpressionType.LONG);
|
ExprEval<?> upperCast = upperEval.castTo(ExpressionType.LONG);
|
||||||
if (upperCast.isNumericNull()) {
|
if (upperCast.isNumericNull()) {
|
||||||
// upper value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
// upper value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||||
|
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||||
|
// but just in case...
|
||||||
return DruidLongPredicate.ALWAYS_FALSE;
|
return DruidLongPredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
|
if (upperOpen) {
|
||||||
|
// upper bound is open, take the ceil so that x < 1.1 can match 1 but not 2
|
||||||
|
upperBound = (long) Math.ceil(upperEval.asDouble());
|
||||||
|
} else {
|
||||||
|
// upper bound is closed, take the floor so that x <= 1.1 can match 1 but not 2
|
||||||
|
upperBound = (long) Math.floor(upperEval.asDouble());
|
||||||
|
}
|
||||||
hasUpperBound = true;
|
hasUpperBound = true;
|
||||||
upperBound = upperCast.asLong();
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
hasUpperBound = false;
|
hasUpperBound = false;
|
||||||
|
@ -562,8 +580,10 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
if (hasLowerBound()) {
|
if (hasLowerBound()) {
|
||||||
ExprEval<?> lowerCast = lowerEval.castTo(ExpressionType.DOUBLE);
|
ExprEval<?> lowerCast = lowerEval.castTo(ExpressionType.DOUBLE);
|
||||||
if (lowerCast.isNumericNull()) {
|
if (lowerCast.isNumericNull()) {
|
||||||
hasLowerBound = false;
|
// lower value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||||
lowerBound = Double.NEGATIVE_INFINITY;
|
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||||
|
// but just in case...
|
||||||
|
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
lowerBound = lowerCast.asDouble();
|
lowerBound = lowerCast.asDouble();
|
||||||
hasLowerBound = true;
|
hasLowerBound = true;
|
||||||
|
@ -577,6 +597,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
ExprEval<?> upperCast = upperEval.castTo(ExpressionType.DOUBLE);
|
ExprEval<?> upperCast = upperEval.castTo(ExpressionType.DOUBLE);
|
||||||
if (upperCast.isNumericNull()) {
|
if (upperCast.isNumericNull()) {
|
||||||
// upper value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
// upper value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||||
|
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||||
|
// but just in case...
|
||||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||||
} else {
|
} else {
|
||||||
hasUpperBound = true;
|
hasUpperBound = true;
|
||||||
|
@ -595,9 +617,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
||||||
{
|
{
|
||||||
return Suppliers.memoize(() -> {
|
return Suppliers.memoize(() -> {
|
||||||
final Comparator<String> stringComparator = matchValueType.isNumeric()
|
final Comparator<String> stringComparator =
|
||||||
? StringComparators.NUMERIC
|
matchValueType.isNumeric() ? StringComparators.NUMERIC : StringComparators.LEXICOGRAPHIC;
|
||||||
: StringComparators.LEXICOGRAPHIC;
|
|
||||||
final String lowerBound = hasLowerBound() ? lowerEval.castTo(ExpressionType.STRING).asString() : null;
|
final String lowerBound = hasLowerBound() ? lowerEval.castTo(ExpressionType.STRING).asString() : null;
|
||||||
final String upperBound = hasUpperBound() ? upperEval.castTo(ExpressionType.STRING).asString() : null;
|
final String upperBound = hasUpperBound() ? upperEval.castTo(ExpressionType.STRING).asString() : null;
|
||||||
|
|
||||||
|
@ -607,15 +628,37 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
private Predicate<Object[]> makeArrayPredicate(TypeSignature<ValueType> inputType)
|
private Predicate<Object[]> makeArrayPredicate(TypeSignature<ValueType> inputType)
|
||||||
{
|
{
|
||||||
final Comparator<Object[]> arrayComparator = inputType.getNullableStrategy();
|
final Comparator<Object[]> arrayComparator;
|
||||||
|
if (inputType.getElementType().is(ValueType.STRING) && Types.isNumericOrNumericArray(matchValueType)) {
|
||||||
|
arrayComparator = new NumericStringArrayComparator();
|
||||||
|
} else {
|
||||||
|
arrayComparator = inputType.getNullableStrategy();
|
||||||
|
}
|
||||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(inputType);
|
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(inputType);
|
||||||
final RangeType rangeType = RangeType.of(hasLowerBound(), lowerOpen, hasUpperBound(), upperOpen);
|
final RangeType rangeType = RangeType.of(hasLowerBound(), lowerOpen, hasUpperBound(), upperOpen);
|
||||||
final Object[] lowerBound = hasLowerBound() ? lowerEval.castTo(expressionType).asArray() : null;
|
|
||||||
final Object[] upperBound = hasUpperBound() ? upperEval.castTo(expressionType).asArray() : null;
|
final Object[] lowerBound;
|
||||||
|
final Object[] upperBound;
|
||||||
|
if (hasLowerBound()) {
|
||||||
|
if (lowerOpen) {
|
||||||
|
lowerBound = lowerEval.castTo(expressionType).asArray();
|
||||||
|
} else {
|
||||||
|
lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, expressionType);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
lowerBound = null;
|
||||||
|
}
|
||||||
|
if (hasUpperBound()) {
|
||||||
|
if (upperOpen) {
|
||||||
|
upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, expressionType);
|
||||||
|
} else {
|
||||||
|
upperBound = upperEval.castTo(expressionType).asArray();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
upperBound = null;
|
||||||
|
}
|
||||||
return makeComparatorPredicate(rangeType, arrayComparator, lowerBound, upperBound);
|
return makeComparatorPredicate(rangeType, arrayComparator, lowerBound, upperBound);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -631,7 +674,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
}
|
}
|
||||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
||||||
final Object[] upperBound = upperEval.castTo(val.type()).asArray();
|
final Object[] upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, val.asArrayType());
|
||||||
final Comparator<Object[]> comparator = val.type().getNullableStrategy();
|
final Comparator<Object[]> comparator = val.type().getNullableStrategy();
|
||||||
final int lowerComparing = comparator.compare(val.asArray(), lowerBound);
|
final int lowerComparing = comparator.compare(val.asArray(), lowerBound);
|
||||||
final int upperComparing = comparator.compare(upperBound, val.asArray());
|
final int upperComparing = comparator.compare(upperBound, val.asArray());
|
||||||
|
@ -656,8 +699,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType());
|
||||||
final Object[] upperBound = upperEval.castTo(val.type()).asArray();
|
final Object[] upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, val.asArrayType());
|
||||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||||
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
||||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||||
|
@ -669,7 +712,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType());
|
||||||
final Object[] upperBound = upperEval.castTo(val.type()).asArray();
|
final Object[] upperBound = upperEval.castTo(val.type()).asArray();
|
||||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||||
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
||||||
|
@ -682,7 +725,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||||
final Object[] upperBound = upperEval.castTo(val.type()).asArray();
|
final Object[] upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, val.asArrayType());
|
||||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||||
return upperComparing > 0;
|
return upperComparing > 0;
|
||||||
|
@ -715,7 +758,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType());
|
||||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||||
final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray());
|
final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray());
|
||||||
return lowerComparing >= 0;
|
return lowerComparing >= 0;
|
||||||
|
@ -739,7 +782,10 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
@Override
|
@Override
|
||||||
public Predicate<String> makeStringPredicate()
|
public Predicate<String> makeStringPredicate()
|
||||||
{
|
{
|
||||||
return stringPredicateSupplier.get();
|
return new FallbackPredicate<>(
|
||||||
|
stringPredicateSupplier.get(),
|
||||||
|
ExpressionType.STRING
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -748,8 +794,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
if (matchValueType.isNumeric()) {
|
if (matchValueType.isNumeric()) {
|
||||||
return longPredicateSupplier.get();
|
return longPredicateSupplier.get();
|
||||||
}
|
}
|
||||||
Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
Predicate<String> stringPredicate = makeStringPredicate();
|
||||||
return input -> stringPredicate.apply(String.valueOf(input));
|
return input -> stringPredicate.apply(Evals.asString(input));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -758,8 +804,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
if (matchValueType.isNumeric()) {
|
if (matchValueType.isNumeric()) {
|
||||||
return floatPredicateSupplier.get();
|
return floatPredicateSupplier.get();
|
||||||
}
|
}
|
||||||
Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
Predicate<String> stringPredicate = makeStringPredicate();
|
||||||
return input -> stringPredicate.apply(String.valueOf(input));
|
return input -> stringPredicate.apply(Evals.asString(input));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -768,8 +814,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
if (matchValueType.isNumeric()) {
|
if (matchValueType.isNumeric()) {
|
||||||
return doublePredicateSupplier.get();
|
return doublePredicateSupplier.get();
|
||||||
}
|
}
|
||||||
Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
Predicate<String> stringPredicate = makeStringPredicate();
|
||||||
return input -> stringPredicate.apply(String.valueOf(input));
|
return input -> stringPredicate.apply(Evals.asString(input));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -778,9 +824,9 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
if (inputType == null) {
|
if (inputType == null) {
|
||||||
return typeDetectingArrayPredicateSupplier.get();
|
return typeDetectingArrayPredicateSupplier.get();
|
||||||
}
|
}
|
||||||
return arrayPredicates.computeIfAbsent(
|
return new FallbackPredicate<>(
|
||||||
inputType,
|
arrayPredicates.computeIfAbsent(inputType, (existing) -> RangeFilter.this.makeArrayPredicate(inputType)),
|
||||||
(existing) -> RangeFilter.this.makeArrayPredicate(inputType)
|
ExpressionType.fromColumnTypeStrict(inputType)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -812,6 +858,27 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method is like {@link ExprEval#castTo(ExpressionType)} and {@link ExprEval#asArray()}, but when the target
|
||||||
|
* type is {@link ExpressionType#LONG_ARRAY}, the array elements are treated as decimals and passed to
|
||||||
|
* {@link Math#ceil(double)} before converting to a LONG instead of the typical flooring that would happen when
|
||||||
|
* casting.
|
||||||
|
*/
|
||||||
|
private static Object[] castArrayForComparisonWithCeilIfNeeded(ExprEval<?> valueToCast, ExpressionType typeToCastTo)
|
||||||
|
{
|
||||||
|
if (ExpressionType.LONG_ARRAY.equals(typeToCastTo)) {
|
||||||
|
final ExprEval<?> doubleArray = valueToCast.castTo(ExpressionType.DOUBLE_ARRAY);
|
||||||
|
final Object[] o = doubleArray.asArray();
|
||||||
|
final Object[] ceilArray = new Object[o.length];
|
||||||
|
for (int i = 0; i < o.length; i++) {
|
||||||
|
ceilArray[i] = o[i] == null ? null : (long) Math.ceil((Double) o[i]);
|
||||||
|
}
|
||||||
|
return ceilArray;
|
||||||
|
} else {
|
||||||
|
return valueToCast.castTo(typeToCastTo).asArray();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static DruidLongPredicate makeLongPredicate(
|
public static DruidLongPredicate makeLongPredicate(
|
||||||
final RangeType rangeType,
|
final RangeType rangeType,
|
||||||
final long lowerLongBound,
|
final long lowerLongBound,
|
||||||
|
@ -1035,4 +1102,31 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
||||||
return UNBOUNDED;
|
return UNBOUNDED;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class NumericStringArrayComparator implements Comparator<Object[]>
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int compare(Object[] o1, Object[] o2)
|
||||||
|
{
|
||||||
|
//noinspection ArrayEquality
|
||||||
|
if (o1 == o2) {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
if (o1 == null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
if (o2 == null) {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
final int iter = Math.min(o1.length, o2.length);
|
||||||
|
for (int i = 0; i < iter; i++) {
|
||||||
|
final int cmp = StringComparators.NUMERIC.compare((String) o1[i], (String) o2[i]);
|
||||||
|
if (cmp == 0) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
return cmp;
|
||||||
|
}
|
||||||
|
return Integer.compare(o1.length, o2.length);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,7 +51,7 @@ public class ArrayVectorValueMatcher implements VectorValueMatcherFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException(
|
throw new UnsupportedOperationException(
|
||||||
"Vectorized matcher cannot make object matcher for ARRAY types"
|
"Vectorized matcher cannot make object matcher for ARRAY types"
|
||||||
|
|
|
@ -55,14 +55,17 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||||
ExprEval<?> cast = eval.castTo(ExpressionType.DOUBLE);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE);
|
||||||
if (cast.isNumericNull()) {
|
if (castForComparison == null) {
|
||||||
|
return BooleanVectorValueMatcher.of(selector, false);
|
||||||
|
}
|
||||||
|
if (castForComparison.isNumericNull()) {
|
||||||
return makeNullValueMatcher(selector);
|
return makeNullValueMatcher(selector);
|
||||||
}
|
}
|
||||||
return makeDoubleMatcher(cast.asDouble());
|
return makeDoubleMatcher(castForComparison.asDouble());
|
||||||
}
|
}
|
||||||
|
|
||||||
private BaseVectorValueMatcher makeDoubleMatcher(double matchValDouble)
|
private BaseVectorValueMatcher makeDoubleMatcher(double matchValDouble)
|
||||||
|
|
|
@ -57,14 +57,17 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||||
ExprEval<?> cast = eval.castTo(ExpressionType.DOUBLE);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE);
|
||||||
if (cast.isNumericNull()) {
|
if (castForComparison == null) {
|
||||||
|
return BooleanVectorValueMatcher.of(selector, false);
|
||||||
|
}
|
||||||
|
if (castForComparison.isNumericNull()) {
|
||||||
return makeNullValueMatcher(selector);
|
return makeNullValueMatcher(selector);
|
||||||
}
|
}
|
||||||
return makeFloatMatcher((float) cast.asDouble());
|
return makeFloatMatcher((float) castForComparison.asDouble());
|
||||||
}
|
}
|
||||||
|
|
||||||
private BaseVectorValueMatcher makeFloatMatcher(float matchValFloat)
|
private BaseVectorValueMatcher makeFloatMatcher(float matchValFloat)
|
||||||
|
|
|
@ -57,14 +57,17 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||||
ExprEval<?> cast = eval.castTo(ExpressionType.LONG);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.LONG);
|
||||||
if (cast.isNumericNull()) {
|
if (castForComparison == null) {
|
||||||
|
return BooleanVectorValueMatcher.of(selector, false);
|
||||||
|
}
|
||||||
|
if (castForComparison.isNumericNull()) {
|
||||||
return makeNullValueMatcher(selector);
|
return makeNullValueMatcher(selector);
|
||||||
}
|
}
|
||||||
return makeLongMatcher(cast.asLong());
|
return makeLongMatcher(castForComparison.asLong());
|
||||||
}
|
}
|
||||||
|
|
||||||
private BaseVectorValueMatcher makeLongMatcher(long matchValLong)
|
private BaseVectorValueMatcher makeLongMatcher(long matchValLong)
|
||||||
|
|
|
@ -101,11 +101,14 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||||
ExprEval<?> cast = eval.castTo(ExpressionType.STRING);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING);
|
||||||
return makeMatcher(cast.asString());
|
if (castForComparison == null) {
|
||||||
|
return BooleanVectorValueMatcher.of(selector, false);
|
||||||
|
}
|
||||||
|
return makeMatcher(castForComparison.asString());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -54,9 +54,9 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
return BooleanVectorValueMatcher.of(selector, value == null);
|
return BooleanVectorValueMatcher.of(selector, matchValue == null);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -109,11 +109,14 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||||
ExprEval<?> cast = eval.castTo(ExpressionType.STRING);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING);
|
||||||
return makeMatcher(cast.asString());
|
if (castForComparison == null) {
|
||||||
|
return BooleanVectorValueMatcher.of(selector, false);
|
||||||
|
}
|
||||||
|
return makeMatcher(castForComparison.asString());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -70,11 +70,14 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueMatcher makeMatcher(Object value, ColumnType type)
|
public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType)
|
||||||
{
|
{
|
||||||
ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value);
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue);
|
||||||
ExprEval<?> cast = eval.castTo(ExpressionType.STRING);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING);
|
||||||
return makeMatcher(cast.asString());
|
if (castForComparison == null) {
|
||||||
|
return BooleanVectorValueMatcher.of(selector, false);
|
||||||
|
}
|
||||||
|
return makeMatcher(castForComparison.asString());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -27,9 +27,17 @@ import javax.annotation.Nullable;
|
||||||
|
|
||||||
public interface VectorValueMatcherFactory
|
public interface VectorValueMatcherFactory
|
||||||
{
|
{
|
||||||
|
/**
|
||||||
|
* Specialized value matcher for string equality used by {@link org.apache.druid.query.filter.SelectorDimFilter}
|
||||||
|
*/
|
||||||
VectorValueMatcher makeMatcher(@Nullable String value);
|
VectorValueMatcher makeMatcher(@Nullable String value);
|
||||||
|
|
||||||
VectorValueMatcher makeMatcher(Object value, ColumnType type);
|
/**
|
||||||
|
* Specialized value matcher for equality used by {@link org.apache.druid.query.filter.EqualityFilter}. The
|
||||||
|
* matchValue parameter must be the appropriate Java type for the matchValueType {@link ColumnType}. Implementors can
|
||||||
|
* use this information to coerce the match value to the native type of the values to match against as necessary.
|
||||||
|
*/
|
||||||
|
VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType);
|
||||||
|
|
||||||
VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory);
|
VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory);
|
||||||
|
|
||||||
|
|
|
@ -19,9 +19,11 @@
|
||||||
|
|
||||||
package org.apache.druid.segment;
|
package org.apache.druid.segment;
|
||||||
|
|
||||||
|
import com.google.common.primitives.Doubles;
|
||||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||||
import org.apache.druid.collections.bitmap.MutableBitmap;
|
import org.apache.druid.collections.bitmap.MutableBitmap;
|
||||||
import org.apache.druid.common.config.NullHandling;
|
import org.apache.druid.common.config.NullHandling;
|
||||||
|
import org.apache.druid.common.guava.GuavaUtils;
|
||||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
|
@ -489,37 +491,37 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
||||||
public boolean isNull()
|
public boolean isNull()
|
||||||
{
|
{
|
||||||
final Object o = getObject();
|
final Object o = getObject();
|
||||||
return !(o instanceof Number);
|
return computeNumber(o) == null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public float getFloat()
|
public float getFloat()
|
||||||
{
|
{
|
||||||
Object value = getObject();
|
Number value = computeNumber(getObject());
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return ((Number) value).floatValue();
|
return value.floatValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public double getDouble()
|
public double getDouble()
|
||||||
{
|
{
|
||||||
Object value = getObject();
|
Number value = computeNumber(getObject());
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return ((Number) value).doubleValue();
|
return value.doubleValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getLong()
|
public long getLong()
|
||||||
{
|
{
|
||||||
Object value = getObject();
|
Number value = computeNumber(getObject());
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return ((Number) value).longValue();
|
return value.longValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -544,6 +546,22 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
||||||
return defaultValue;
|
return defaultValue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
private Number computeNumber(@Nullable Object o)
|
||||||
|
{
|
||||||
|
if (o instanceof Number) {
|
||||||
|
return (Number) o;
|
||||||
|
}
|
||||||
|
if (o instanceof String) {
|
||||||
|
Long l = GuavaUtils.tryParseLong((String) o);
|
||||||
|
if (l != null) {
|
||||||
|
return l;
|
||||||
|
}
|
||||||
|
return Doubles.tryParse((String) o);
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Class<?> classOfObject()
|
public Class<?> classOfObject()
|
||||||
{
|
{
|
||||||
|
|
|
@ -114,6 +114,26 @@ public class Types
|
||||||
(typeSignature2 != null && typeSignature2.is(typeDescriptor));
|
(typeSignature2 != null && typeSignature2.is(typeDescriptor));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if {@link TypeSignature} is not null and is {@link TypeSignature#isNumeric()}
|
||||||
|
*/
|
||||||
|
public static <T extends TypeDescriptor> boolean isNumeric(@Nullable TypeSignature<T> typeSignature)
|
||||||
|
{
|
||||||
|
return typeSignature != null && typeSignature.isNumeric();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if {@link TypeSignature} is not null and is {@link TypeSignature#isNumeric()} or has
|
||||||
|
* {@link TypeSignature#getElementType()} that is numeric.
|
||||||
|
*/
|
||||||
|
public static <T extends TypeDescriptor> boolean isNumericOrNumericArray(@Nullable TypeSignature<T> typeSignature)
|
||||||
|
{
|
||||||
|
if (typeSignature == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return typeSignature.isNumeric() || (typeSignature.isArray() && typeSignature.getElementType().isNumeric());
|
||||||
|
}
|
||||||
|
|
||||||
public static class IncompatibleTypeException extends IAE
|
public static class IncompatibleTypeException extends IAE
|
||||||
{
|
{
|
||||||
public IncompatibleTypeException(TypeSignature<?> type, TypeSignature<?> other)
|
public IncompatibleTypeException(TypeSignature<?> type, TypeSignature<?> other)
|
||||||
|
|
|
@ -729,10 +729,26 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
||||||
boolean endStrict
|
boolean endStrict
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
final Long startLong;
|
||||||
|
final Long endLong;
|
||||||
|
if (startValue == null) {
|
||||||
|
startLong = null;
|
||||||
|
} else if (startStrict) {
|
||||||
|
startLong = (long) Math.floor(startValue.doubleValue());
|
||||||
|
} else {
|
||||||
|
startLong = (long) Math.ceil(startValue.doubleValue());
|
||||||
|
}
|
||||||
|
if (endValue == null) {
|
||||||
|
endLong = null;
|
||||||
|
} else if (endStrict) {
|
||||||
|
endLong = (long) Math.ceil(endValue.doubleValue());
|
||||||
|
} else {
|
||||||
|
endLong = (long) Math.floor(endValue.doubleValue());
|
||||||
|
}
|
||||||
return makeRangeIndex(
|
return makeRangeIndex(
|
||||||
startValue != null ? startValue.longValue() : null,
|
startLong,
|
||||||
startStrict,
|
startStrict,
|
||||||
endValue != null ? endValue.longValue() : null,
|
endLong,
|
||||||
endStrict,
|
endStrict,
|
||||||
localDictionarySupplier.get(),
|
localDictionarySupplier.get(),
|
||||||
globalLongDictionarySupplier.get(),
|
globalLongDictionarySupplier.get(),
|
||||||
|
|
|
@ -28,6 +28,8 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import org.apache.druid.common.guava.GuavaUtils;
|
import org.apache.druid.common.guava.GuavaUtils;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.UOE;
|
import org.apache.druid.java.util.common.UOE;
|
||||||
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
import org.apache.druid.query.extraction.ExtractionFn;
|
import org.apache.druid.query.extraction.ExtractionFn;
|
||||||
import org.apache.druid.query.filter.ValueMatcher;
|
import org.apache.druid.query.filter.ValueMatcher;
|
||||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
@ -77,6 +79,8 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
||||||
private final FieldTypeInfo.TypeSet types;
|
private final FieldTypeInfo.TypeSet types;
|
||||||
@Nullable
|
@Nullable
|
||||||
private final ColumnType singleType;
|
private final ColumnType singleType;
|
||||||
|
private final ColumnType logicalType;
|
||||||
|
private final ExpressionType logicalExpressionType;
|
||||||
private final ColumnarLongs longsColumn;
|
private final ColumnarLongs longsColumn;
|
||||||
private final ColumnarDoubles doublesColumn;
|
private final ColumnarDoubles doublesColumn;
|
||||||
private final ColumnarInts column;
|
private final ColumnarInts column;
|
||||||
|
@ -108,6 +112,12 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.types = types;
|
this.types = types;
|
||||||
|
ColumnType leastRestrictive = null;
|
||||||
|
for (ColumnType type : FieldTypeInfo.convertToSet(types.getByteValue())) {
|
||||||
|
leastRestrictive = ColumnType.leastRestrictiveType(leastRestrictive, type);
|
||||||
|
}
|
||||||
|
this.logicalType = leastRestrictive;
|
||||||
|
this.logicalExpressionType = ExpressionType.fromColumnTypeStrict(logicalType);
|
||||||
this.singleType = types.getSingleType();
|
this.singleType = types.getSingleType();
|
||||||
this.longsColumn = longsColumn;
|
this.longsColumn = longsColumn;
|
||||||
this.doublesColumn = doublesColumn;
|
this.doublesColumn = doublesColumn;
|
||||||
|
@ -250,6 +260,29 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
||||||
throw new IllegalArgumentException("not a scalar in the dictionary");
|
throw new IllegalArgumentException("not a scalar in the dictionary");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Lookup value from appropriate scalar value dictionary, coercing the value to {@link #logicalType}, particularly
|
||||||
|
* useful for the vector query engine which prefers all the types are consistent
|
||||||
|
* <p>
|
||||||
|
* This method should NEVER be used when values must round trip to be able to be looked up from the array value
|
||||||
|
* dictionary since it might coerce element values to a different type
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
private Object lookupGlobalScalarValueAndCast(int globalId)
|
||||||
|
{
|
||||||
|
|
||||||
|
if (globalId == 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
if (singleType != null) {
|
||||||
|
return lookupGlobalScalarObject(globalId);
|
||||||
|
} else {
|
||||||
|
final ExprEval<?> eval = ExprEval.ofType(logicalExpressionType, lookupGlobalScalarObject(globalId));
|
||||||
|
return eval.value();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
ReadableOffset offset,
|
ReadableOffset offset,
|
||||||
|
@ -667,7 +700,17 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
||||||
if (nullMark == offsetMark) {
|
if (nullMark == offsetMark) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
return DimensionHandlerUtils.isNumericNull(getObject());
|
final int localId = column.get(offset.getOffset());
|
||||||
|
final int globalId = dictionary.get(localId);
|
||||||
|
// zero is always null
|
||||||
|
if (globalId == 0) {
|
||||||
|
return true;
|
||||||
|
} else if (globalId < adjustLongId) {
|
||||||
|
final String value = StringUtils.fromUtf8Nullable(globalDictionary.get(globalId));
|
||||||
|
return GuavaUtils.tryParseLong(value) == null && Doubles.tryParse(value) == null;
|
||||||
|
}
|
||||||
|
// if id is less than array ids, it is definitely a number and not null (since null is 0)
|
||||||
|
return globalId >= adjustArrayId;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -745,77 +788,53 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
||||||
@Override
|
@Override
|
||||||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||||
{
|
{
|
||||||
if (singleType != null && singleType.isArray()) {
|
// if the logical type is string, use simplified string vector object selector
|
||||||
return new VectorObjectSelector()
|
if (ColumnType.STRING.equals(logicalType)) {
|
||||||
|
final class StringVectorSelector extends StringUtf8DictionaryEncodedColumn.StringVectorObjectSelector
|
||||||
{
|
{
|
||||||
private final int[] vector = new int[offset.getMaxVectorSize()];
|
public StringVectorSelector()
|
||||||
private final Object[] objects = new Object[offset.getMaxVectorSize()];
|
|
||||||
private int id = ReadableVectorInspector.NULL_ID;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
|
|
||||||
public Object[] getObjectVector()
|
|
||||||
{
|
{
|
||||||
if (id == offset.getId()) {
|
super(column, offset);
|
||||||
return objects;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (offset.isContiguous()) {
|
|
||||||
column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
|
||||||
} else {
|
|
||||||
column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
|
||||||
}
|
|
||||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
|
||||||
final int globalId = dictionary.get(vector[i]);
|
|
||||||
if (globalId < adjustArrayId) {
|
|
||||||
objects[i] = lookupGlobalScalarObject(globalId);
|
|
||||||
} else {
|
|
||||||
int[] arr = globalArrayDictionary.get(globalId - adjustArrayId);
|
|
||||||
if (arr == null) {
|
|
||||||
objects[i] = null;
|
|
||||||
} else {
|
|
||||||
final Object[] array = new Object[arr.length];
|
|
||||||
for (int j = 0; j < arr.length; j++) {
|
|
||||||
array[j] = lookupGlobalScalarObject(arr[j]);
|
|
||||||
}
|
|
||||||
objects[i] = array;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
id = offset.getId();
|
|
||||||
|
|
||||||
return objects;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public int getMaxVectorSize()
|
public String lookupName(int id)
|
||||||
{
|
{
|
||||||
return offset.getMaxVectorSize();
|
return NestedFieldDictionaryEncodedColumn.this.lookupName(id);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
return new StringVectorSelector();
|
||||||
public int getCurrentVectorSize()
|
|
||||||
{
|
|
||||||
return offset.getCurrentVectorSize();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
final class StringVectorSelector extends StringUtf8DictionaryEncodedColumn.StringVectorObjectSelector
|
// mixed type, this coerces values to the logical type so that the vector engine can deal with consistently typed
|
||||||
|
// values
|
||||||
|
return new VariantColumn.VariantVectorObjectSelector(
|
||||||
|
offset,
|
||||||
|
column,
|
||||||
|
globalArrayDictionary,
|
||||||
|
logicalExpressionType,
|
||||||
|
adjustArrayId
|
||||||
|
)
|
||||||
{
|
{
|
||||||
public StringVectorSelector()
|
|
||||||
{
|
|
||||||
super(column, offset);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
@Override
|
@Override
|
||||||
public String lookupName(int id)
|
public int adjustDictionaryId(int id)
|
||||||
{
|
{
|
||||||
return NestedFieldDictionaryEncodedColumn.this.lookupName(id);
|
return dictionary.get(id);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
return new StringVectorSelector();
|
@Override
|
||||||
|
public @Nullable Object lookupScalarValue(int dictionaryId)
|
||||||
|
{
|
||||||
|
return NestedFieldDictionaryEncodedColumn.this.lookupGlobalScalarObject(dictionaryId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public @Nullable Object lookupScalarValueAndCast(int dictionaryId)
|
||||||
|
{
|
||||||
|
return NestedFieldDictionaryEncodedColumn.this.lookupGlobalScalarValueAndCast(dictionaryId);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.druid.segment.data.CompressedColumnarDoublesSuppliers;
|
||||||
import org.apache.druid.segment.data.FixedIndexed;
|
import org.apache.druid.segment.data.FixedIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
import org.apache.druid.segment.data.VByte;
|
import org.apache.druid.segment.data.VByte;
|
||||||
|
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||||
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
||||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
||||||
|
@ -231,13 +232,12 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
||||||
@Override
|
@Override
|
||||||
public BitmapColumnIndex forValue(@Nonnull Object value, TypeSignature<ValueType> valueType)
|
public BitmapColumnIndex forValue(@Nonnull Object value, TypeSignature<ValueType> valueType)
|
||||||
{
|
{
|
||||||
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value)
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value);
|
||||||
.castTo(ExpressionType.DOUBLE);
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE);
|
||||||
if (eval.isNumericNull()) {
|
if (castForComparison == null) {
|
||||||
// value wasn't null, but not a number?
|
return new AllFalseBitmapColumnIndex(bitmapFactory);
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
final double doubleValue = eval.asDouble();
|
final double doubleValue = castForComparison.asDouble();
|
||||||
return new SimpleBitmapColumnIndex()
|
return new SimpleBitmapColumnIndex()
|
||||||
{
|
{
|
||||||
final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
|
final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.druid.segment.data.CompressedColumnarLongsSupplier;
|
||||||
import org.apache.druid.segment.data.FixedIndexed;
|
import org.apache.druid.segment.data.FixedIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
import org.apache.druid.segment.data.VByte;
|
import org.apache.druid.segment.data.VByte;
|
||||||
|
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||||
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
||||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
||||||
|
@ -231,14 +232,13 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
||||||
@Override
|
@Override
|
||||||
public BitmapColumnIndex forValue(@Nonnull Object value, TypeSignature<ValueType> valueType)
|
public BitmapColumnIndex forValue(@Nonnull Object value, TypeSignature<ValueType> valueType)
|
||||||
{
|
{
|
||||||
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value);
|
||||||
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value)
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.LONG);
|
||||||
.castTo(ExpressionType.LONG);
|
if (castForComparison == null) {
|
||||||
if (eval.isNumericNull()) {
|
return new AllFalseBitmapColumnIndex(bitmapFactory);
|
||||||
// value wasn't null, but not a number
|
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
final long longValue = eval.asLong();
|
final long longValue = castForComparison.asLong();
|
||||||
|
|
||||||
return new SimpleBitmapColumnIndex()
|
return new SimpleBitmapColumnIndex()
|
||||||
{
|
{
|
||||||
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
|
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
|
||||||
|
@ -416,12 +416,23 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
|
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
|
||||||
IntIntPair range = dictionary.getRange(
|
final Long startLong;
|
||||||
startValue == null ? null : startValue.longValue(),
|
final Long endLong;
|
||||||
startStrict,
|
if (startValue == null) {
|
||||||
endValue == null ? null : endValue.longValue(),
|
startLong = null;
|
||||||
endStrict
|
} else if (startStrict) {
|
||||||
);
|
startLong = (long) Math.floor(startValue.doubleValue());
|
||||||
|
} else {
|
||||||
|
startLong = (long) Math.ceil(startValue.doubleValue());
|
||||||
|
}
|
||||||
|
if (endValue == null) {
|
||||||
|
endLong = null;
|
||||||
|
} else if (endStrict) {
|
||||||
|
endLong = (long) Math.ceil(endValue.doubleValue());
|
||||||
|
} else {
|
||||||
|
endLong = (long) Math.floor(endValue.doubleValue());
|
||||||
|
}
|
||||||
|
final IntIntPair range = dictionary.getRange(startLong, startStrict, endLong, endStrict);
|
||||||
|
|
||||||
final int startIndex = range.leftInt();
|
final int startIndex = range.leftInt();
|
||||||
final int endIndex = range.rightInt();
|
final int endIndex = range.rightInt();
|
||||||
|
|
|
@ -367,7 +367,7 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
@Nullable ExtractionFn extractionFn
|
@Nullable ExtractionFn extractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (logicalType.isArray()) {
|
if (variantTypes == null && logicalType.isArray()) {
|
||||||
throw new IAE("Dimension selector is currently unsupported for [%s]", logicalType);
|
throw new IAE("Dimension selector is currently unsupported for [%s]", logicalType);
|
||||||
}
|
}
|
||||||
// copy everywhere all the time
|
// copy everywhere all the time
|
||||||
|
@ -628,9 +628,10 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
return f == null ? 0f : f;
|
return f == null ? 0f : f;
|
||||||
} else if (id < adjustDoubleId) {
|
} else if (id < adjustDoubleId) {
|
||||||
return longDictionary.get(id - adjustLongId).floatValue();
|
return longDictionary.get(id - adjustLongId).floatValue();
|
||||||
} else {
|
} else if (id < adjustArrayId) {
|
||||||
return doubleDictionary.get(id - adjustDoubleId).floatValue();
|
return doubleDictionary.get(id - adjustDoubleId).floatValue();
|
||||||
}
|
}
|
||||||
|
return 0L;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -646,9 +647,10 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
return d == null ? 0.0 : d;
|
return d == null ? 0.0 : d;
|
||||||
} else if (id < adjustDoubleId) {
|
} else if (id < adjustDoubleId) {
|
||||||
return longDictionary.get(id - adjustLongId).doubleValue();
|
return longDictionary.get(id - adjustLongId).doubleValue();
|
||||||
} else {
|
} else if (id < adjustArrayId) {
|
||||||
return doubleDictionary.get(id - adjustDoubleId);
|
return doubleDictionary.get(id - adjustDoubleId);
|
||||||
}
|
}
|
||||||
|
return 0.0;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -664,8 +666,10 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
return l == null ? 0L : l;
|
return l == null ? 0L : l;
|
||||||
} else if (id < adjustDoubleId) {
|
} else if (id < adjustDoubleId) {
|
||||||
return longDictionary.get(id - adjustLongId);
|
return longDictionary.get(id - adjustLongId);
|
||||||
} else {
|
} else if (id < adjustArrayId) {
|
||||||
return doubleDictionary.get(id - adjustDoubleId).longValue();
|
return doubleDictionary.get(id - adjustDoubleId).longValue();
|
||||||
|
} else {
|
||||||
|
return 0L;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -688,7 +692,16 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
if (nullMark == offsetMark) {
|
if (nullMark == offsetMark) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
return DimensionHandlerUtils.isNumericNull(getObject());
|
final int id = encodedValueColumn.get(offset.getOffset());
|
||||||
|
// zero is always null
|
||||||
|
if (id == 0) {
|
||||||
|
return true;
|
||||||
|
} else if (id < adjustLongId) {
|
||||||
|
final String value = StringUtils.fromUtf8Nullable(stringDictionary.get(id));
|
||||||
|
return GuavaUtils.tryParseLong(value) == null && Doubles.tryParse(value) == null;
|
||||||
|
}
|
||||||
|
// if id is less than array ids, its definitely a number and not null (since null is 0)
|
||||||
|
return id >= adjustArrayId;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -747,57 +760,32 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
@Override
|
@Override
|
||||||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||||
{
|
{
|
||||||
return new VectorObjectSelector()
|
return new VariantVectorObjectSelector(
|
||||||
|
offset,
|
||||||
|
encodedValueColumn,
|
||||||
|
arrayDictionary,
|
||||||
|
logicalExpressionType,
|
||||||
|
adjustArrayId
|
||||||
|
)
|
||||||
{
|
{
|
||||||
private final int[] vector = new int[offset.getMaxVectorSize()];
|
|
||||||
private final Object[] objects = new Object[offset.getMaxVectorSize()];
|
|
||||||
private int offsetId = ReadableVectorInspector.NULL_ID;
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
public int adjustDictionaryId(int id)
|
||||||
public Object[] getObjectVector()
|
|
||||||
{
|
{
|
||||||
if (offsetId == offset.getId()) {
|
return id;
|
||||||
return objects;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (offset.isContiguous()) {
|
|
||||||
encodedValueColumn.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
|
||||||
} else {
|
|
||||||
encodedValueColumn.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
|
||||||
}
|
|
||||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
|
||||||
final int dictionaryId = vector[i];
|
|
||||||
if (dictionaryId < adjustArrayId) {
|
|
||||||
objects[i] = lookupScalarValueStrict(dictionaryId);
|
|
||||||
} else {
|
|
||||||
int[] arr = arrayDictionary.get(dictionaryId - adjustArrayId);
|
|
||||||
if (arr == null) {
|
|
||||||
objects[i] = null;
|
|
||||||
} else {
|
|
||||||
final Object[] array = new Object[arr.length];
|
|
||||||
for (int j = 0; j < arr.length; j++) {
|
|
||||||
array[j] = lookupScalarValue(arr[j]);
|
|
||||||
}
|
|
||||||
objects[i] = ExprEval.ofType(logicalExpressionType, array).asArray();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
offsetId = offset.getId();
|
|
||||||
|
|
||||||
return objects;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public int getMaxVectorSize()
|
public Object lookupScalarValue(int dictionaryId)
|
||||||
{
|
{
|
||||||
return offset.getMaxVectorSize();
|
return VariantColumn.this.lookupScalarValue(dictionaryId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public int getCurrentVectorSize()
|
public Object lookupScalarValueAndCast(int dictionaryId)
|
||||||
{
|
{
|
||||||
return offset.getCurrentVectorSize();
|
return VariantColumn.this.lookupScalarValueAndCast(dictionaryId);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -815,7 +803,8 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
* This method should NEVER be used when values must round trip to be able to be looked up from the array value
|
* This method should NEVER be used when values must round trip to be able to be looked up from the array value
|
||||||
* dictionary since it might coerce element values to a different type
|
* dictionary since it might coerce element values to a different type
|
||||||
*/
|
*/
|
||||||
private Object lookupScalarValueStrict(int id)
|
@Nullable
|
||||||
|
private Object lookupScalarValueAndCast(int id)
|
||||||
{
|
{
|
||||||
if (id == 0) {
|
if (id == 0) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -828,6 +817,7 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
private Object lookupScalarValue(int id)
|
private Object lookupScalarValue(int id)
|
||||||
{
|
{
|
||||||
if (id < adjustLongId) {
|
if (id < adjustLongId) {
|
||||||
|
@ -839,4 +829,90 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||||
}
|
}
|
||||||
throw new IllegalArgumentException("not a scalar in the dictionary");
|
throw new IllegalArgumentException("not a scalar in the dictionary");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Make a {@link VectorObjectSelector} for a dictionary encoded column that coerces mixed types to a common type
|
||||||
|
*/
|
||||||
|
public abstract static class VariantVectorObjectSelector implements VectorObjectSelector
|
||||||
|
{
|
||||||
|
private final int[] vector;
|
||||||
|
private final Object[] objects;
|
||||||
|
private int offsetId = ReadableVectorInspector.NULL_ID;
|
||||||
|
private final ReadableVectorOffset offset;
|
||||||
|
private final ColumnarInts encodedValueColumn;
|
||||||
|
private final FrontCodedIntArrayIndexed arrayDictionary;
|
||||||
|
private final ExpressionType logicalExpressionType;
|
||||||
|
private final int adjustArrayId;
|
||||||
|
|
||||||
|
protected VariantVectorObjectSelector(
|
||||||
|
ReadableVectorOffset offset,
|
||||||
|
ColumnarInts encodedValueColumn,
|
||||||
|
FrontCodedIntArrayIndexed arrayDictionary,
|
||||||
|
ExpressionType logicalExpressionType,
|
||||||
|
int adjustArrayId
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.offset = offset;
|
||||||
|
this.encodedValueColumn = encodedValueColumn;
|
||||||
|
this.arrayDictionary = arrayDictionary;
|
||||||
|
this.logicalExpressionType = logicalExpressionType;
|
||||||
|
this.adjustArrayId = adjustArrayId;
|
||||||
|
this.objects = new Object[offset.getMaxVectorSize()];
|
||||||
|
this.vector = new int[offset.getMaxVectorSize()];
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract int adjustDictionaryId(int id);
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public abstract Object lookupScalarValue(int dictionaryId);
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public abstract Object lookupScalarValueAndCast(int dictionaryId);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object[] getObjectVector()
|
||||||
|
{
|
||||||
|
if (offsetId == offset.getId()) {
|
||||||
|
return objects;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (offset.isContiguous()) {
|
||||||
|
encodedValueColumn.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||||
|
} else {
|
||||||
|
encodedValueColumn.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||||
|
}
|
||||||
|
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||||
|
final int dictionaryId = adjustDictionaryId(vector[i]);
|
||||||
|
if (dictionaryId < adjustArrayId) {
|
||||||
|
objects[i] = lookupScalarValueAndCast(dictionaryId);
|
||||||
|
} else {
|
||||||
|
int[] arr = arrayDictionary.get(dictionaryId - adjustArrayId);
|
||||||
|
if (arr == null) {
|
||||||
|
objects[i] = null;
|
||||||
|
} else {
|
||||||
|
final Object[] array = new Object[arr.length];
|
||||||
|
for (int j = 0; j < arr.length; j++) {
|
||||||
|
array[j] = lookupScalarValue(arr[j]);
|
||||||
|
}
|
||||||
|
objects[i] = ExprEval.ofType(logicalExpressionType, array).asArray();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
offsetId = offset.getId();
|
||||||
|
|
||||||
|
return objects;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getMaxVectorSize()
|
||||||
|
{
|
||||||
|
return offset.getMaxVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCurrentVectorSize()
|
||||||
|
{
|
||||||
|
return offset.getCurrentVectorSize();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -350,9 +350,15 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
@Override
|
@Override
|
||||||
public BitmapColumnIndex forValue(@Nonnull Object value, TypeSignature<ValueType> valueType)
|
public BitmapColumnIndex forValue(@Nonnull Object value, TypeSignature<ValueType> valueType)
|
||||||
{
|
{
|
||||||
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value)
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value);
|
||||||
.castTo(ExpressionType.fromColumnTypeStrict(logicalType));
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(
|
||||||
final Object[] arrayToMatch = eval.asArray();
|
eval,
|
||||||
|
ExpressionType.fromColumnTypeStrict(logicalType)
|
||||||
|
);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return new AllFalseBitmapColumnIndex(bitmapFactory);
|
||||||
|
}
|
||||||
|
final Object[] arrayToMatch = castForComparison.asArray();
|
||||||
Indexed elements;
|
Indexed elements;
|
||||||
final int elementOffset;
|
final int elementOffset;
|
||||||
switch (logicalType.getElementType().getType()) {
|
switch (logicalType.getElementType().getType()) {
|
||||||
|
@ -425,9 +431,14 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
@Override
|
@Override
|
||||||
public BitmapColumnIndex containsValue(@Nullable Object value, TypeSignature<ValueType> elementValueType)
|
public BitmapColumnIndex containsValue(@Nullable Object value, TypeSignature<ValueType> elementValueType)
|
||||||
{
|
{
|
||||||
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(elementValueType), value)
|
final ExprEval<?> eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(elementValueType), value);
|
||||||
.castTo(ExpressionType.fromColumnTypeStrict(logicalType.getElementType()));
|
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(
|
||||||
|
eval,
|
||||||
|
ExpressionType.fromColumnTypeStrict(logicalType.getElementType())
|
||||||
|
);
|
||||||
|
if (castForComparison == null) {
|
||||||
|
return new AllFalseBitmapColumnIndex(bitmapFactory);
|
||||||
|
}
|
||||||
Indexed elements;
|
Indexed elements;
|
||||||
final int elementOffset;
|
final int elementOffset;
|
||||||
switch (logicalType.getElementType().getType()) {
|
switch (logicalType.getElementType().getType()) {
|
||||||
|
@ -475,12 +486,12 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
|
|
||||||
private int getElementId()
|
private int getElementId()
|
||||||
{
|
{
|
||||||
if (eval.value() == null) {
|
if (castForComparison.value() == null) {
|
||||||
return 0;
|
return 0;
|
||||||
} else if (eval.type().is(ExprType.STRING)) {
|
} else if (castForComparison.type().is(ExprType.STRING)) {
|
||||||
return elements.indexOf(StringUtils.toUtf8ByteBuffer(eval.asString()));
|
return elements.indexOf(StringUtils.toUtf8ByteBuffer(castForComparison.asString()));
|
||||||
} else {
|
} else {
|
||||||
return elements.indexOf(eval.value()) + elementOffset;
|
return elements.indexOf(castForComparison.value()) + elementOffset;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -211,9 +211,8 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
||||||
columnConfig
|
columnConfig
|
||||||
);
|
);
|
||||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
// technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn
|
||||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
// interface, so do not make the claim in the ColumnCapabilities
|
||||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
|
||||||
builder.setType(logicalType);
|
builder.setType(logicalType);
|
||||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||||
builder.setIndexSupplier(supplier, true, false);
|
builder.setIndexSupplier(supplier, true, false);
|
||||||
|
@ -234,9 +233,8 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
||||||
columnConfig
|
columnConfig
|
||||||
);
|
);
|
||||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
// technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn
|
||||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
// interface, so do not make the claim in the ColumnCapabilities
|
||||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
|
||||||
builder.setType(logicalType);
|
builder.setType(logicalType);
|
||||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||||
builder.setIndexSupplier(supplier, true, false);
|
builder.setIndexSupplier(supplier, true, false);
|
||||||
|
@ -259,7 +257,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
||||||
);
|
);
|
||||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||||
// if we are a mixed type, don't call ourself dictionary encoded for now so we don't end up doing the wrong thing
|
// if we are a mixed type, don't call ourself dictionary encoded for now so we don't end up doing the wrong thing
|
||||||
// in places. technically we could probably get by by indicating that our dictionary ids are not unique/sorted
|
// in places. technically we could probably get by with indicating that our dictionary ids are not unique/sorted
|
||||||
// but just in case that still causes problems, skip it all...
|
// but just in case that still causes problems, skip it all...
|
||||||
if (supplier.getVariantTypeSetByte() == null) {
|
if (supplier.getVariantTypeSetByte() == null) {
|
||||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||||
|
|
|
@ -250,7 +250,7 @@ public class ExpressionPlan
|
||||||
|
|
||||||
// null constants can sometimes trip up the type inference to report STRING, so check if explicitly supplied
|
// null constants can sometimes trip up the type inference to report STRING, so check if explicitly supplied
|
||||||
// output type is numeric and stick with that if so
|
// output type is numeric and stick with that if so
|
||||||
if (outputTypeHint != null && outputTypeHint.isNumeric()) {
|
if (Types.isNumeric(outputTypeHint)) {
|
||||||
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(outputTypeHint);
|
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(outputTypeHint);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -344,12 +344,17 @@ public class ExpressionSelectors
|
||||||
final boolean homogenizeNullMultiValueStringArrays =
|
final boolean homogenizeNullMultiValueStringArrays =
|
||||||
plan.is(ExpressionPlan.Trait.NEEDS_APPLIED) || ExpressionProcessing.isHomogenizeNullMultiValueStringArrays();
|
plan.is(ExpressionPlan.Trait.NEEDS_APPLIED) || ExpressionProcessing.isHomogenizeNullMultiValueStringArrays();
|
||||||
|
|
||||||
if (capabilities == null || capabilities.isArray() || useObjectSupplierForMultiValueStringArray) {
|
if (capabilities == null || useObjectSupplierForMultiValueStringArray) {
|
||||||
// Unknown type, array type, or output array uses an Object selector and see if that gives anything useful
|
// Unknown type, or implicitly mapped mvd, use Object selector and see if that gives anything useful
|
||||||
supplier = supplierFromObjectSelector(
|
supplier = supplierFromObjectSelector(
|
||||||
columnSelectorFactory.makeColumnValueSelector(columnName),
|
columnSelectorFactory.makeColumnValueSelector(columnName),
|
||||||
homogenizeNullMultiValueStringArrays
|
homogenizeNullMultiValueStringArrays
|
||||||
);
|
);
|
||||||
|
} else if (capabilities.isArray()) {
|
||||||
|
supplier = supplierFromObjectSelector(
|
||||||
|
columnSelectorFactory.makeColumnValueSelector(columnName),
|
||||||
|
false
|
||||||
|
);
|
||||||
} else if (capabilities.is(ValueType.FLOAT)) {
|
} else if (capabilities.is(ValueType.FLOAT)) {
|
||||||
ColumnValueSelector<?> selector = columnSelectorFactory.makeColumnValueSelector(columnName);
|
ColumnValueSelector<?> selector = columnSelectorFactory.makeColumnValueSelector(columnName);
|
||||||
supplier = makeNullableNumericSupplier(selector, selector::getFloat);
|
supplier = makeNullableNumericSupplier(selector, selector::getFloat);
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.Numbers;
|
import org.apache.druid.java.util.common.Numbers;
|
||||||
import org.apache.druid.math.expr.Evals;
|
import org.apache.druid.math.expr.Evals;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
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.cache.CacheKeyBuilder;
|
||||||
import org.apache.druid.query.dimension.DimensionSpec;
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
import org.apache.druid.query.extraction.ExtractionFn;
|
import org.apache.druid.query.extraction.ExtractionFn;
|
||||||
|
@ -508,14 +509,28 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
||||||
}
|
}
|
||||||
BaseColumn column = holder.getColumn();
|
BaseColumn column = holder.getColumn();
|
||||||
|
|
||||||
// processFromRaw is true, that means JSON_QUERY, which can return partial results, otherwise this virtual column
|
|
||||||
// is JSON_VALUE which only returns literals, so we can use the nested columns value selector
|
|
||||||
if (column instanceof NestedDataComplexColumn) {
|
if (column instanceof NestedDataComplexColumn) {
|
||||||
final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column;
|
final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column;
|
||||||
if (processFromRaw) {
|
if (processFromRaw) {
|
||||||
|
// processFromRaw is true, that means JSON_QUERY, which can return partial results, otherwise this virtual column
|
||||||
|
// is JSON_VALUE which only returns literals, so we can use the nested columns value selector
|
||||||
return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), parts);
|
return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), parts);
|
||||||
}
|
}
|
||||||
return complexColumn.makeVectorObjectSelector(parts, offset);
|
Set<ColumnType> types = complexColumn.getColumnTypes(parts);
|
||||||
|
ColumnType leastRestrictiveType = null;
|
||||||
|
if (types != null) {
|
||||||
|
for (ColumnType type : types) {
|
||||||
|
leastRestrictiveType = ColumnType.leastRestrictiveType(leastRestrictiveType, type);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
final VectorObjectSelector objectSelector = complexColumn.makeVectorObjectSelector(parts, offset);
|
||||||
|
if (leastRestrictiveType != null && leastRestrictiveType.isArray() && !expectedType.isArray()) {
|
||||||
|
final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(leastRestrictiveType.getElementType());
|
||||||
|
final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(expectedType);
|
||||||
|
return makeVectorArrayToScalarObjectSelector(offset, objectSelector, elementType, castTo);
|
||||||
|
}
|
||||||
|
return objectSelector;
|
||||||
}
|
}
|
||||||
// not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts
|
// not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts
|
||||||
if (parts.isEmpty()) {
|
if (parts.isEmpty()) {
|
||||||
|
@ -535,6 +550,15 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
||||||
expectedType
|
expectedType
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
// if the underlying column is array typed, the vector object selector it spits out will homogenize stuff to
|
||||||
|
// make all of the objects a consistent type, which is typically a good thing, but if we are doing mixed type
|
||||||
|
// stuff and expect the output type to be scalar typed, then we should coerce things to only extract the scalars
|
||||||
|
if (capabilities.isArray() && !expectedType.isArray()) {
|
||||||
|
final VectorObjectSelector delegate = column.makeVectorObjectSelector(offset);
|
||||||
|
final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(capabilities.getElementType());
|
||||||
|
final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(expectedType);
|
||||||
|
return makeVectorArrayToScalarObjectSelector(offset, delegate, elementType, castTo);
|
||||||
|
}
|
||||||
return column.makeVectorObjectSelector(offset);
|
return column.makeVectorObjectSelector(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -591,6 +615,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
||||||
return NilVectorSelector.create(offset);
|
return NilVectorSelector.create(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public VectorValueSelector makeVectorValueSelector(
|
public VectorValueSelector makeVectorValueSelector(
|
||||||
|
@ -1272,6 +1297,61 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a {@link VectorObjectSelector} from a base selector which may return ARRAY types, coercing to some scalar
|
||||||
|
* value. Single element arrays will be unwrapped, while multi-element arrays will become null values. Non-arrays
|
||||||
|
* will be best effort cast to the castTo type.
|
||||||
|
*/
|
||||||
|
private static VectorObjectSelector makeVectorArrayToScalarObjectSelector(
|
||||||
|
ReadableVectorOffset offset,
|
||||||
|
VectorObjectSelector delegate,
|
||||||
|
ExpressionType elementType,
|
||||||
|
ExpressionType castTo
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new VectorObjectSelector()
|
||||||
|
{
|
||||||
|
final Object[] scalars = new Object[offset.getMaxVectorSize()];
|
||||||
|
private int id = ReadableVectorInspector.NULL_ID;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object[] getObjectVector()
|
||||||
|
{
|
||||||
|
if (offset.getId() != id) {
|
||||||
|
Object[] result = delegate.getObjectVector();
|
||||||
|
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||||
|
if (result[i] instanceof Object[]) {
|
||||||
|
Object[] o = (Object[]) result[i];
|
||||||
|
if (o == null || o.length != 1) {
|
||||||
|
scalars[i] = null;
|
||||||
|
} else {
|
||||||
|
ExprEval<?> element = ExprEval.ofType(elementType, o[0]);
|
||||||
|
scalars[i] = element.castTo(castTo).value();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
ExprEval<?> element = ExprEval.bestEffortOf(result[i]);
|
||||||
|
scalars[i] = element.castTo(castTo).value();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
id = offset.getId();
|
||||||
|
}
|
||||||
|
return scalars;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getMaxVectorSize()
|
||||||
|
{
|
||||||
|
return offset.getMaxVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCurrentVectorSize()
|
||||||
|
{
|
||||||
|
return offset.getCurrentVectorSize();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Process the "raw" data to extract non-complex values. Like {@link RawFieldColumnSelector} but does not return
|
* Process the "raw" data to extract non-complex values. Like {@link RawFieldColumnSelector} but does not return
|
||||||
* complex nested objects and does not wrap the results in {@link StructuredData}.
|
* complex nested objects and does not wrap the results in {@link StructuredData}.
|
||||||
|
@ -1320,7 +1400,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
||||||
public boolean isNull()
|
public boolean isNull()
|
||||||
{
|
{
|
||||||
final Object o = getObject();
|
final Object o = getObject();
|
||||||
return !(o instanceof Number || (o instanceof String && Doubles.tryParse((String) o) != null));
|
if (o instanceof Number) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (o instanceof String) {
|
||||||
|
return GuavaUtils.tryParseLong((String) o) == null && Doubles.tryParse((String) o) == null;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.query.groupby;
|
package org.apache.druid.query.groupby;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.common.config.NullHandling;
|
import org.apache.druid.common.config.NullHandling;
|
||||||
|
@ -36,7 +35,6 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
|
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
|
||||||
import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
|
import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
import org.apache.druid.segment.TestHelper;
|
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.RowSignature;
|
import org.apache.druid.segment.column.RowSignature;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
|
@ -66,17 +64,14 @@ import java.util.stream.Collectors;
|
||||||
public class NestedGroupByArrayQueryTest
|
public class NestedGroupByArrayQueryTest
|
||||||
{
|
{
|
||||||
private static final Logger LOG = new Logger(NestedDataGroupByQueryTest.class);
|
private static final Logger LOG = new Logger(NestedDataGroupByQueryTest.class);
|
||||||
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||||
|
|
||||||
private final Closer closer;
|
private final Closer closer;
|
||||||
private final GroupByQueryConfig config;
|
|
||||||
private final QueryContexts.Vectorize vectorize;
|
private final QueryContexts.Vectorize vectorize;
|
||||||
private final AggregationTestHelper helper;
|
private final AggregationTestHelper helper;
|
||||||
private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
|
private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
|
||||||
private final String segmentsName;
|
|
||||||
|
|
||||||
public NestedGroupByArrayQueryTest(
|
public NestedGroupByArrayQueryTest(
|
||||||
GroupByQueryConfig config,
|
GroupByQueryConfig config,
|
||||||
|
@ -85,7 +80,6 @@ public class NestedGroupByArrayQueryTest
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
NestedDataModule.registerHandlersAndSerde();
|
NestedDataModule.registerHandlersAndSerde();
|
||||||
this.config = config;
|
|
||||||
this.vectorize = QueryContexts.Vectorize.fromString(vectorize);
|
this.vectorize = QueryContexts.Vectorize.fromString(vectorize);
|
||||||
this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||||
NestedDataModule.getJacksonModulesList(),
|
NestedDataModule.getJacksonModulesList(),
|
||||||
|
@ -93,7 +87,6 @@ public class NestedGroupByArrayQueryTest
|
||||||
tempFolder
|
tempFolder
|
||||||
);
|
);
|
||||||
this.segmentsGenerator = segmentGenerator;
|
this.segmentsGenerator = segmentGenerator;
|
||||||
this.segmentsName = segmentGenerator.toString();
|
|
||||||
this.closer = Closer.create();
|
this.closer = Closer.create();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -786,26 +786,26 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
if (NullHandling.replaceWithDefault()) {
|
if (NullHandling.replaceWithDefault()) {
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
"["
|
"["
|
||||||
+ "[1672531200000, null, 0, 0.0, true, 51, 1, [], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, null, 0, 0.0, true, 51, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, null, 2, 0.0, false, b, b, 2, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, null, 2, 0.0, false, b, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, a, 1, 1.0, true, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, a, 1, 1.0, true, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, b, 4, 3.3, true, 1, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, b, 4, 3.3, true, 1, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, c, 0, 4.4, true, hello, {}, [], {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, c, 0, 4.4, true, hello, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, d, 5, 5.9, false, null, a, 6, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, d, 5, 5.9, false, null, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]"
|
+ "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, 3.0, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]"
|
||||||
+ "]",
|
+ "]",
|
||||||
resultsSegments.get(0).getEvents().toString()
|
resultsSegments.get(0).getEvents().toString()
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
"["
|
"["
|
||||||
+ "[1672531200000, null, null, null, true, 51, 1, [], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, null, null, null, true, 51, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, , 2, null, false, b, b, 2, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, , 2, null, false, b, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, a, 1, 1.0, true, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, a, 1, 1.0, true, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, b, 4, 3.3, true, 1, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, b, 4, 3.3, true, 1, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, c, null, 4.4, true, hello, {}, [], {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, c, null, 4.4, true, hello, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, d, 5, 5.9, false, null, a, 6, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
+ "[1672531200000, d, 5, 5.9, false, null, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||||
+ "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]"
|
+ "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, 3.0, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]"
|
||||||
+ "]",
|
+ "]",
|
||||||
resultsSegments.get(0).getEvents().toString()
|
resultsSegments.get(0).getEvents().toString()
|
||||||
);
|
);
|
||||||
|
|
|
@ -0,0 +1,648 @@
|
||||||
|
/*
|
||||||
|
* 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.timeseries;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.apache.druid.guice.NestedDataModule;
|
||||||
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
|
import org.apache.druid.query.Druids;
|
||||||
|
import org.apache.druid.query.NestedDataTestUtils;
|
||||||
|
import org.apache.druid.query.QueryContexts;
|
||||||
|
import org.apache.druid.query.QueryRunnerTestHelper;
|
||||||
|
import org.apache.druid.query.Result;
|
||||||
|
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||||
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
|
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
|
import org.apache.druid.query.filter.AndDimFilter;
|
||||||
|
import org.apache.druid.query.filter.EqualityFilter;
|
||||||
|
import org.apache.druid.query.filter.Filter;
|
||||||
|
import org.apache.druid.query.filter.FilterTuning;
|
||||||
|
import org.apache.druid.segment.Segment;
|
||||||
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
|
import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
|
||||||
|
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.function.BiFunction;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class NestedDataTimeseriesQueryTest extends InitializedNullHandlingTest
|
||||||
|
{
|
||||||
|
@Parameterized.Parameters(name = "{0}:vectorize={1}")
|
||||||
|
public static Iterable<Object[]> constructorFeeder()
|
||||||
|
{
|
||||||
|
final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
|
||||||
|
NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE);
|
||||||
|
|
||||||
|
return QueryRunnerTestHelper.cartesian(
|
||||||
|
// runners
|
||||||
|
segmentsGenerators,
|
||||||
|
// vectorize?
|
||||||
|
ImmutableList.of("false", "force")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T> void assertExpectedResults(Iterable<Result<T>> expectedResults, Iterable<Result<T>> results)
|
||||||
|
{
|
||||||
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||||
|
|
||||||
|
private final Closer closer;
|
||||||
|
private final AggregationTestHelper helper;
|
||||||
|
|
||||||
|
private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
|
||||||
|
private final QueryContexts.Vectorize vectorize;
|
||||||
|
private final String segmentsName;
|
||||||
|
|
||||||
|
public NestedDataTimeseriesQueryTest(
|
||||||
|
BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator,
|
||||||
|
String vectorize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.helper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(
|
||||||
|
NestedDataModule.getJacksonModulesList(),
|
||||||
|
tempFolder
|
||||||
|
);
|
||||||
|
this.segmentsGenerator = segmentsGenerator;
|
||||||
|
this.segmentsName = segmentsGenerator.toString();
|
||||||
|
this.vectorize = QueryContexts.Vectorize.fromString(vectorize);
|
||||||
|
this.closer = Closer.create();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, Object> getContext()
|
||||||
|
{
|
||||||
|
return ImmutableMap.of(
|
||||||
|
QueryContexts.VECTORIZE_KEY, vectorize.toString(),
|
||||||
|
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize.toString()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCount()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 14L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSums()
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
"long":1, "double":1.0, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}},
|
||||||
|
"long":2, "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}},
|
||||||
|
"long":3, "double":2.0, "obj":{"a": 300},
|
||||||
|
"long":4, "double":3.3, "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}},
|
||||||
|
"long": null, "double":4.4, "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}},
|
||||||
|
"long":5, "double":5.9, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}},
|
||||||
|
"double":null, "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}},
|
||||||
|
*/
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.aggregators(
|
||||||
|
new CountAggregatorFactory("count"),
|
||||||
|
new LongSumAggregatorFactory("sumLong", "long"),
|
||||||
|
new DoubleSumAggregatorFactory("sumDouble", "double"),
|
||||||
|
new LongSumAggregatorFactory("sumNestedLong", "v0"),
|
||||||
|
new DoubleSumAggregatorFactory("sumNestedDouble", "v1"),
|
||||||
|
new DoubleSumAggregatorFactory("sumNestedLongFromArray", "v2")
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn("obj", "$.a", "v0", ColumnType.LONG),
|
||||||
|
new NestedFieldVirtualColumn("obj", "$.b.y", "v1", ColumnType.DOUBLE),
|
||||||
|
new NestedFieldVirtualColumn("obj", "$.b.z[0]", "v2", ColumnType.LONG)
|
||||||
|
)
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.<String, Object>builder()
|
||||||
|
.put("count", 14L)
|
||||||
|
.put("sumLong", 30L)
|
||||||
|
.put("sumDouble", 33.2)
|
||||||
|
.put("sumNestedLong", 5600L)
|
||||||
|
.put("sumNestedDouble", 11.0)
|
||||||
|
.put("sumNestedLongFromArray", 44.0)
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSumsNoVectorize()
|
||||||
|
{
|
||||||
|
if (QueryContexts.Vectorize.FORCE.equals(vectorize)) {
|
||||||
|
// variant types cannot vectorize aggregators
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
/*
|
||||||
|
"variant": 1, "variantWithArrays": 1,
|
||||||
|
"variant": "b", "variantWithArrays": "b",
|
||||||
|
"variant": 3.0, "variantWithArrays": 3.0,
|
||||||
|
"variant": "1", "variantWithArrays": "1",
|
||||||
|
"variant": "hello", "variantWithArrays": "hello",
|
||||||
|
|
||||||
|
"variant": 51, "variantWithArrays": [51, -35],
|
||||||
|
*/
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.aggregators(
|
||||||
|
new CountAggregatorFactory("count"),
|
||||||
|
new LongSumAggregatorFactory("sumVariantLong", "variant"),
|
||||||
|
new DoubleSumAggregatorFactory("sumVariantDouble", "variant"),
|
||||||
|
new LongSumAggregatorFactory("sumVariantArraysLong", "variantWithArrays"),
|
||||||
|
new DoubleSumAggregatorFactory("sumVariantArraysDouble", "variantWithArrays")
|
||||||
|
)
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.<String, Object>builder()
|
||||||
|
.put("count", 14L)
|
||||||
|
.put("sumVariantLong", 112L)
|
||||||
|
.put("sumVariantDouble", 112.0)
|
||||||
|
.put("sumVariantArraysLong", 10L)
|
||||||
|
.put("sumVariantArraysDouble", 10.0)
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterLong()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("long", ColumnType.LONG, 2L, null),
|
||||||
|
new EqualityFilter("v0", ColumnType.LONG, 2L, null)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"long",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.LONG
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantAsString()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variant", ColumnType.STRING, "hello", null),
|
||||||
|
new EqualityFilter("v0", ColumnType.STRING, "hello", null)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variant",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.STRING
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantAsStringNoIndexes()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variant", ColumnType.STRING, "hello", new FilterTuning(false, null, null)),
|
||||||
|
new EqualityFilter("v0", ColumnType.STRING, "hello", new FilterTuning(false, null, null))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variant",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.STRING
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantAsLong()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variant", ColumnType.LONG, 51L, null),
|
||||||
|
new EqualityFilter("v0", ColumnType.LONG, 51L, null)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variant",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.LONG
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantAsLongNoIndexes()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variant", ColumnType.LONG, 51L, new FilterTuning(false, null, null)),
|
||||||
|
new EqualityFilter("v0", ColumnType.LONG, 51L, new FilterTuning(false, null, null))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variant",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.LONG
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantArrayAsString()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variantWithArrays", ColumnType.STRING, "1", null),
|
||||||
|
new EqualityFilter("v0", ColumnType.STRING, "1", null)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variantWithArrays",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.STRING
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 4L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantArrayAsDouble()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variantWithArrays", ColumnType.DOUBLE, 3.0, null),
|
||||||
|
new EqualityFilter("v0", ColumnType.DOUBLE, 3.0, null)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variantWithArrays",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.STRING
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantArrayAsArray()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new AndDimFilter(
|
||||||
|
new EqualityFilter("variantWithArrays", ColumnType.LONG_ARRAY, Arrays.asList(51, -35), null),
|
||||||
|
new EqualityFilter("v0", ColumnType.LONG_ARRAY, Arrays.asList(51, -35), null)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"variantWithArrays",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.STRING_ARRAY
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantArrayStringArray()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new EqualityFilter("variantWithArrays", ColumnType.STRING_ARRAY, Collections.singletonList("hello"), null)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"long",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.LONG
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterVariantArrayStringArrayNoIndexes()
|
||||||
|
{
|
||||||
|
// this doesn't really have anything to do with nested columns
|
||||||
|
// just a smoke test to make sure everything else is sane
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(Collections.singletonList(Intervals.ETERNITY))
|
||||||
|
.filters(
|
||||||
|
new EqualityFilter("variantWithArrays", ColumnType.STRING_ARRAY, Collections.singletonList("hello"), new FilterTuning(false, null, null))
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"long",
|
||||||
|
"$.",
|
||||||
|
"v0",
|
||||||
|
ColumnType.LONG
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.aggregators(new CountAggregatorFactory("count"))
|
||||||
|
.context(getContext())
|
||||||
|
.build();
|
||||||
|
runResults(
|
||||||
|
query,
|
||||||
|
ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2023-01-01T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of("count", 2L)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private void runResults(
|
||||||
|
TimeseriesQuery timeseriesQuery,
|
||||||
|
List<Result<TimeseriesResultValue>> expectedResults
|
||||||
|
)
|
||||||
|
{
|
||||||
|
List<Segment> segments = segmentsGenerator.apply(tempFolder, closer);
|
||||||
|
Supplier<List<Result<TimeseriesResultValue>>> runner =
|
||||||
|
() -> helper.runQueryOnSegmentsObjs(segments, timeseriesQuery).toList();
|
||||||
|
Filter filter = timeseriesQuery.getFilter() == null ? null : timeseriesQuery.getFilter().toFilter();
|
||||||
|
boolean allCanVectorize = segments.stream()
|
||||||
|
.allMatch(
|
||||||
|
s -> s.asStorageAdapter()
|
||||||
|
.canVectorize(
|
||||||
|
filter,
|
||||||
|
timeseriesQuery.getVirtualColumns(),
|
||||||
|
timeseriesQuery.isDescending()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(NestedDataTestUtils.expectSegmentGeneratorCanVectorize(segmentsName), allCanVectorize);
|
||||||
|
if (!allCanVectorize) {
|
||||||
|
if (vectorize == QueryContexts.Vectorize.FORCE) {
|
||||||
|
Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
|
||||||
|
Assert.assertEquals(
|
||||||
|
"org.apache.druid.java.util.common.ISE: Cannot vectorize!",
|
||||||
|
t.getMessage()
|
||||||
|
);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
List<Result<TimeseriesResultValue>> results = runner.get();
|
||||||
|
assertExpectedResults(
|
||||||
|
expectedResults,
|
||||||
|
results
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -162,6 +162,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
.add(new AutoTypeColumnSchema("arrayString"))
|
.add(new AutoTypeColumnSchema("arrayString"))
|
||||||
.add(new AutoTypeColumnSchema("arrayLong"))
|
.add(new AutoTypeColumnSchema("arrayLong"))
|
||||||
.add(new AutoTypeColumnSchema("arrayDouble"))
|
.add(new AutoTypeColumnSchema("arrayDouble"))
|
||||||
|
.add(new AutoTypeColumnSchema("variant"))
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -185,6 +186,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
.add("arrayString", ColumnType.STRING_ARRAY)
|
.add("arrayString", ColumnType.STRING_ARRAY)
|
||||||
.add("arrayLong", ColumnType.LONG_ARRAY)
|
.add("arrayLong", ColumnType.LONG_ARRAY)
|
||||||
.add("arrayDouble", ColumnType.DOUBLE_ARRAY)
|
.add("arrayDouble", ColumnType.DOUBLE_ARRAY)
|
||||||
|
.add("variant", ColumnType.STRING_ARRAY)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
static final List<InputRow> DEFAULT_ROWS = ImmutableList.of(
|
static final List<InputRow> DEFAULT_ROWS = ImmutableList.of(
|
||||||
|
@ -198,7 +200,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
0L,
|
0L,
|
||||||
ImmutableList.of("a", "b", "c"),
|
ImmutableList.of("a", "b", "c"),
|
||||||
ImmutableList.of(1L, 2L, 3L),
|
ImmutableList.of(1L, 2L, 3L),
|
||||||
ImmutableList.of(1.1, 2.2, 3.3)
|
ImmutableList.of(1.1, 2.2, 3.3),
|
||||||
|
"abc"
|
||||||
),
|
),
|
||||||
makeDefaultSchemaRow(
|
makeDefaultSchemaRow(
|
||||||
"1",
|
"1",
|
||||||
|
@ -210,7 +213,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
100L,
|
100L,
|
||||||
ImmutableList.of(),
|
ImmutableList.of(),
|
||||||
ImmutableList.of(),
|
ImmutableList.of(),
|
||||||
new Object[]{1.1, 2.2, 3.3}
|
new Object[]{1.1, 2.2, 3.3},
|
||||||
|
100L
|
||||||
),
|
),
|
||||||
makeDefaultSchemaRow(
|
makeDefaultSchemaRow(
|
||||||
"2",
|
"2",
|
||||||
|
@ -222,7 +226,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
40L,
|
40L,
|
||||||
null,
|
null,
|
||||||
new Object[]{1L, 2L, 3L},
|
new Object[]{1L, 2L, 3L},
|
||||||
Collections.singletonList(null)
|
Collections.singletonList(null),
|
||||||
|
"100"
|
||||||
),
|
),
|
||||||
makeDefaultSchemaRow(
|
makeDefaultSchemaRow(
|
||||||
"3",
|
"3",
|
||||||
|
@ -234,7 +239,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
null,
|
null,
|
||||||
new Object[]{"a", "b", "c"},
|
new Object[]{"a", "b", "c"},
|
||||||
null,
|
null,
|
||||||
ImmutableList.of()
|
ImmutableList.of(),
|
||||||
|
Arrays.asList(1.1, 2.2, 3.3)
|
||||||
),
|
),
|
||||||
makeDefaultSchemaRow(
|
makeDefaultSchemaRow(
|
||||||
"4",
|
"4",
|
||||||
|
@ -246,7 +252,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
9001L,
|
9001L,
|
||||||
ImmutableList.of("c", "d"),
|
ImmutableList.of("c", "d"),
|
||||||
Collections.singletonList(null),
|
Collections.singletonList(null),
|
||||||
new Object[]{-1.1, -333.3}
|
new Object[]{-1.1, -333.3},
|
||||||
|
12.34
|
||||||
),
|
),
|
||||||
makeDefaultSchemaRow(
|
makeDefaultSchemaRow(
|
||||||
"5",
|
"5",
|
||||||
|
@ -258,7 +265,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||||
12345L,
|
12345L,
|
||||||
Collections.singletonList(null),
|
Collections.singletonList(null),
|
||||||
new Object[]{123L, 345L},
|
new Object[]{123L, 345L},
|
||||||
null
|
null,
|
||||||
|
Arrays.asList(100, 200, 300)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -338,10 +338,26 @@ public class EqualityFilterTests
|
||||||
assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 765.432, null), ImmutableList.of("5"));
|
assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 765.432, null), ImmutableList.of("5"));
|
||||||
assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 765.431, null), ImmutableList.of());
|
assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 765.431, null), ImmutableList.of());
|
||||||
|
|
||||||
|
// different type matcher
|
||||||
|
assertFilterMatches(
|
||||||
|
new EqualityFilter("d0", ColumnType.LONG, 0L, null),
|
||||||
|
canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "2") : ImmutableList.of("0")
|
||||||
|
);
|
||||||
|
assertFilterMatches(new EqualityFilter("d0", ColumnType.LONG, 60L, null), ImmutableList.of("4"));
|
||||||
|
|
||||||
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 100L, null), ImmutableList.of("1"));
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 100L, null), ImmutableList.of("1"));
|
||||||
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 40L, null), ImmutableList.of("2"));
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 40L, null), ImmutableList.of("2"));
|
||||||
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9001L, null), ImmutableList.of("4"));
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9001L, null), ImmutableList.of("4"));
|
||||||
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9000L, null), ImmutableList.of());
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9000L, null), ImmutableList.of());
|
||||||
|
|
||||||
|
// test loss of precision
|
||||||
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.DOUBLE, 100.1, null), ImmutableList.of());
|
||||||
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.DOUBLE, 100.0, null), ImmutableList.of("1"));
|
||||||
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.DOUBLE, 40.1, null), ImmutableList.of());
|
||||||
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.DOUBLE, 40.0, null), ImmutableList.of("2"));
|
||||||
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.DOUBLE, 9001.1, null), ImmutableList.of());
|
||||||
|
assertFilterMatches(new EqualityFilter("l0", ColumnType.DOUBLE, 9001.0, null), ImmutableList.of("4"));
|
||||||
|
|
||||||
if (!isAutoSchema()) {
|
if (!isAutoSchema()) {
|
||||||
// auto schema doesn't store float columns as floats, rather they are stored as doubles... the predicate matcher
|
// auto schema doesn't store float columns as floats, rather they are stored as doubles... the predicate matcher
|
||||||
// matches fine, but the string value set index does not match correctly if we expect the input float values
|
// matches fine, but the string value set index does not match correctly if we expect the input float values
|
||||||
|
@ -430,6 +446,8 @@ public class EqualityFilterTests
|
||||||
),
|
),
|
||||||
ImmutableList.of()
|
ImmutableList.of()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
assertFilterMatches(
|
assertFilterMatches(
|
||||||
new EqualityFilter(
|
new EqualityFilter(
|
||||||
"arrayLong",
|
"arrayLong",
|
||||||
|
@ -475,6 +493,37 @@ public class EqualityFilterTests
|
||||||
),
|
),
|
||||||
ImmutableList.of()
|
ImmutableList.of()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
// 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(
|
assertFilterMatches(
|
||||||
new EqualityFilter(
|
new EqualityFilter(
|
||||||
"arrayDouble",
|
"arrayDouble",
|
||||||
|
@ -522,6 +571,73 @@ public class EqualityFilterTests
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testVariant()
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
dim0 .. variant
|
||||||
|
"0", .. "abc"
|
||||||
|
"1", .. 100L
|
||||||
|
"2", .. "100"
|
||||||
|
"3", .. [1.1, 2.2, 3.3]
|
||||||
|
"4", .. 12.34
|
||||||
|
"5", .. [100, 200, 300]
|
||||||
|
|
||||||
|
*/
|
||||||
|
if (isAutoSchema()) {
|
||||||
|
assertFilterMatches(
|
||||||
|
new EqualityFilter(
|
||||||
|
"variant",
|
||||||
|
ColumnType.STRING_ARRAY,
|
||||||
|
ImmutableList.of("a", "b", "c"),
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
|
||||||
|
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.STRING,
|
||||||
|
"100",
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("1", "2")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new EqualityFilter(
|
||||||
|
"variant",
|
||||||
|
ColumnType.LONG_ARRAY,
|
||||||
|
Arrays.asList(100, 200, 300),
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class EqualityFilterNonParameterizedTests extends InitializedNullHandlingTest
|
public static class EqualityFilterNonParameterizedTests extends InitializedNullHandlingTest
|
||||||
|
|
|
@ -53,6 +53,7 @@ import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
@RunWith(Enclosed.class)
|
@RunWith(Enclosed.class)
|
||||||
|
@ -566,6 +567,123 @@ public class RangeFilterTests
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNumericMatchStringyBounds()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.STRING, "abc", null, true, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.STRING, "abc", "def", true, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.STRING, null, "abc", true, true, null),
|
||||||
|
canTestNumericNullsAsDefaultValues
|
||||||
|
? ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
|
||||||
|
: ImmutableList.of("0", "1", "2", "4", "5", "6")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNumericMatchPrecisionLoss()
|
||||||
|
{
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12345.1, 12345.4, false, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.1, 12345.4, false, false, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.1, 12345.4, false, true, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.1, 12345.4, true, false, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.1, 12345.4, false, true, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12345.0, 12345.1, false, true, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12345.0, 12345.1, true, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.0, 12344.9, false, false, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.0, 12344.9, false, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.5, null, true, true, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12344.5, null, false, true, null),
|
||||||
|
ImmutableList.of("5")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12345.5, null, true, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, 12345.5, null, false, true, null),
|
||||||
|
ImmutableList.of()
|
||||||
|
);
|
||||||
|
|
||||||
|
if (canTestNumericNullsAsDefaultValues) {
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12344.5, true, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12344.5, false, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "6", "7")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12345.5, true, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12345.5, false, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12344.5, true, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "4", "6")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12344.5, false, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "4", "6")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12345.5, true, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "4", "5", "6")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter("l0", ColumnType.DOUBLE, null, 12345.5, false, true, null),
|
||||||
|
ImmutableList.of("0", "1", "2", "4", "5", "6")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNumericNullsAndZeros()
|
public void testNumericNullsAndZeros()
|
||||||
{
|
{
|
||||||
|
@ -1065,6 +1183,240 @@ public class RangeFilterTests
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testArrayRangesPrecisionLoss()
|
||||||
|
{
|
||||||
|
if (isAutoSchema()) {
|
||||||
|
// only auto schema supports array columns currently, this means the match value will need to be coerceable to
|
||||||
|
// the column value type...
|
||||||
|
|
||||||
|
/* dim0 .. arrayLong
|
||||||
|
"0", .. [1L, 2L, 3L],
|
||||||
|
"1", .. [],
|
||||||
|
"2", .. [1L, 2L, 3L],
|
||||||
|
"3", .. null,
|
||||||
|
"4", .. [null],
|
||||||
|
"5", .. [123L, 345L],
|
||||||
|
"6", .. [100, 200],
|
||||||
|
"7", .. [1234, 3456L, null]
|
||||||
|
*/
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
null,
|
||||||
|
new Object[]{1.0, 2.0, 3.0},
|
||||||
|
true,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "1", "2", "4")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
null,
|
||||||
|
new Object[]{1.0, 2.0, 3.0},
|
||||||
|
true,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("1", "4")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
null,
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
true,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "1", "2", "4")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{1.0, 2.0, 3.0},
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2", "5", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{0.8, 1.8, 2.8},
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2", "5", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{0.8, 1.8, 2.8},
|
||||||
|
null,
|
||||||
|
true,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2", "5", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{1.0, 2.0, 3.0},
|
||||||
|
null,
|
||||||
|
true,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("5", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("5", "6", "7")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
null,
|
||||||
|
true,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("5", "6", "7")
|
||||||
|
);
|
||||||
|
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{0.8, 1.8, 2.8},
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
true,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{0.8, 1.8, 2.8},
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
false,
|
||||||
|
true,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{0.8, 1.8, 2.8},
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
true,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"arrayLong",
|
||||||
|
ColumnType.DOUBLE_ARRAY,
|
||||||
|
new Object[]{0.8, 1.8, 2.8},
|
||||||
|
new Object[]{1.1, 2.1, 3.1},
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "2")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testVariant()
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
dim0 .. variant
|
||||||
|
"0", .. "abc"
|
||||||
|
"1", .. 100L
|
||||||
|
"2", .. "100"
|
||||||
|
"3", .. [1.1, 2.2, 3.3]
|
||||||
|
"4", .. 12.34
|
||||||
|
"5", .. [100, 200, 300]
|
||||||
|
"6", .. null
|
||||||
|
"7", .. null
|
||||||
|
*/
|
||||||
|
if (isAutoSchema()) {
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"variant",
|
||||||
|
ColumnType.LONG,
|
||||||
|
100L,
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("1", "2", "5")
|
||||||
|
);
|
||||||
|
// lexicographical comparison
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"variant",
|
||||||
|
ColumnType.STRING,
|
||||||
|
"100",
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("0", "1", "2", "4", "5")
|
||||||
|
);
|
||||||
|
assertFilterMatches(
|
||||||
|
new RangeFilter(
|
||||||
|
"variant",
|
||||||
|
ColumnType.LONG_ARRAY,
|
||||||
|
Collections.singletonList(100L),
|
||||||
|
Arrays.asList(100L, 200L, 300L),
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableList.of("1", "2", "5")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class RangeFilterNonParameterizedTests extends InitializedNullHandlingTest
|
public static class RangeFilterNonParameterizedTests extends InitializedNullHandlingTest
|
||||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||||
import org.apache.druid.segment.VirtualColumns;
|
import org.apache.druid.segment.VirtualColumns;
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
import org.apache.druid.segment.column.StringEncodingStrategy;
|
||||||
|
import org.apache.druid.segment.column.Types;
|
||||||
import org.apache.druid.segment.data.CompressionFactory;
|
import org.apache.druid.segment.data.CompressionFactory;
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
import org.apache.druid.segment.data.FrontCodedIndexed;
|
||||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||||
|
@ -254,7 +255,7 @@ public class ExpressionVectorSelectorsTest extends InitializedNullHandlingTest
|
||||||
} else {
|
} else {
|
||||||
VectorValueSelector selector = null;
|
VectorValueSelector selector = null;
|
||||||
VectorObjectSelector objectSelector = null;
|
VectorObjectSelector objectSelector = null;
|
||||||
if (outputType != null && outputType.isNumeric()) {
|
if (Types.isNumeric(outputType)) {
|
||||||
selector = cursor.getColumnSelectorFactory().makeValueSelector("v");
|
selector = cursor.getColumnSelectorFactory().makeValueSelector("v");
|
||||||
} else {
|
} else {
|
||||||
objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v");
|
objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v");
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "variantWithArrays": 1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantEmptyObj":"b", "variantEmtpyArray":2, "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantEmptyObj":"b", "variantEmtpyArray":2, "variantWithArrays": "b", "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "obj":{"a": 300}, "complexObj":{"x": 4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "variantWithArrays": 3.0, "obj":{"a": 300}, "complexObj":{"x": 4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "variantWithArrays": "1", "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantEmptyObj":{}, "variantEmtpyArray":[], "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantEmptyObj":{}, "variantEmtpyArray":[], "variantWithArrays": "hello", "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantEmptyObj":1, "variantEmtpyArray":[], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantEmptyObj":1, "variantEmtpyArray":[], "variantWithArrays": [51, -35], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.druid.query.ordering.StringComparators;
|
||||||
import org.apache.druid.segment.column.ColumnHolder;
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.RowSignature;
|
import org.apache.druid.segment.column.RowSignature;
|
||||||
|
import org.apache.druid.segment.column.Types;
|
||||||
import org.apache.druid.sql.calcite.filtration.BoundRefKey;
|
import org.apache.druid.sql.calcite.filtration.BoundRefKey;
|
||||||
import org.apache.druid.sql.calcite.filtration.Bounds;
|
import org.apache.druid.sql.calcite.filtration.Bounds;
|
||||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||||
|
@ -512,7 +513,7 @@ public class Expressions
|
||||||
|
|
||||||
final DimFilter equalFilter;
|
final DimFilter equalFilter;
|
||||||
final ColumnType outputType = druidExpression.getDruidType();
|
final ColumnType outputType = druidExpression.getDruidType();
|
||||||
final boolean isOutputNumeric = outputType != null && outputType.isNumeric();
|
final boolean isOutputNumeric = Types.isNumeric(outputType);
|
||||||
// if a simple extraction, we can typically use the base column directly for filtering. however, some expressions
|
// if a simple extraction, we can typically use the base column directly for filtering. however, some expressions
|
||||||
// such as cast also appear as a simple extraction because some native layer things can handle the cast
|
// such as cast also appear as a simple extraction because some native layer things can handle the cast
|
||||||
// themselves, so we check the output type of the expression and compare it to the type of the direct column. a
|
// themselves, so we check the output type of the expression and compare it to the type of the direct column. a
|
||||||
|
|
|
@ -1319,6 +1319,52 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
.run();
|
.run();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByRootSingleTypeLongJsonValueFilter()
|
||||||
|
{
|
||||||
|
testQuery(
|
||||||
|
"SELECT "
|
||||||
|
+ "long, "
|
||||||
|
+ "SUM(cnt) "
|
||||||
|
+ "FROM druid.nested WHERE JSON_VALUE(long, '$.') = '1' GROUP BY 1",
|
||||||
|
ImmutableList.of(
|
||||||
|
GroupByQuery.builder()
|
||||||
|
.setDataSource(DATA_SOURCE)
|
||||||
|
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||||
|
.setGranularity(Granularities.ALL)
|
||||||
|
.setDimensions(
|
||||||
|
dimensions(
|
||||||
|
new DefaultDimensionSpec("long", "d0", ColumnType.LONG)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.setVirtualColumns(
|
||||||
|
new NestedFieldVirtualColumn(
|
||||||
|
"long",
|
||||||
|
"v0",
|
||||||
|
ColumnType.STRING,
|
||||||
|
Collections.emptyList(),
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
false
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.setDimFilter(
|
||||||
|
equality("v0", "1", ColumnType.STRING)
|
||||||
|
)
|
||||||
|
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||||
|
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||||
|
.build()
|
||||||
|
),
|
||||||
|
ImmutableList.of(
|
||||||
|
new Object[]{1L, 1L}
|
||||||
|
),
|
||||||
|
RowSignature.builder()
|
||||||
|
.add("long", ColumnType.LONG)
|
||||||
|
.add("EXPR$1", ColumnType.LONG)
|
||||||
|
.build()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGroupByRootSingleTypeArrayLongNullsFilteredArrayEquality()
|
public void testGroupByRootSingleTypeArrayLongNullsFilteredArrayEquality()
|
||||||
{
|
{
|
||||||
|
@ -2745,6 +2791,43 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByPathSelectorFilterVariant2Double()
|
||||||
|
{
|
||||||
|
testQuery(
|
||||||
|
"SELECT "
|
||||||
|
+ "JSON_VALUE(nest, '$.x'), "
|
||||||
|
+ "SUM(cnt) "
|
||||||
|
+ "FROM druid.nested WHERE JSON_VALUE(nest, '$.mixed2') = 1.1 GROUP BY 1",
|
||||||
|
ImmutableList.of(
|
||||||
|
GroupByQuery.builder()
|
||||||
|
.setDataSource(DATA_SOURCE)
|
||||||
|
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||||
|
.setGranularity(Granularities.ALL)
|
||||||
|
.setVirtualColumns(
|
||||||
|
new NestedFieldVirtualColumn("nest", "$.mixed2", "v0", ColumnType.DOUBLE),
|
||||||
|
new NestedFieldVirtualColumn("nest", "$.x", "v1", ColumnType.STRING)
|
||||||
|
)
|
||||||
|
.setDimensions(
|
||||||
|
dimensions(
|
||||||
|
new DefaultDimensionSpec("v1", "d0")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.setDimFilter(equality("v0", 1.1, ColumnType.DOUBLE))
|
||||||
|
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||||
|
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||||
|
.build()
|
||||||
|
),
|
||||||
|
ImmutableList.of(
|
||||||
|
new Object[]{"100", 1L}
|
||||||
|
),
|
||||||
|
RowSignature.builder()
|
||||||
|
.add("EXPR$0", ColumnType.STRING)
|
||||||
|
.add("EXPR$1", ColumnType.LONG)
|
||||||
|
.build()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGroupByPathSelectorFilterVariant2BothTypesMatcher()
|
public void testGroupByPathSelectorFilterVariant2BothTypesMatcher()
|
||||||
{
|
{
|
||||||
|
@ -5562,7 +5645,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"str",
|
"str",
|
||||||
"variant",
|
"variant",
|
||||||
"variantEmptyObj",
|
"variantEmptyObj",
|
||||||
"variantEmtpyArray"
|
"variantEmtpyArray",
|
||||||
|
"variantWithArrays"
|
||||||
)
|
)
|
||||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
.legacy(false)
|
.legacy(false)
|
||||||
|
@ -5579,6 +5663,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"51",
|
"51",
|
||||||
"1",
|
"1",
|
||||||
"[]",
|
"[]",
|
||||||
|
"[51,-35]",
|
||||||
"{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}",
|
"{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}",
|
||||||
"{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}",
|
"{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}",
|
||||||
null,
|
null,
|
||||||
|
@ -5615,6 +5700,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"b",
|
"b",
|
||||||
"\"b\"",
|
"\"b\"",
|
||||||
"2",
|
"2",
|
||||||
|
"b",
|
||||||
"{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}",
|
"{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}",
|
||||||
"{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}",
|
"{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}",
|
||||||
"[\"a\",\"b\",\"c\"]",
|
"[\"a\",\"b\",\"c\"]",
|
||||||
|
@ -5651,6 +5737,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"1",
|
"1",
|
||||||
"1",
|
"1",
|
||||||
"1",
|
"1",
|
||||||
|
"1",
|
||||||
"{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}",
|
"{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}",
|
||||||
"{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
"{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
@ -5687,6 +5774,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"1",
|
"1",
|
||||||
"{}",
|
"{}",
|
||||||
"4",
|
"4",
|
||||||
|
"1",
|
||||||
"{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}",
|
"{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}",
|
||||||
"{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
"{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
||||||
"[\"d\",\"e\"]",
|
"[\"d\",\"e\"]",
|
||||||
|
@ -5723,6 +5811,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"hello",
|
"hello",
|
||||||
"{}",
|
"{}",
|
||||||
"[]",
|
"[]",
|
||||||
|
"hello",
|
||||||
"{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}",
|
"{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}",
|
||||||
"{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}",
|
"{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}",
|
||||||
null,
|
null,
|
||||||
|
@ -5759,6 +5848,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"",
|
"",
|
||||||
"\"a\"",
|
"\"a\"",
|
||||||
"6",
|
"6",
|
||||||
|
null,
|
||||||
"{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}",
|
"{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}",
|
||||||
null,
|
null,
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
@ -5795,6 +5885,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"3.0",
|
"3.0",
|
||||||
"3.3",
|
"3.3",
|
||||||
"3",
|
"3",
|
||||||
|
"3.0",
|
||||||
"{\"a\":300}",
|
"{\"a\":300}",
|
||||||
"{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}",
|
"{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}",
|
||||||
"[\"b\",\"c\"]",
|
"[\"b\",\"c\"]",
|
||||||
|
@ -5833,6 +5924,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"51",
|
"51",
|
||||||
"1",
|
"1",
|
||||||
"[]",
|
"[]",
|
||||||
|
"[51,-35]",
|
||||||
"{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}",
|
"{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}",
|
||||||
"{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}",
|
"{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}",
|
||||||
null,
|
null,
|
||||||
|
@ -5869,6 +5961,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"b",
|
"b",
|
||||||
"\"b\"",
|
"\"b\"",
|
||||||
"2",
|
"2",
|
||||||
|
"b",
|
||||||
"{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}",
|
"{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}",
|
||||||
"{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}",
|
"{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}",
|
||||||
"[\"a\",\"b\",\"c\"]",
|
"[\"a\",\"b\",\"c\"]",
|
||||||
|
@ -5905,6 +5998,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"1",
|
"1",
|
||||||
"1",
|
"1",
|
||||||
"1",
|
"1",
|
||||||
|
"1",
|
||||||
"{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}",
|
"{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}",
|
||||||
"{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
"{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
@ -5941,6 +6035,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"1",
|
"1",
|
||||||
"{}",
|
"{}",
|
||||||
"4",
|
"4",
|
||||||
|
"1",
|
||||||
"{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}",
|
"{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}",
|
||||||
"{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
"{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
|
||||||
"[\"d\",\"e\"]",
|
"[\"d\",\"e\"]",
|
||||||
|
@ -5977,6 +6072,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"hello",
|
"hello",
|
||||||
"{}",
|
"{}",
|
||||||
"[]",
|
"[]",
|
||||||
|
"hello",
|
||||||
"{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}",
|
"{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}",
|
||||||
"{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}",
|
"{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}",
|
||||||
null,
|
null,
|
||||||
|
@ -6013,6 +6109,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
null,
|
null,
|
||||||
"\"a\"",
|
"\"a\"",
|
||||||
"6",
|
"6",
|
||||||
|
null,
|
||||||
"{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}",
|
"{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}",
|
||||||
null,
|
null,
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
@ -6049,6 +6146,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
"3.0",
|
"3.0",
|
||||||
"3.3",
|
"3.3",
|
||||||
"3",
|
"3",
|
||||||
|
"3.0",
|
||||||
"{\"a\":300}",
|
"{\"a\":300}",
|
||||||
"{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}",
|
"{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}",
|
||||||
"[\"b\",\"c\"]",
|
"[\"b\",\"c\"]",
|
||||||
|
@ -6086,6 +6184,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||||
.add("variant", ColumnType.STRING)
|
.add("variant", ColumnType.STRING)
|
||||||
.add("variantEmptyObj", ColumnType.NESTED_DATA)
|
.add("variantEmptyObj", ColumnType.NESTED_DATA)
|
||||||
.add("variantEmtpyArray", ColumnType.LONG_ARRAY)
|
.add("variantEmtpyArray", ColumnType.LONG_ARRAY)
|
||||||
|
.add("variantWithArrays", ColumnType.STRING_ARRAY)
|
||||||
.add("obj", ColumnType.NESTED_DATA)
|
.add("obj", ColumnType.NESTED_DATA)
|
||||||
.add("complexObj", ColumnType.NESTED_DATA)
|
.add("complexObj", ColumnType.NESTED_DATA)
|
||||||
.add("arrayString", ColumnType.STRING_ARRAY)
|
.add("arrayString", ColumnType.STRING_ARRAY)
|
||||||
|
|
Loading…
Reference in New Issue