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:
Clint Wylie 2023-07-27 16:01:21 -07:00 committed by GitHub
parent 9a9038c7ae
commit d406bafdfc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 2002 additions and 280 deletions

View File

@ -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

View File

@ -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;
} }
/** /**

View File

@ -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;

View File

@ -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

View File

@ -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());
}
}

View File

@ -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);
}
}
} }

View File

@ -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"

View File

@ -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)

View File

@ -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)

View File

@ -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)

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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);

View File

@ -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()
{ {

View File

@ -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)

View File

@ -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(),

View File

@ -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

View File

@ -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();

View File

@ -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();

View File

@ -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();
}
}
} }

View File

@ -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;
} }
} }
}; };

View File

@ -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);

View File

@ -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);
} }

View File

@ -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);

View File

@ -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

View File

@ -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();
} }

View File

@ -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()
); );

View File

@ -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
);
}
}

View File

@ -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)
) )
); );

View File

@ -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

View File

@ -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

View File

@ -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");

View File

@ -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}]}

View File

@ -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

View File

@ -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)