From d406bafdfc62d808eb2135f35c1c65b52356c6e3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Jul 2023 16:01:21 -0700 Subject: [PATCH] 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 --- .../org/apache/druid/math/expr/ExprEval.java | 31 + .../math/expr/ExpressionTypeConversion.java | 2 +- .../SimpleLongAggregatorFactory.java | 1 - .../druid/query/filter/EqualityFilter.java | 76 +- .../druid/query/filter/FallbackPredicate.java | 66 ++ .../druid/query/filter/RangeFilter.java | 154 ++++- .../vector/ArrayVectorValueMatcher.java | 2 +- .../vector/DoubleVectorValueMatcher.java | 13 +- .../vector/FloatVectorValueMatcher.java | 13 +- .../filter/vector/LongVectorValueMatcher.java | 13 +- .../MultiValueStringVectorValueMatcher.java | 11 +- .../vector/ObjectVectorValueMatcher.java | 4 +- .../SingleValueStringVectorValueMatcher.java | 11 +- .../StringObjectVectorValueMatcher.java | 11 +- .../vector/VectorValueMatcherFactory.java | 10 +- .../druid/segment/AutoTypeColumnIndexer.java | 32 +- .../apache/druid/segment/column/Types.java | 20 + .../NestedFieldColumnIndexSupplier.java | 20 +- .../NestedFieldDictionaryEncodedColumn.java | 135 ++-- .../ScalarDoubleColumnAndIndexSupplier.java | 12 +- .../ScalarLongColumnAndIndexSupplier.java | 37 +- .../druid/segment/nested/VariantColumn.java | 168 +++-- .../nested/VariantColumnAndIndexSupplier.java | 31 +- .../NestedCommonFormatColumnPartSerde.java | 12 +- .../druid/segment/virtual/ExpressionPlan.java | 2 +- .../segment/virtual/ExpressionSelectors.java | 9 +- .../virtual/NestedFieldVirtualColumn.java | 94 ++- .../groupby/NestedGroupByArrayQueryTest.java | 7 - .../query/scan/NestedDataScanQueryTest.java | 28 +- .../NestedDataTimeseriesQueryTest.java | 648 ++++++++++++++++++ .../druid/segment/filter/BaseFilterTest.java | 20 +- .../segment/filter/EqualityFilterTests.java | 116 ++++ .../segment/filter/RangeFilterTests.java | 352 ++++++++++ .../ExpressionVectorSelectorsTest.java | 3 +- .../resources/nested-all-types-test-data.json | 14 +- .../sql/calcite/expression/Expressions.java | 3 +- .../calcite/CalciteNestedDataQueryTest.java | 101 ++- 37 files changed, 2002 insertions(+), 280 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/filter/FallbackPredicate.java create mode 100644 processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java index 01d27e9006f..72910836026 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java @@ -644,6 +644,37 @@ public abstract class ExprEval 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 private boolean stringValueCached = false; @Nullable diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java index faa17d45e8d..ba576afa0c1 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java @@ -247,7 +247,7 @@ public class ExpressionTypeConversion { final ExpressionType functionType = ExpressionTypeConversion.function(type, other); // any number is long - return functionType != null && functionType.isNumeric() ? ExpressionType.LONG : functionType; + return Types.isNumeric(functionType) ? ExpressionType.LONG : functionType; } /** diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java index e01b61ad348..173a9cb229d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java @@ -39,7 +39,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; - import java.util.Collections; import java.util.Comparator; import java.util.List; diff --git a/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java b/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java index ba6d17ccc4c..ef507e4f20b 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java @@ -53,6 +53,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.TypeStrategy; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.BooleanValueMatcher; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.PredicateValueMatcherFactory; import org.apache.druid.segment.filter.ValueMatchers; @@ -384,7 +385,10 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt return typeDetectingArrayPredicateSupplier.get(); } - return arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType)); + return new FallbackPredicate<>( + arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType)), + ExpressionType.fromColumnTypeStrict(arrayType) + ); } @Override @@ -395,19 +399,24 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt private Supplier> 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 makeLongPredicateSupplier() { return Suppliers.memoize(() -> { - final Long valueAsLong = (Long) matchValue.castTo(ExpressionType.LONG).valueOrDefault(); - - if (valueAsLong == null) { + final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.LONG); + if (castForComparison == null) { return DruidLongPredicate.ALWAYS_FALSE; } else { // store the primitive, so we don't unbox for every comparison - final long unboxedLong = valueAsLong; + final long unboxedLong = castForComparison.asLong(); return input -> input == unboxedLong; } }); @@ -416,13 +425,12 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt private Supplier makeFloatPredicateSupplier() { return Suppliers.memoize(() -> { - final Double doubleValue = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault(); - - if (doubleValue == null) { + final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE); + if (castForComparison == null) { return DruidFloatPredicate.ALWAYS_FALSE; } else { // 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; } }); @@ -431,13 +439,12 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt private Supplier makeDoublePredicateSupplier() { return Suppliers.memoize(() -> { - final Double aDouble = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault(); - - if (aDouble == null) { + final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE); + if (castForComparison == null) { return DruidDoublePredicate.ALWAYS_FALSE; } else { // 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; } }); @@ -458,7 +465,11 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt return Suppliers.memoize(() -> input -> { final ExprEval eval = ExprEval.bestEffortOf(input); final Comparator 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; }); } @@ -466,7 +477,12 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt { final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType); final Comparator 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; } @@ -520,29 +536,41 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt @Override public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue) { - return ValueMatchers.makeStringValueMatcher( - selector, - matchValue.castTo(ExpressionType.STRING).asString(), - multiValue - ); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.STRING); + if (castForComparison == null) { + return BooleanValueMatcher.of(false); + } + return ValueMatchers.makeStringValueMatcher(selector, castForComparison.asString(), multiValue); } @Override 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 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 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 diff --git a/processing/src/main/java/org/apache/druid/query/filter/FallbackPredicate.java b/processing/src/main/java/org/apache/druid/query/filter/FallbackPredicate.java new file mode 100644 index 00000000000..570d804bc6a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/FallbackPredicate.java @@ -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 implements Predicate +{ + private final Predicate delegate; + private final ExpressionType expectedType; + private boolean needsCast = false; + + public FallbackPredicate(Predicate 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 castEval = ExprEval.bestEffortOf(input).castTo(expectedType); + return delegate.apply(castEval.value()); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java b/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java index 1f67edac0cc..f65bb6870cc 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java @@ -34,6 +34,7 @@ import com.google.common.collect.TreeRangeSet; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.IAE; 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.ExpressionType; 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.TypeSignature; 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.filter.Filters; import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; @@ -514,10 +516,18 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter if (hasLowerBound()) { ExprEval lowerCast = lowerEval.castTo(ExpressionType.LONG); if (lowerCast.isNumericNull()) { - hasLowerBound = false; - lowerBound = Long.MIN_VALUE; + // lower 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; } 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; } } else { @@ -529,10 +539,18 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter ExprEval upperCast = upperEval.castTo(ExpressionType.LONG); if (upperCast.isNumericNull()) { // 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; } 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; - upperBound = upperCast.asLong(); } } else { hasUpperBound = false; @@ -562,8 +580,10 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter if (hasLowerBound()) { ExprEval lowerCast = lowerEval.castTo(ExpressionType.DOUBLE); if (lowerCast.isNumericNull()) { - hasLowerBound = false; - lowerBound = Double.NEGATIVE_INFINITY; + // lower 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; } else { lowerBound = lowerCast.asDouble(); hasLowerBound = true; @@ -577,6 +597,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter ExprEval upperCast = upperEval.castTo(ExpressionType.DOUBLE); if (upperCast.isNumericNull()) { // 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; } else { hasUpperBound = true; @@ -595,9 +617,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter private Supplier> makeStringPredicateSupplier() { return Suppliers.memoize(() -> { - final Comparator stringComparator = matchValueType.isNumeric() - ? StringComparators.NUMERIC - : StringComparators.LEXICOGRAPHIC; + final Comparator stringComparator = + matchValueType.isNumeric() ? StringComparators.NUMERIC : StringComparators.LEXICOGRAPHIC; final String lowerBound = hasLowerBound() ? lowerEval.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 makeArrayPredicate(TypeSignature inputType) { - final Comparator arrayComparator = inputType.getNullableStrategy(); + final Comparator arrayComparator; + if (inputType.getElementType().is(ValueType.STRING) && Types.isNumericOrNumericArray(matchValueType)) { + arrayComparator = new NumericStringArrayComparator(); + } else { + arrayComparator = inputType.getNullableStrategy(); + } final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(inputType); 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); } @@ -631,7 +674,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter } ExprEval val = ExprEval.bestEffortOf(input); 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 comparator = val.type().getNullableStrategy(); final int lowerComparing = comparator.compare(val.asArray(), lowerBound); final int upperComparing = comparator.compare(upperBound, val.asArray()); @@ -656,8 +699,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter return false; } ExprEval val = ExprEval.bestEffortOf(input); - final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); - final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType()); + final Object[] upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, val.asArrayType()); final Comparator arrayComparator = val.type().getNullableStrategy(); final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound); final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); @@ -669,7 +712,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter return false; } 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 Comparator arrayComparator = val.type().getNullableStrategy(); final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound); @@ -682,7 +725,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter return false; } ExprEval val = ExprEval.bestEffortOf(input); - final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Object[] upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, val.asArrayType()); final Comparator arrayComparator = val.type().getNullableStrategy(); final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); return upperComparing > 0; @@ -715,7 +758,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter return false; } ExprEval val = ExprEval.bestEffortOf(input); - final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType()); final Comparator arrayComparator = val.type().getNullableStrategy(); final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray()); return lowerComparing >= 0; @@ -739,7 +782,10 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter @Override public Predicate makeStringPredicate() { - return stringPredicateSupplier.get(); + return new FallbackPredicate<>( + stringPredicateSupplier.get(), + ExpressionType.STRING + ); } @Override @@ -748,8 +794,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter if (matchValueType.isNumeric()) { return longPredicateSupplier.get(); } - Predicate stringPredicate = stringPredicateSupplier.get(); - return input -> stringPredicate.apply(String.valueOf(input)); + Predicate stringPredicate = makeStringPredicate(); + return input -> stringPredicate.apply(Evals.asString(input)); } @Override @@ -758,8 +804,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter if (matchValueType.isNumeric()) { return floatPredicateSupplier.get(); } - Predicate stringPredicate = stringPredicateSupplier.get(); - return input -> stringPredicate.apply(String.valueOf(input)); + Predicate stringPredicate = makeStringPredicate(); + return input -> stringPredicate.apply(Evals.asString(input)); } @Override @@ -768,8 +814,8 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter if (matchValueType.isNumeric()) { return doublePredicateSupplier.get(); } - Predicate stringPredicate = stringPredicateSupplier.get(); - return input -> stringPredicate.apply(String.valueOf(input)); + Predicate stringPredicate = makeStringPredicate(); + return input -> stringPredicate.apply(Evals.asString(input)); } @Override @@ -778,9 +824,9 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter if (inputType == null) { return typeDetectingArrayPredicateSupplier.get(); } - return arrayPredicates.computeIfAbsent( - inputType, - (existing) -> RangeFilter.this.makeArrayPredicate(inputType) + return new FallbackPredicate<>( + arrayPredicates.computeIfAbsent(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( final RangeType rangeType, final long lowerLongBound, @@ -1035,4 +1102,31 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter return UNBOUNDED; } } + + private static class NumericStringArrayComparator implements Comparator + { + @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); + } + } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java index 13c53aeee02..7d889c85aa8 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java @@ -51,7 +51,7 @@ public class ArrayVectorValueMatcher implements VectorValueMatcherFactory } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { throw new UnsupportedOperationException( "Vectorized matcher cannot make object matcher for ARRAY types" diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java index 66064fc4693..91249ae5612 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java @@ -55,14 +55,17 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); - ExprEval cast = eval.castTo(ExpressionType.DOUBLE); - if (cast.isNumericNull()) { + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE); + if (castForComparison == null) { + return BooleanVectorValueMatcher.of(selector, false); + } + if (castForComparison.isNumericNull()) { return makeNullValueMatcher(selector); } - return makeDoubleMatcher(cast.asDouble()); + return makeDoubleMatcher(castForComparison.asDouble()); } private BaseVectorValueMatcher makeDoubleMatcher(double matchValDouble) diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java index ed8b787668e..823f88d3cf2 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java @@ -57,14 +57,17 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); - ExprEval cast = eval.castTo(ExpressionType.DOUBLE); - if (cast.isNumericNull()) { + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE); + if (castForComparison == null) { + return BooleanVectorValueMatcher.of(selector, false); + } + if (castForComparison.isNumericNull()) { return makeNullValueMatcher(selector); } - return makeFloatMatcher((float) cast.asDouble()); + return makeFloatMatcher((float) castForComparison.asDouble()); } private BaseVectorValueMatcher makeFloatMatcher(float matchValFloat) diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java index a38703451c5..7eb865f7aa8 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java @@ -57,14 +57,17 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); - ExprEval cast = eval.castTo(ExpressionType.LONG); - if (cast.isNumericNull()) { + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.LONG); + if (castForComparison == null) { + return BooleanVectorValueMatcher.of(selector, false); + } + if (castForComparison.isNumericNull()) { return makeNullValueMatcher(selector); } - return makeLongMatcher(cast.asLong()); + return makeLongMatcher(castForComparison.asLong()); } private BaseVectorValueMatcher makeLongMatcher(long matchValLong) diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java index 866a9608fdd..000be2a26a6 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java @@ -101,11 +101,14 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); - ExprEval cast = eval.castTo(ExpressionType.STRING); - return makeMatcher(cast.asString()); + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING); + if (castForComparison == null) { + return BooleanVectorValueMatcher.of(selector, false); + } + return makeMatcher(castForComparison.asString()); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java index 42ca5eeb819..c464985be87 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java @@ -54,9 +54,9 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory } @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 diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java index 181241726f4..052758c1220 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java @@ -109,11 +109,14 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); - ExprEval cast = eval.castTo(ExpressionType.STRING); - return makeMatcher(cast.asString()); + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING); + if (castForComparison == null) { + return BooleanVectorValueMatcher.of(selector, false); + } + return makeMatcher(castForComparison.asString()); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java index e587b313cae..4078394ccfe 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java @@ -70,11 +70,14 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory } @Override - public VectorValueMatcher makeMatcher(Object value, ColumnType type) + public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); - ExprEval cast = eval.castTo(ExpressionType.STRING); - return makeMatcher(cast.asString()); + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING); + if (castForComparison == null) { + return BooleanVectorValueMatcher.of(selector, false); + } + return makeMatcher(castForComparison.asString()); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java index 666e85f4f23..01b746a839e 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java @@ -27,9 +27,17 @@ import javax.annotation.Nullable; 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(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); diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index f0b52669fa2..799b1293d16 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -19,9 +19,11 @@ package org.apache.druid.segment; +import com.google.common.primitives.Doubles; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; 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.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -489,37 +491,37 @@ public class AutoTypeColumnIndexer implements DimensionIndexer classOfObject() { diff --git a/processing/src/main/java/org/apache/druid/segment/column/Types.java b/processing/src/main/java/org/apache/druid/segment/column/Types.java index 02da5c8bfa7..2a0256f7667 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/Types.java +++ b/processing/src/main/java/org/apache/druid/segment/column/Types.java @@ -114,6 +114,26 @@ public class Types (typeSignature2 != null && typeSignature2.is(typeDescriptor)); } + /** + * Returns true if {@link TypeSignature} is not null and is {@link TypeSignature#isNumeric()} + */ + public static boolean isNumeric(@Nullable TypeSignature 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 boolean isNumericOrNumericArray(@Nullable TypeSignature typeSignature) + { + if (typeSignature == null) { + return false; + } + return typeSignature.isNumeric() || (typeSignature.isArray() && typeSignature.getElementType().isNumeric()); + } + public static class IncompatibleTypeException extends IAE { public IncompatibleTypeException(TypeSignature type, TypeSignature other) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java index 06d29bcf9e7..64c062bf417 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java @@ -729,10 +729,26 @@ public class NestedFieldColumnIndexSupplier + * 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 public DimensionSelector makeDimensionSelector( ReadableOffset offset, @@ -667,7 +700,17 @@ public class NestedFieldDictionaryEncodedColumn= adjustArrayId; } @Override @@ -745,77 +788,53 @@ public class NestedFieldDictionaryEncodedColumn valueType) { - final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) - .castTo(ExpressionType.DOUBLE); - if (eval.isNumericNull()) { - // value wasn't null, but not a number? - return null; + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE); + if (castForComparison == null) { + return new AllFalseBitmapColumnIndex(bitmapFactory); } - final double doubleValue = eval.asDouble(); + final double doubleValue = castForComparison.asDouble(); return new SimpleBitmapColumnIndex() { final FixedIndexed dictionary = doubleDictionarySupplier.get(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java index 9682c781e58..d8a7e9893cf 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.data.CompressedColumnarLongsSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.GenericIndexed; 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.SimpleBitmapColumnIndex; import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; @@ -231,14 +232,13 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier valueType) { - - final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) - .castTo(ExpressionType.LONG); - if (eval.isNumericNull()) { - // value wasn't null, but not a number - return null; + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value); + final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.LONG); + if (castForComparison == null) { + return new AllFalseBitmapColumnIndex(bitmapFactory); } - final long longValue = eval.asLong(); + final long longValue = castForComparison.asLong(); + return new SimpleBitmapColumnIndex() { final FixedIndexed dictionary = longDictionarySupplier.get(); @@ -416,12 +416,23 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier dictionary = longDictionarySupplier.get(); - IntIntPair range = dictionary.getRange( - startValue == null ? null : startValue.longValue(), - startStrict, - endValue == null ? null : endValue.longValue(), - 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()); + } + final IntIntPair range = dictionary.getRange(startLong, startStrict, endLong, endStrict); final int startIndex = range.leftInt(); final int endIndex = range.rightInt(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java index d5406ba8c45..820527b91d1 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java @@ -367,7 +367,7 @@ public class VariantColumn> @Nullable ExtractionFn extractionFn ) { - if (logicalType.isArray()) { + if (variantTypes == null && logicalType.isArray()) { throw new IAE("Dimension selector is currently unsupported for [%s]", logicalType); } // copy everywhere all the time @@ -628,9 +628,10 @@ public class VariantColumn> return f == null ? 0f : f; } else if (id < adjustDoubleId) { return longDictionary.get(id - adjustLongId).floatValue(); - } else { + } else if (id < adjustArrayId) { return doubleDictionary.get(id - adjustDoubleId).floatValue(); } + return 0L; } @Override @@ -646,9 +647,10 @@ public class VariantColumn> return d == null ? 0.0 : d; } else if (id < adjustDoubleId) { return longDictionary.get(id - adjustLongId).doubleValue(); - } else { + } else if (id < adjustArrayId) { return doubleDictionary.get(id - adjustDoubleId); } + return 0.0; } @Override @@ -664,8 +666,10 @@ public class VariantColumn> return l == null ? 0L : l; } else if (id < adjustDoubleId) { return longDictionary.get(id - adjustLongId); - } else { + } else if (id < adjustArrayId) { return doubleDictionary.get(id - adjustDoubleId).longValue(); + } else { + return 0L; } } @@ -688,7 +692,16 @@ public class VariantColumn> if (nullMark == offsetMark) { 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 @@ -747,57 +760,32 @@ public class VariantColumn> @Override 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 - - public Object[] getObjectVector() + public int adjustDictionaryId(int id) { - 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 = 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; + return id; } + @Nullable @Override - public int getMaxVectorSize() + public Object lookupScalarValue(int dictionaryId) { - return offset.getMaxVectorSize(); + return VariantColumn.this.lookupScalarValue(dictionaryId); } + @Nullable @Override - public int getCurrentVectorSize() + public Object lookupScalarValueAndCast(int dictionaryId) { - return offset.getCurrentVectorSize(); + return VariantColumn.this.lookupScalarValueAndCast(dictionaryId); } }; } @@ -815,7 +803,8 @@ public class VariantColumn> * 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 */ - private Object lookupScalarValueStrict(int id) + @Nullable + private Object lookupScalarValueAndCast(int id) { if (id == 0) { return null; @@ -828,6 +817,7 @@ public class VariantColumn> } } + @Nullable private Object lookupScalarValue(int id) { if (id < adjustLongId) { @@ -839,4 +829,90 @@ public class VariantColumn> } 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(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java index f67123123d4..c51790a812e 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java @@ -350,9 +350,15 @@ public class VariantColumnAndIndexSupplier implements Supplier valueType) { - final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) - .castTo(ExpressionType.fromColumnTypeStrict(logicalType)); - final Object[] arrayToMatch = eval.asArray(); + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value); + final ExprEval castForComparison = ExprEval.castForEqualityComparison( + eval, + ExpressionType.fromColumnTypeStrict(logicalType) + ); + if (castForComparison == null) { + return new AllFalseBitmapColumnIndex(bitmapFactory); + } + final Object[] arrayToMatch = castForComparison.asArray(); Indexed elements; final int elementOffset; switch (logicalType.getElementType().getType()) { @@ -425,9 +431,14 @@ public class VariantColumnAndIndexSupplier implements Supplier elementValueType) { - final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(elementValueType), value) - .castTo(ExpressionType.fromColumnTypeStrict(logicalType.getElementType())); - + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(elementValueType), value); + final ExprEval castForComparison = ExprEval.castForEqualityComparison( + eval, + ExpressionType.fromColumnTypeStrict(logicalType.getElementType()) + ); + if (castForComparison == null) { + return new AllFalseBitmapColumnIndex(bitmapFactory); + } Indexed elements; final int elementOffset; switch (logicalType.getElementType().getType()) { @@ -475,12 +486,12 @@ public class VariantColumnAndIndexSupplier implements Supplier selector = columnSelectorFactory.makeColumnValueSelector(columnName); supplier = makeNullableNumericSupplier(selector, selector::getFloat); diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index dd28ea2eb5e..c1834e1eb05 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Numbers; import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; @@ -508,14 +509,28 @@ public class NestedFieldVirtualColumn implements VirtualColumn } 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) { final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column; 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 complexColumn.makeVectorObjectSelector(parts, offset); + Set 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 if (parts.isEmpty()) { @@ -535,6 +550,15 @@ public class NestedFieldVirtualColumn implements VirtualColumn 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); } @@ -591,6 +615,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn return NilVectorSelector.create(offset); } + @Nullable @Override 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 * complex nested objects and does not wrap the results in {@link StructuredData}. @@ -1320,7 +1400,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn public boolean isNull() { 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 diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java index 3d992521735..4d09df32e31 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java @@ -19,7 +19,6 @@ package org.apache.druid.query.groupby; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; 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.strategy.GroupByStrategySelector; 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.RowSignature; import org.apache.druid.segment.column.ValueType; @@ -66,17 +64,14 @@ import java.util.stream.Collectors; public class NestedGroupByArrayQueryTest { private static final Logger LOG = new Logger(NestedDataGroupByQueryTest.class); - private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); private final Closer closer; - private final GroupByQueryConfig config; private final QueryContexts.Vectorize vectorize; private final AggregationTestHelper helper; private final BiFunction> segmentsGenerator; - private final String segmentsName; public NestedGroupByArrayQueryTest( GroupByQueryConfig config, @@ -85,7 +80,6 @@ public class NestedGroupByArrayQueryTest ) { NestedDataModule.registerHandlersAndSerde(); - this.config = config; this.vectorize = QueryContexts.Vectorize.fromString(vectorize); this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( NestedDataModule.getJacksonModulesList(), @@ -93,7 +87,6 @@ public class NestedGroupByArrayQueryTest tempFolder ); this.segmentsGenerator = segmentGenerator; - this.segmentsName = segmentGenerator.toString(); this.closer = Closer.create(); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 6133a16b1ea..53bb1b11c4d 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -786,26 +786,26 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest if (NullHandling.replaceWithDefault()) { 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, 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, 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, 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, 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, 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, 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, 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, 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, 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, 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, {}, [], 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, 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, 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() ); } else { 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, , 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, 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, 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, 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, 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, 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, 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, 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, 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, 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, {}, [], 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, 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, 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() ); diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java new file mode 100644 index 00000000000..60de2c0e23d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -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 constructorFeeder() + { + final List>> segmentsGenerators = + NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE); + + return QueryRunnerTestHelper.cartesian( + // runners + segmentsGenerators, + // vectorize? + ImmutableList.of("false", "force") + ); + } + + private static void assertExpectedResults(Iterable> expectedResults, Iterable> results) + { + TestHelper.assertExpectedResults(expectedResults, results); + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + private final Closer closer; + private final AggregationTestHelper helper; + + private final BiFunction> segmentsGenerator; + private final QueryContexts.Vectorize vectorize; + private final String segmentsName; + + public NestedDataTimeseriesQueryTest( + BiFunction> 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 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.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.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> expectedResults + ) + { + List segments = segmentsGenerator.apply(tempFolder, closer); + Supplier>> 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> results = runner.get(); + assertExpectedResults( + expectedResults, + results + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index b1e31638877..59822b03d55 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -162,6 +162,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .add(new AutoTypeColumnSchema("arrayString")) .add(new AutoTypeColumnSchema("arrayLong")) .add(new AutoTypeColumnSchema("arrayDouble")) + .add(new AutoTypeColumnSchema("variant")) .build() ); @@ -185,6 +186,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .add("arrayString", ColumnType.STRING_ARRAY) .add("arrayLong", ColumnType.LONG_ARRAY) .add("arrayDouble", ColumnType.DOUBLE_ARRAY) + .add("variant", ColumnType.STRING_ARRAY) .build(); static final List DEFAULT_ROWS = ImmutableList.of( @@ -198,7 +200,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest 0L, ImmutableList.of("a", "b", "c"), ImmutableList.of(1L, 2L, 3L), - ImmutableList.of(1.1, 2.2, 3.3) + ImmutableList.of(1.1, 2.2, 3.3), + "abc" ), makeDefaultSchemaRow( "1", @@ -210,7 +213,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest 100L, ImmutableList.of(), ImmutableList.of(), - new Object[]{1.1, 2.2, 3.3} + new Object[]{1.1, 2.2, 3.3}, + 100L ), makeDefaultSchemaRow( "2", @@ -222,7 +226,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest 40L, null, new Object[]{1L, 2L, 3L}, - Collections.singletonList(null) + Collections.singletonList(null), + "100" ), makeDefaultSchemaRow( "3", @@ -234,7 +239,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest null, new Object[]{"a", "b", "c"}, null, - ImmutableList.of() + ImmutableList.of(), + Arrays.asList(1.1, 2.2, 3.3) ), makeDefaultSchemaRow( "4", @@ -246,7 +252,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest 9001L, ImmutableList.of("c", "d"), Collections.singletonList(null), - new Object[]{-1.1, -333.3} + new Object[]{-1.1, -333.3}, + 12.34 ), makeDefaultSchemaRow( "5", @@ -258,7 +265,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest 12345L, Collections.singletonList(null), new Object[]{123L, 345L}, - null + null, + Arrays.asList(100, 200, 300) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index 87f2c4f1fff..8f9cc38aa09 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -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.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, 40L, null), ImmutableList.of("2")); assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9001L, null), ImmutableList.of("4")); 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()) { // 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 @@ -430,6 +446,8 @@ public class EqualityFilterTests ), ImmutableList.of() ); + + assertFilterMatches( new EqualityFilter( "arrayLong", @@ -475,6 +493,37 @@ public class EqualityFilterTests ), 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( new EqualityFilter( "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 diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java index 6313ad93908..4ac578afce8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java @@ -53,6 +53,7 @@ import org.junit.runners.Parameterized; import java.io.Closeable; import java.util.Arrays; +import java.util.Collections; import java.util.List; @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 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 diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java index 0a667ee081d..570177d8310 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.generator.GeneratorBasicSchemas; @@ -254,7 +255,7 @@ public class ExpressionVectorSelectorsTest extends InitializedNullHandlingTest } else { VectorValueSelector selector = null; VectorObjectSelector objectSelector = null; - if (outputType != null && outputType.isNumeric()) { + if (Types.isNumeric(outputType)) { selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); } else { objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v"); diff --git a/processing/src/test/resources/nested-all-types-test-data.json b/processing/src/test/resources/nested-all-types-test-data.json index 95a43a70d1c..e7219e42a49 100644 --- a/processing/src/test/resources/nested-all-types-test-data.json +++ b/processing/src/test/resources/nested-all-types-test-data.json @@ -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":"", "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":"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":"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":"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":"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":"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, "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, "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, "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":[], "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":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}]} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index 5fc88491b2d..ef572c8b621 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -54,6 +54,7 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; 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.Bounds; import org.apache.druid.sql.calcite.filtration.Filtration; @@ -512,7 +513,7 @@ public class Expressions final DimFilter equalFilter; 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 // 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 diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 5cd593947ea..93e542345eb 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -1319,6 +1319,52 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .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 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 public void testGroupByPathSelectorFilterVariant2BothTypesMatcher() { @@ -5562,7 +5645,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "str", "variant", "variantEmptyObj", - "variantEmtpyArray" + "variantEmtpyArray", + "variantWithArrays" ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) @@ -5579,6 +5663,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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, @@ -5615,6 +5700,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5651,6 +5737,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5687,6 +5774,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5723,6 +5811,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "hello", "{}", "[]", + "hello", "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, @@ -5759,6 +5848,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "", "\"a\"", "6", + null, "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", null, "[\"a\",\"b\"]", @@ -5795,6 +5885,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5833,6 +5924,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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, @@ -5869,6 +5961,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5905,6 +5998,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5941,6 +6035,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -5977,6 +6072,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "hello", "{}", "[]", + "hello", "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, @@ -6013,6 +6109,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest null, "\"a\"", "6", + null, "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", null, "[\"a\",\"b\"]", @@ -6049,6 +6146,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "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\"]", @@ -6086,6 +6184,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .add("variant", ColumnType.STRING) .add("variantEmptyObj", ColumnType.NESTED_DATA) .add("variantEmtpyArray", ColumnType.LONG_ARRAY) + .add("variantWithArrays", ColumnType.STRING_ARRAY) .add("obj", ColumnType.NESTED_DATA) .add("complexObj", ColumnType.NESTED_DATA) .add("arrayString", ColumnType.STRING_ARRAY)