mirror of https://github.com/apache/druid.git
Harmonization and bug-fixing for selector and filter behavior on unknown types. (#9484)
* Harmonization and bug-fixing for selector and filter behavior on unknown types. - Migrate ValueMatcherColumnSelectorStrategy to newer ColumnProcessorFactory system, and set defaultType COMPLEX so unknown types can be dynamically matched. - Remove ValueGetters in favor of ColumnComparisonFilter doing its own thing. - Switch various methods to use convertObjectToX when casting to numbers, rather than ad-hoc and inconsistent logic. - Fix bug in RowBasedExpressionColumnValueSelector: isBindingArray should return true even for 0- or 1- element arrays. - Adjust various javadocs. * Add throwParseExceptions option to Rows.objectToNumber, switch back to that. * Update tests. * Adjust moment sketch tests.
This commit is contained in:
parent
8b9fe6f584
commit
c6c2282b59
|
@ -90,7 +90,7 @@ public class MapBasedRow implements Row
|
|||
@Override
|
||||
public Number getMetric(String metric)
|
||||
{
|
||||
return Rows.objectToNumber(metric, event.get(metric));
|
||||
return Rows.objectToNumber(metric, event.get(metric), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -35,6 +35,7 @@ import java.util.TreeMap;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public final class Rows
|
||||
{
|
||||
|
@ -75,25 +76,30 @@ public final class Rows
|
|||
}
|
||||
|
||||
/**
|
||||
* Convert an object to a number. Nulls are treated as zeroes unless
|
||||
* druid.generic.useDefaultValueForNull is set to false.
|
||||
* Convert an object to a number.
|
||||
*
|
||||
* @param name field name of the object being converted (may be used for exception messages)
|
||||
* @param inputValue the actual object being converted
|
||||
* If {@link NullHandling#replaceWithDefault()} is true, this method will never return null. If false, it will return
|
||||
* {@link NullHandling#defaultLongValue()} instead of null.
|
||||
*
|
||||
* @return a number
|
||||
* @param name field name of the object being converted (may be used for exception messages)
|
||||
* @param inputValue the actual object being converted
|
||||
* @param throwParseExceptions whether this method should throw a {@link ParseException} or use a default/null value
|
||||
* when {@param inputValue} is not numeric
|
||||
*
|
||||
* @throws NullPointerException if the string is null
|
||||
* @throws ParseException if the column cannot be converted to a number
|
||||
* @return a Number; will not necessarily be the same type as {@param zeroClass}
|
||||
*
|
||||
* @throws ParseException if the input cannot be converted to a number and {@code throwParseExceptions} is true
|
||||
*/
|
||||
@Nullable
|
||||
public static Number objectToNumber(final String name, final Object inputValue)
|
||||
public static <T extends Number> Number objectToNumber(
|
||||
final String name,
|
||||
final Object inputValue,
|
||||
final boolean throwParseExceptions
|
||||
)
|
||||
{
|
||||
if (inputValue == null) {
|
||||
return NullHandling.defaultLongValue();
|
||||
}
|
||||
|
||||
if (inputValue instanceof Number) {
|
||||
} else if (inputValue instanceof Number) {
|
||||
return (Number) inputValue;
|
||||
} else if (inputValue instanceof String) {
|
||||
try {
|
||||
|
@ -109,10 +115,18 @@ public final class Rows
|
|||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new ParseException(e, "Unable to parse value[%s] for field[%s]", inputValue, name);
|
||||
if (throwParseExceptions) {
|
||||
throw new ParseException(e, "Unable to parse value[%s] for field[%s]", inputValue, name);
|
||||
} else {
|
||||
return NullHandling.defaultLongValue();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new ParseException("Unknown type[%s] for field[%s]", inputValue.getClass(), name);
|
||||
if (throwParseExceptions) {
|
||||
throw new ParseException("Unknown type[%s] for field[%s]", inputValue.getClass(), name);
|
||||
} else {
|
||||
return NullHandling.defaultLongValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -131,8 +131,12 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(400.0, sketchObject.getPowerSums()[0], 1e-10);
|
||||
|
||||
MomentSketchWrapper sketchObjectWithNulls = (MomentSketchWrapper) row.get(1); // "sketchWithNulls"
|
||||
// 23 null values, nulls at ingestion time are not replaced with default values for complex metrics inputs
|
||||
Assert.assertEquals(377.0, sketchObjectWithNulls.getPowerSums()[0], 1e-10);
|
||||
// 23 null values (377 when nulls are not replaced with default)
|
||||
Assert.assertEquals(
|
||||
NullHandling.replaceWithDefault() ? 400.0 : 377.0,
|
||||
sketchObjectWithNulls.getPowerSums()[0],
|
||||
1e-10
|
||||
);
|
||||
|
||||
double[] quantilesArray = (double[]) row.get(2); // "quantiles"
|
||||
Assert.assertEquals(0, quantilesArray[0], 0.05);
|
||||
|
@ -146,12 +150,16 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(0.9969, maxValue, 0.0001);
|
||||
|
||||
double[] quantilesArrayWithNulls = (double[]) row.get(5); // "quantilesWithNulls"
|
||||
Assert.assertEquals(5.0, quantilesArrayWithNulls[0], 0.05);
|
||||
Assert.assertEquals(7.57, quantilesArrayWithNulls[1], 0.05);
|
||||
Assert.assertEquals(NullHandling.replaceWithDefault() ? 0.0 : 5.0, quantilesArrayWithNulls[0], 0.05);
|
||||
Assert.assertEquals(
|
||||
NullHandling.replaceWithDefault() ? 7.721400294818661d : 7.57,
|
||||
quantilesArrayWithNulls[1],
|
||||
0.05
|
||||
);
|
||||
Assert.assertEquals(10.0, quantilesArrayWithNulls[2], 0.05);
|
||||
|
||||
Double minValueWithNulls = (Double) row.get(6); // "minWithNulls"
|
||||
Assert.assertEquals(5.0164, minValueWithNulls, 0.0001);
|
||||
Assert.assertEquals(NullHandling.replaceWithDefault() ? 0.0 : 5.0164, minValueWithNulls, 0.0001);
|
||||
|
||||
Double maxValueWithNulls = (Double) row.get(7); // "maxWithNulls"
|
||||
Assert.assertEquals(9.9788, maxValueWithNulls, 0.0001);
|
||||
|
|
|
@ -536,10 +536,10 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
|
|||
List<ResultRow> results = seq.toList();
|
||||
Assert.assertEquals(1, results.size());
|
||||
ResultRow row = results.get(0);
|
||||
Assert.assertEquals("sketch", 30.0, (double) row.get(0), 0);
|
||||
Assert.assertEquals("estimate", 30.0, (double) row.get(1), 0);
|
||||
Assert.assertEquals("union", 30.0, (double) row.get(3), 0);
|
||||
Assert.assertEquals("intersection", 30.0, (double) row.get(4), 0);
|
||||
Assert.assertEquals("sketch", NullHandling.replaceWithDefault() ? 40.0 : 30.0, (double) row.get(0), 0);
|
||||
Assert.assertEquals("estimate", NullHandling.replaceWithDefault() ? 40.0 : 30.0, (double) row.get(1), 0);
|
||||
Assert.assertEquals("union", NullHandling.replaceWithDefault() ? 40.0 : 30.0, (double) row.get(3), 0);
|
||||
Assert.assertEquals("intersection", NullHandling.replaceWithDefault() ? 40.0 : 30.0, (double) row.get(4), 0);
|
||||
Assert.assertEquals("anotb", 0, (double) row.get(5), 0);
|
||||
|
||||
Object meansObj = row.get(6); // means
|
||||
|
@ -548,20 +548,20 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
|
|||
Assert.assertEquals(3, means.length);
|
||||
Assert.assertEquals(1.0, means[0], 0);
|
||||
Assert.assertEquals(2.0, means[1], 0);
|
||||
Assert.assertEquals(3.0, means[2], 0.1);
|
||||
Assert.assertEquals(NullHandling.replaceWithDefault() ? 2.25 : 3.0, means[2], 0.1);
|
||||
|
||||
Object obj = row.get(2); // quantiles-sketch
|
||||
Assert.assertTrue(obj instanceof DoublesSketch);
|
||||
DoublesSketch ds = (DoublesSketch) obj;
|
||||
Assert.assertEquals(30, ds.getN());
|
||||
Assert.assertEquals(NullHandling.replaceWithDefault() ? 40 : 30, ds.getN());
|
||||
Assert.assertEquals(2.0, ds.getMinValue(), 0);
|
||||
Assert.assertEquals(2.0, ds.getMaxValue(), 0);
|
||||
|
||||
Object objSketch2 = row.get(7); // quantiles-sketch-with-nulls
|
||||
Assert.assertTrue(objSketch2 instanceof DoublesSketch);
|
||||
DoublesSketch ds2 = (DoublesSketch) objSketch2;
|
||||
Assert.assertEquals(30, ds2.getN());
|
||||
Assert.assertEquals(3.0, ds2.getMinValue(), 0);
|
||||
Assert.assertEquals(NullHandling.replaceWithDefault() ? 40 : 30, ds2.getN());
|
||||
Assert.assertEquals(NullHandling.replaceWithDefault() ? 0.0 : 3.0, ds2.getMinValue(), 0);
|
||||
Assert.assertEquals(3.0, ds2.getMaxValue(), 0);
|
||||
}
|
||||
|
||||
|
|
|
@ -70,11 +70,11 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde
|
|||
if (rawValue instanceof Collection) {
|
||||
for (final Object next : ((Collection) rawValue)) {
|
||||
if (next != null) {
|
||||
h.offer(Rows.objectToNumber(metricName, next).floatValue());
|
||||
h.offer(Rows.objectToNumber(metricName, next, true).floatValue());
|
||||
}
|
||||
}
|
||||
} else {
|
||||
h.offer(Rows.objectToNumber(metricName, rawValue).floatValue());
|
||||
h.offer(Rows.objectToNumber(metricName, rawValue, true).floatValue());
|
||||
}
|
||||
|
||||
return h;
|
||||
|
|
|
@ -110,7 +110,7 @@ public class FixedBucketsHistogramSerde extends ComplexMetricSerde
|
|||
} else if (rawValue instanceof String) {
|
||||
Number numberAttempt;
|
||||
try {
|
||||
numberAttempt = Rows.objectToNumber(metricName, rawValue);
|
||||
numberAttempt = Rows.objectToNumber(metricName, rawValue, true);
|
||||
FixedBucketsHistogram fbh = new FixedBucketsHistogram(
|
||||
aggregatorFactory.getLowerLimit(),
|
||||
aggregatorFactory.getUpperLimit(),
|
||||
|
|
|
@ -158,7 +158,7 @@ public class InputRowSerdeTest
|
|||
Assert.assertEquals(5.0f, out.getMetric("m1out").floatValue(), 0.00001);
|
||||
Assert.assertEquals(100L, out.getMetric("m2out"));
|
||||
Assert.assertEquals(1, ((HyperLogLogCollector) out.getRaw("m3out")).estimateCardinality(), 0.001);
|
||||
Assert.assertEquals(0L, out.getMetric("unparseable"));
|
||||
Assert.assertEquals(NullHandling.defaultLongValue(), out.getMetric("unparseable"));
|
||||
|
||||
EasyMock.verify(mockedAggregator);
|
||||
EasyMock.verify(mockedNullAggregator);
|
||||
|
|
|
@ -1,95 +0,0 @@
|
|||
/*
|
||||
* 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 org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
||||
|
||||
public class DoubleValueMatcherColumnSelectorStrategy
|
||||
implements ValueMatcherColumnSelectorStrategy<BaseDoubleColumnValueSelector>
|
||||
{
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final BaseDoubleColumnValueSelector selector, final String value)
|
||||
{
|
||||
final Double matchVal = DimensionHandlerUtils.convertObjectToDouble(value);
|
||||
if (matchVal == null) {
|
||||
return ValueMatcher.primitiveNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
final long matchValLongBits = Double.doubleToLongBits(matchVal);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return Double.doubleToLongBits(selector.getDouble()) == matchValLongBits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(
|
||||
final BaseDoubleColumnValueSelector selector,
|
||||
DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidDoublePredicate predicate = predicateFactory.makeDoublePredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyDouble(selector.getDouble());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueGetter makeValueGetter(final BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return () -> {
|
||||
if (selector.isNull()) {
|
||||
return null;
|
||||
}
|
||||
return new String[]{Double.toString(selector.getDouble())};
|
||||
};
|
||||
}
|
||||
}
|
|
@ -1,94 +0,0 @@
|
|||
/*
|
||||
* 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 org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
||||
public class FloatValueMatcherColumnSelectorStrategy
|
||||
implements ValueMatcherColumnSelectorStrategy<BaseFloatColumnValueSelector>
|
||||
{
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final BaseFloatColumnValueSelector selector, final String value)
|
||||
{
|
||||
final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
|
||||
if (matchVal == null) {
|
||||
return ValueMatcher.primitiveNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
final int matchValIntBits = Float.floatToIntBits(matchVal);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return Float.floatToIntBits(selector.getFloat()) == matchValIntBits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(
|
||||
final BaseFloatColumnValueSelector selector,
|
||||
DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyFloat(selector.getFloat());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueGetter makeValueGetter(final BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return () -> {
|
||||
if (selector.isNull()) {
|
||||
return null;
|
||||
}
|
||||
return new String[]{Float.toString(selector.getFloat())};
|
||||
};
|
||||
}
|
||||
}
|
|
@ -32,8 +32,6 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.TreeRangeSet;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Floats;
|
||||
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
||||
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
|
||||
import it.unimi.dsi.fastutil.longs.LongArrayList;
|
||||
|
@ -156,7 +154,12 @@ public class InDimFilter implements DimFilter
|
|||
{
|
||||
InDimFilter inFilter = optimizeLookup();
|
||||
if (inFilter.values.size() == 1) {
|
||||
return new SelectorDimFilter(inFilter.dimension, inFilter.values.first(), inFilter.getExtractionFn(), filterTuning);
|
||||
return new SelectorDimFilter(
|
||||
inFilter.dimension,
|
||||
inFilter.values.first(),
|
||||
inFilter.getExtractionFn(),
|
||||
filterTuning
|
||||
);
|
||||
}
|
||||
return inFilter;
|
||||
}
|
||||
|
@ -272,29 +275,27 @@ public class InDimFilter implements DimFilter
|
|||
{
|
||||
return Objects.hash(values, dimension, extractionFn, filterTuning);
|
||||
}
|
||||
|
||||
|
||||
private DruidLongPredicate createLongPredicate()
|
||||
{
|
||||
LongArrayList longs = new LongArrayList(values.size());
|
||||
for (String value : values) {
|
||||
final Long longValue = DimensionHandlerUtils.getExactLongFromDecimalString(value);
|
||||
final Long longValue = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
if (longValue != null) {
|
||||
longs.add(longValue);
|
||||
longs.add((long) longValue);
|
||||
}
|
||||
}
|
||||
|
||||
if (longs.size() > NUMERIC_HASHING_THRESHOLD) {
|
||||
final LongOpenHashSet longHashSet = new LongOpenHashSet(longs);
|
||||
|
||||
return input -> longHashSet.contains(input);
|
||||
return longHashSet::contains;
|
||||
} else {
|
||||
final long[] longArray = longs.toLongArray();
|
||||
Arrays.sort(longArray);
|
||||
|
||||
return input -> Arrays.binarySearch(longArray, input) >= 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// As the set of filtered values can be large, parsing them as longs should be done only if needed, and only once.
|
||||
// Pass in a common long predicate supplier to all filters created by .toFilter(), so that
|
||||
// we only compute the long hashset/array once per query.
|
||||
|
@ -304,12 +305,12 @@ public class InDimFilter implements DimFilter
|
|||
Supplier<DruidLongPredicate> longPredicate = () -> createLongPredicate();
|
||||
return Suppliers.memoize(longPredicate);
|
||||
}
|
||||
|
||||
|
||||
private DruidFloatPredicate createFloatPredicate()
|
||||
{
|
||||
IntArrayList floatBits = new IntArrayList(values.size());
|
||||
for (String value : values) {
|
||||
Float floatValue = Floats.tryParse(value);
|
||||
Float floatValue = DimensionHandlerUtils.convertObjectToFloat(value);
|
||||
if (floatValue != null) {
|
||||
floatBits.add(Float.floatToIntBits(floatValue));
|
||||
}
|
||||
|
@ -326,18 +327,18 @@ public class InDimFilter implements DimFilter
|
|||
return input -> Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private Supplier<DruidFloatPredicate> getFloatPredicateSupplier()
|
||||
{
|
||||
Supplier<DruidFloatPredicate> floatPredicate = () -> createFloatPredicate();
|
||||
return Suppliers.memoize(floatPredicate);
|
||||
}
|
||||
|
||||
|
||||
private DruidDoublePredicate createDoublePredicate()
|
||||
{
|
||||
LongArrayList doubleBits = new LongArrayList(values.size());
|
||||
for (String value : values) {
|
||||
Double doubleValue = Doubles.tryParse(value);
|
||||
Double doubleValue = DimensionHandlerUtils.convertObjectToDouble(value);
|
||||
if (doubleValue != null) {
|
||||
doubleBits.add(Double.doubleToLongBits((doubleValue)));
|
||||
}
|
||||
|
|
|
@ -1,93 +0,0 @@
|
|||
/*
|
||||
* 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 org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
||||
public class LongValueMatcherColumnSelectorStrategy
|
||||
implements ValueMatcherColumnSelectorStrategy<BaseLongColumnValueSelector>
|
||||
{
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final BaseLongColumnValueSelector selector, final String value)
|
||||
{
|
||||
final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
if (matchVal == null) {
|
||||
return ValueMatcher.primitiveNullValueMatcher(selector);
|
||||
}
|
||||
final long matchValLong = matchVal;
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return selector.getLong() == matchValLong;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(
|
||||
final BaseLongColumnValueSelector selector,
|
||||
DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyLong(selector.getLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueGetter makeValueGetter(final BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return () -> {
|
||||
if (selector.isNull()) {
|
||||
return null;
|
||||
}
|
||||
return new String[]{Long.toString(selector.getLong())};
|
||||
};
|
||||
}
|
||||
}
|
|
@ -23,16 +23,11 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.TreeRangeSet;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Floats;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
||||
|
@ -44,6 +39,7 @@ import java.util.Objects;
|
|||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class SelectorDimFilter implements DimFilter
|
||||
{
|
||||
|
@ -56,11 +52,7 @@ public class SelectorDimFilter implements DimFilter
|
|||
@Nullable
|
||||
private final FilterTuning filterTuning;
|
||||
|
||||
private final Object initLock = new Object();
|
||||
|
||||
private DruidLongPredicate longPredicate;
|
||||
private DruidFloatPredicate floatPredicate;
|
||||
private DruidDoublePredicate druidDoublePredicate;
|
||||
private final DruidPredicateFactory predicateFactory;
|
||||
|
||||
@JsonCreator
|
||||
public SelectorDimFilter(
|
||||
|
@ -76,6 +68,10 @@ public class SelectorDimFilter implements DimFilter
|
|||
this.value = NullHandling.emptyToNullIfNeeded(value);
|
||||
this.extractionFn = extractionFn;
|
||||
this.filterTuning = filterTuning;
|
||||
|
||||
// Create this just in case "toFilter" needs it. It's okay to do this here, because initialization is lazy
|
||||
// (and therefore construction is cheap).
|
||||
this.predicateFactory = new SelectorPredicateFactory(this.value);
|
||||
}
|
||||
|
||||
public SelectorDimFilter(String dimension, String value, @Nullable ExtractionFn extractionFn)
|
||||
|
@ -109,36 +105,6 @@ public class SelectorDimFilter implements DimFilter
|
|||
if (extractionFn == null) {
|
||||
return new SelectorFilter(dimension, value, filterTuning);
|
||||
} else {
|
||||
|
||||
final DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
{
|
||||
return Predicates.equalTo(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
initLongPredicate();
|
||||
return longPredicate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
initFloatPredicate();
|
||||
return floatPredicate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
initDoublePredicate();
|
||||
return druidDoublePredicate;
|
||||
}
|
||||
};
|
||||
return new DimensionPredicateFilter(dimension, predicateFactory, extractionFn, filterTuning);
|
||||
}
|
||||
}
|
||||
|
@ -225,78 +191,4 @@ public class SelectorDimFilter implements DimFilter
|
|||
{
|
||||
return ImmutableSet.of(dimension);
|
||||
}
|
||||
|
||||
|
||||
private void initLongPredicate()
|
||||
{
|
||||
if (longPredicate != null) {
|
||||
return;
|
||||
}
|
||||
synchronized (initLock) {
|
||||
if (longPredicate != null) {
|
||||
return;
|
||||
}
|
||||
if (value == null) {
|
||||
longPredicate = DruidLongPredicate.MATCH_NULL_ONLY;
|
||||
return;
|
||||
}
|
||||
final Long valueAsLong = GuavaUtils.tryParseLong(value);
|
||||
if (valueAsLong == null) {
|
||||
longPredicate = DruidLongPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = valueAsLong.longValue();
|
||||
longPredicate = input -> input == unboxedLong;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void initFloatPredicate()
|
||||
{
|
||||
if (floatPredicate != null) {
|
||||
return;
|
||||
}
|
||||
synchronized (initLock) {
|
||||
if (floatPredicate != null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (value == null) {
|
||||
floatPredicate = DruidFloatPredicate.MATCH_NULL_ONLY;
|
||||
return;
|
||||
}
|
||||
final Float valueAsFloat = Floats.tryParse(value);
|
||||
|
||||
if (valueAsFloat == null) {
|
||||
floatPredicate = DruidFloatPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
final int floatBits = Float.floatToIntBits(valueAsFloat);
|
||||
floatPredicate = input -> Float.floatToIntBits(input) == floatBits;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void initDoublePredicate()
|
||||
{
|
||||
if (druidDoublePredicate != null) {
|
||||
return;
|
||||
}
|
||||
synchronized (initLock) {
|
||||
if (druidDoublePredicate != null) {
|
||||
return;
|
||||
}
|
||||
if (value == null) {
|
||||
druidDoublePredicate = DruidDoublePredicate.MATCH_NULL_ONLY;
|
||||
return;
|
||||
}
|
||||
final Double aDouble = Doubles.tryParse(value);
|
||||
|
||||
if (aDouble == null) {
|
||||
druidDoublePredicate = DruidDoublePredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
final long bits = Double.doubleToLongBits(aDouble);
|
||||
druidDoublePredicate = input -> Double.doubleToLongBits(input) == bits;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,150 @@
|
|||
/*
|
||||
* 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 com.google.common.base.Predicates;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* A {@link DruidPredicateFactory} that checks if input values equal a specific, provided value. Initialization work
|
||||
* is lazy and thread-safe.
|
||||
*/
|
||||
public class SelectorPredicateFactory implements DruidPredicateFactory
|
||||
{
|
||||
@Nullable
|
||||
private final String value;
|
||||
|
||||
private final Object initLock = new Object();
|
||||
|
||||
private volatile DruidLongPredicate longPredicate;
|
||||
private volatile DruidFloatPredicate floatPredicate;
|
||||
private volatile DruidDoublePredicate doublePredicate;
|
||||
|
||||
public SelectorPredicateFactory(@Nullable String value)
|
||||
{
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
{
|
||||
return Predicates.equalTo(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
initLongPredicate();
|
||||
return longPredicate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
initFloatPredicate();
|
||||
return floatPredicate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
initDoublePredicate();
|
||||
return doublePredicate;
|
||||
}
|
||||
|
||||
private void initLongPredicate()
|
||||
{
|
||||
if (longPredicate != null) {
|
||||
return;
|
||||
}
|
||||
synchronized (initLock) {
|
||||
if (longPredicate != null) {
|
||||
return;
|
||||
}
|
||||
if (value == null) {
|
||||
longPredicate = DruidLongPredicate.MATCH_NULL_ONLY;
|
||||
return;
|
||||
}
|
||||
final Long valueAsLong = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
|
||||
if (valueAsLong == null) {
|
||||
longPredicate = DruidLongPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = valueAsLong;
|
||||
longPredicate = input -> input == unboxedLong;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void initFloatPredicate()
|
||||
{
|
||||
if (floatPredicate != null) {
|
||||
return;
|
||||
}
|
||||
synchronized (initLock) {
|
||||
if (floatPredicate != null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (value == null) {
|
||||
floatPredicate = DruidFloatPredicate.MATCH_NULL_ONLY;
|
||||
return;
|
||||
}
|
||||
final Float valueAsFloat = DimensionHandlerUtils.convertObjectToFloat(value);
|
||||
|
||||
if (valueAsFloat == null) {
|
||||
floatPredicate = DruidFloatPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// Compare with floatToIntBits instead of == to canonicalize NaNs.
|
||||
final int floatBits = Float.floatToIntBits(valueAsFloat);
|
||||
floatPredicate = input -> Float.floatToIntBits(input) == floatBits;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void initDoublePredicate()
|
||||
{
|
||||
if (doublePredicate != null) {
|
||||
return;
|
||||
}
|
||||
synchronized (initLock) {
|
||||
if (doublePredicate != null) {
|
||||
return;
|
||||
}
|
||||
if (value == null) {
|
||||
doublePredicate = DruidDoublePredicate.MATCH_NULL_ONLY;
|
||||
return;
|
||||
}
|
||||
final Double aDouble = DimensionHandlerUtils.convertObjectToDouble(value);
|
||||
|
||||
if (aDouble == null) {
|
||||
doublePredicate = DruidDoublePredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
// Compare with doubleToLongBits instead of == to canonicalize NaNs.
|
||||
final long bits = Double.doubleToLongBits(aDouble);
|
||||
doublePredicate = input -> Double.doubleToLongBits(input) == bits;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,131 +0,0 @@
|
|||
/*
|
||||
* 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.common.config.NullHandling;
|
||||
import org.apache.druid.segment.DimensionDictionarySelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
|
||||
{
|
||||
private static final String[] NULL_VALUE = new String[]{null};
|
||||
private static final ValueGetter NULL_VALUE_GETTER = () -> NULL_VALUE;
|
||||
|
||||
private final boolean hasMultipleValues;
|
||||
|
||||
public StringValueMatcherColumnSelectorStrategy(final boolean hasMultipleValues)
|
||||
{
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static Boolean toBooleanIfPossible(
|
||||
final DimensionDictionarySelector selector,
|
||||
final boolean hasMultipleValues,
|
||||
final Predicate<String> predicate
|
||||
)
|
||||
{
|
||||
if (selector.getValueCardinality() == 0) {
|
||||
// Column has no values (it doesn't exist, or it's all empty arrays).
|
||||
// Match if and only if "predicate" matches null.
|
||||
return predicate.apply(null);
|
||||
} else if (!hasMultipleValues && selector.getValueCardinality() == 1 && selector.nameLookupPossibleInAdvance()) {
|
||||
// Every row has the same value. Match if and only if "predicate" matches the possible value.
|
||||
return predicate.apply(selector.lookupName(0));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static ValueMatcher toBooleanMatcherIfPossible(
|
||||
final DimensionSelector selector,
|
||||
final boolean hasMultipleValues,
|
||||
final Predicate<String> predicate
|
||||
)
|
||||
{
|
||||
final Boolean booleanValue = StringValueMatcherColumnSelectorStrategy.toBooleanIfPossible(
|
||||
selector,
|
||||
hasMultipleValues,
|
||||
predicate
|
||||
);
|
||||
return booleanValue == null ? null : BooleanValueMatcher.of(booleanValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final DimensionSelector selector, final String value)
|
||||
{
|
||||
final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(
|
||||
selector,
|
||||
hasMultipleValues,
|
||||
s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
|
||||
);
|
||||
|
||||
if (booleanMatcher != null) {
|
||||
return booleanMatcher;
|
||||
} else {
|
||||
return selector.makeValueMatcher(value);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(
|
||||
final DimensionSelector selector,
|
||||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate);
|
||||
|
||||
if (booleanMatcher != null) {
|
||||
return booleanMatcher;
|
||||
} else {
|
||||
return selector.makeValueMatcher(predicate);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueGetter makeValueGetter(final DimensionSelector selector)
|
||||
{
|
||||
if (selector.getValueCardinality() == 0) {
|
||||
return NULL_VALUE_GETTER;
|
||||
} else {
|
||||
return () -> {
|
||||
final IndexedInts row = selector.getRow();
|
||||
final int size = row.size();
|
||||
if (size == 0) {
|
||||
return NULL_VALUE;
|
||||
} else {
|
||||
String[] values = new String[size];
|
||||
for (int i = 0; i < size; ++i) {
|
||||
values[i] = selector.lookupName(row.get(i));
|
||||
}
|
||||
return values;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,37 +0,0 @@
|
|||
/*
|
||||
* 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 javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ValueGetter
|
||||
{
|
||||
/**
|
||||
* It is not ideal that Long and Float values will get
|
||||
* converted to strings. We should also add functions
|
||||
* for these and modify ColumnComparisonFilter to handle
|
||||
* comparing Long and Float columns to eachother.
|
||||
* Returns null when the underlying Long/Float value is null.
|
||||
*/
|
||||
@Nullable
|
||||
String[] get();
|
||||
}
|
|
@ -21,8 +21,6 @@ package org.apache.druid.query.filter;
|
|||
|
||||
import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||
import org.apache.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseNullableColumnValueSelector;
|
||||
|
||||
/**
|
||||
* An object that returns a boolean indicating if the "current" row should be selected or not. The most prominent use
|
||||
|
@ -35,28 +33,4 @@ public interface ValueMatcher extends HotLoopCallee
|
|||
{
|
||||
@CalledFromHotLoop
|
||||
boolean matches();
|
||||
|
||||
// Utility method to match null values.
|
||||
|
||||
/**
|
||||
* Returns a ValueMatcher that matches when the primitive long, double, or float value from {@code selector}
|
||||
* should be treated as null.
|
||||
*/
|
||||
static ValueMatcher primitiveNullValueMatcher(BaseNullableColumnValueSelector selector)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return selector.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,51 +0,0 @@
|
|||
/*
|
||||
* 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 org.apache.druid.query.dimension.ColumnSelectorStrategy;
|
||||
|
||||
public interface ValueMatcherColumnSelectorStrategy<ValueSelectorType> extends ColumnSelectorStrategy
|
||||
{
|
||||
/**
|
||||
* Create a single value ValueMatcher.
|
||||
*
|
||||
* @param selector Column selector
|
||||
* @param value Value to match against
|
||||
* @return ValueMatcher that matches on 'value'
|
||||
*/
|
||||
ValueMatcher makeValueMatcher(ValueSelectorType selector, String value);
|
||||
|
||||
/**
|
||||
* Create a predicate-based ValueMatcher.
|
||||
*
|
||||
* @param selector Column selector
|
||||
* @param predicateFactory A DruidPredicateFactory that provides the filter predicates to be matched
|
||||
* @return A ValueMatcher that applies the predicate for this DimensionQueryHelper's value type from the predicateFactory
|
||||
*/
|
||||
ValueMatcher makeValueMatcher(ValueSelectorType selector, DruidPredicateFactory predicateFactory);
|
||||
|
||||
/**
|
||||
* Create a ValueGetter.
|
||||
*
|
||||
* @param selector Column selector
|
||||
* @return A ValueGetter that returns the value(s) of the selected column
|
||||
*/
|
||||
ValueGetter makeValueGetter(ValueSelectorType selector);
|
||||
}
|
|
@ -1,63 +0,0 @@
|
|||
/*
|
||||
* 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 org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
public class ValueMatcherColumnSelectorStrategyFactory
|
||||
implements ColumnSelectorStrategyFactory<ValueMatcherColumnSelectorStrategy>
|
||||
{
|
||||
private static final ValueMatcherColumnSelectorStrategyFactory INSTANCE = new ValueMatcherColumnSelectorStrategyFactory();
|
||||
|
||||
private ValueMatcherColumnSelectorStrategyFactory()
|
||||
{
|
||||
// Singleton.
|
||||
}
|
||||
|
||||
public static ValueMatcherColumnSelectorStrategyFactory instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcherColumnSelectorStrategy makeColumnSelectorStrategy(
|
||||
ColumnCapabilities capabilities,
|
||||
ColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
ValueType type = capabilities.getType();
|
||||
switch (type) {
|
||||
case STRING:
|
||||
return new StringValueMatcherColumnSelectorStrategy(capabilities.hasMultipleValues());
|
||||
case LONG:
|
||||
return new LongValueMatcherColumnSelectorStrategy();
|
||||
case FLOAT:
|
||||
return new FloatValueMatcherColumnSelectorStrategy();
|
||||
case DOUBLE:
|
||||
return new DoubleValueMatcherColumnSelectorStrategy();
|
||||
default:
|
||||
throw new IAE("Cannot create column selector strategy from invalid type [%s]", type);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -22,8 +22,8 @@ package org.apache.druid.query.filter.vector;
|
|||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.StringValueMatcherColumnSelectorStrategy;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.filter.ValueMatchers;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -45,7 +45,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
final Predicate<String> predicate
|
||||
)
|
||||
{
|
||||
final Boolean booleanValue = StringValueMatcherColumnSelectorStrategy.toBooleanIfPossible(
|
||||
final Boolean booleanValue = ValueMatchers.toBooleanIfPossible(
|
||||
selector,
|
||||
false,
|
||||
predicate
|
||||
|
|
|
@ -374,7 +374,12 @@ public class RowBasedGrouperHelper
|
|||
}
|
||||
};
|
||||
|
||||
return RowBasedColumnSelectorFactory.create(adapter, supplier::get, GroupByQueryHelper.rowSignatureFor(query));
|
||||
return RowBasedColumnSelectorFactory.create(
|
||||
adapter,
|
||||
supplier::get,
|
||||
GroupByQueryHelper.rowSignatureFor(query),
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.druid.query.aggregation.MetricManipulationFn;
|
|||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.context.ResponseContext;
|
||||
import org.apache.druid.segment.RowAdapters;
|
||||
import org.apache.druid.segment.RowBasedColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -212,11 +213,16 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
|
|||
Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()];
|
||||
String[] aggregatorNames = new String[aggregatorSpecs.size()];
|
||||
for (int i = 0; i < aggregatorSpecs.size(); i++) {
|
||||
aggregators[i] = aggregatorSpecs.get(i)
|
||||
.factorize(RowBasedColumnSelectorFactory.create(() -> new MapBasedRow(
|
||||
null,
|
||||
null
|
||||
), null));
|
||||
aggregators[i] =
|
||||
aggregatorSpecs.get(i)
|
||||
.factorize(
|
||||
RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
() -> new MapBasedRow(null, null),
|
||||
null,
|
||||
false
|
||||
)
|
||||
);
|
||||
aggregatorNames[i] = aggregatorSpecs.get(i).getName();
|
||||
}
|
||||
final DateTime start = query.getIntervals().isEmpty() ? DateTimes.EPOCH : query.getIntervals().get(0).getStart();
|
||||
|
|
|
@ -47,13 +47,39 @@ public interface ColumnProcessorFactory<T>
|
|||
*/
|
||||
ValueType defaultType();
|
||||
|
||||
T makeDimensionProcessor(DimensionSelector selector);
|
||||
/**
|
||||
* Create a processor for a string column.
|
||||
*
|
||||
* @param selector dimension selector
|
||||
* @param multiValue whether the selector *might* have multiple values
|
||||
*/
|
||||
T makeDimensionProcessor(DimensionSelector selector, boolean multiValue);
|
||||
|
||||
/**
|
||||
* Create a processor for a float column.
|
||||
*
|
||||
* @param selector float selector
|
||||
*/
|
||||
T makeFloatProcessor(BaseFloatColumnValueSelector selector);
|
||||
|
||||
/**
|
||||
* Create a processor for a double column.
|
||||
*
|
||||
* @param selector double selector
|
||||
*/
|
||||
T makeDoubleProcessor(BaseDoubleColumnValueSelector selector);
|
||||
|
||||
/**
|
||||
* Create a processor for a long column.
|
||||
*
|
||||
* @param selector long selector
|
||||
*/
|
||||
T makeLongProcessor(BaseLongColumnValueSelector selector);
|
||||
|
||||
/**
|
||||
* Create a processor for a complex column.
|
||||
*
|
||||
* @param selector object selector
|
||||
*/
|
||||
T makeComplexProcessor(BaseObjectColumnValueSelector<?> selector);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,9 @@ import com.google.common.base.Function;
|
|||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.virtual.ExpressionSelectors;
|
||||
|
||||
|
@ -54,7 +56,7 @@ public class ColumnProcessors
|
|||
)
|
||||
{
|
||||
return makeProcessorInternal(
|
||||
factory -> getColumnType(factory, column),
|
||||
factory -> factory.getColumnCapabilities(column),
|
||||
factory -> factory.makeDimensionSelector(DefaultDimensionSpec.of(column)),
|
||||
factory -> factory.makeColumnValueSelector(column),
|
||||
processorFactory,
|
||||
|
@ -62,12 +64,52 @@ public class ColumnProcessors
|
|||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Make a processor for a particular {@link DimensionSpec}.
|
||||
*
|
||||
* @param dimensionSpec the dimension spec
|
||||
* @param processorFactory the processor factory
|
||||
* @param selectorFactory the column selector factory
|
||||
* @param <T> processor type
|
||||
*/
|
||||
public static <T> T makeProcessor(
|
||||
final DimensionSpec dimensionSpec,
|
||||
final ColumnProcessorFactory<T> processorFactory,
|
||||
final ColumnSelectorFactory selectorFactory
|
||||
)
|
||||
{
|
||||
return makeProcessorInternal(
|
||||
factory -> {
|
||||
// Capabilities of the column that the dimensionSpec is reading. We can't return these straight-up, because
|
||||
// the _result_ of the dimensionSpec might have different capabilities. But what we return will generally be
|
||||
// based on them.
|
||||
final ColumnCapabilities dimensionCapabilities = factory.getColumnCapabilities(dimensionSpec.getDimension());
|
||||
|
||||
if (dimensionSpec.getExtractionFn() != null || dimensionSpec.mustDecorate()) {
|
||||
// DimensionSpec is doing some sort of transformation. The result is always a string.
|
||||
|
||||
return new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.STRING)
|
||||
.setHasMultipleValues(dimensionSpec.mustDecorate() || mayBeMultiValue(dimensionCapabilities));
|
||||
} else {
|
||||
// No transformation. Pass through.
|
||||
return dimensionCapabilities;
|
||||
}
|
||||
},
|
||||
factory -> factory.makeDimensionSelector(dimensionSpec),
|
||||
factory -> factory.makeColumnValueSelector(dimensionSpec.getDimension()),
|
||||
processorFactory,
|
||||
selectorFactory
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Make a processor for a particular expression. If the expression is a simple identifier, this behaves identically
|
||||
* to {@link #makeProcessor(String, ColumnProcessorFactory, ColumnSelectorFactory)} and accesses the column directly.
|
||||
* Otherwise, it uses an expression selector of type {@code exprTypeHint}.
|
||||
*
|
||||
* @param expr the parsed expression
|
||||
* @param exprTypeHint expression selector type to use for exprs that are not simple identifiers
|
||||
* @param processorFactory the processor factory
|
||||
* @param selectorFactory the column selector factory
|
||||
* @param <T> processor type
|
||||
|
@ -84,7 +126,7 @@ public class ColumnProcessors
|
|||
return makeProcessor(expr.getBindingIfIdentifier(), processorFactory, selectorFactory);
|
||||
} else {
|
||||
return makeProcessorInternal(
|
||||
factory -> exprTypeHint,
|
||||
factory -> new ColumnCapabilitiesImpl().setType(exprTypeHint).setHasMultipleValues(true),
|
||||
factory -> ExpressionSelectors.makeDimensionSelector(factory, expr, null),
|
||||
factory -> ExpressionSelectors.makeColumnValueSelector(factory, expr),
|
||||
processorFactory,
|
||||
|
@ -97,8 +139,10 @@ public class ColumnProcessors
|
|||
* Creates "column processors", which are objects that wrap a single input column and provide some
|
||||
* functionality on top of it.
|
||||
*
|
||||
* @param inputTypeFn function that returns the "natural" input type of the column being processed. This is
|
||||
* permitted to return null; if it does, then processorFactory.defaultType() will be used.
|
||||
* @param inputCapabilitiesFn function that returns capabilities of the column being processed. The type provided
|
||||
* by these capabilities will be used to determine what kind of selector to create. If
|
||||
* this function returns null, then processorFactory.defaultType() will be
|
||||
* used to construct a set of assumed capabilities.
|
||||
* @param dimensionSelectorFn function that creates a DimensionSelector for the column being processed. Will be
|
||||
* called if the column type is string.
|
||||
* @param valueSelectorFunction function that creates a ColumnValueSelector for the column being processed. Will be
|
||||
|
@ -109,19 +153,22 @@ public class ColumnProcessors
|
|||
* @see DimensionHandlerUtils#makeVectorProcessor the vectorized version
|
||||
*/
|
||||
private static <T> T makeProcessorInternal(
|
||||
final Function<ColumnSelectorFactory, ValueType> inputTypeFn,
|
||||
final Function<ColumnSelectorFactory, ColumnCapabilities> inputCapabilitiesFn,
|
||||
final Function<ColumnSelectorFactory, DimensionSelector> dimensionSelectorFn,
|
||||
final Function<ColumnSelectorFactory, ColumnValueSelector<?>> valueSelectorFunction,
|
||||
final ColumnProcessorFactory<T> processorFactory,
|
||||
final ColumnSelectorFactory selectorFactory
|
||||
)
|
||||
{
|
||||
final ValueType type = inputTypeFn.apply(selectorFactory);
|
||||
final ValueType effectiveType = type != null ? type : processorFactory.defaultType();
|
||||
final ColumnCapabilities capabilities = inputCapabilitiesFn.apply(selectorFactory);
|
||||
final ValueType effectiveType = capabilities != null ? capabilities.getType() : processorFactory.defaultType();
|
||||
|
||||
switch (effectiveType) {
|
||||
case STRING:
|
||||
return processorFactory.makeDimensionProcessor(dimensionSelectorFn.apply(selectorFactory));
|
||||
return processorFactory.makeDimensionProcessor(
|
||||
dimensionSelectorFn.apply(selectorFactory),
|
||||
mayBeMultiValue(capabilities)
|
||||
);
|
||||
case LONG:
|
||||
return processorFactory.makeLongProcessor(valueSelectorFunction.apply(selectorFactory));
|
||||
case FLOAT:
|
||||
|
@ -135,10 +182,12 @@ public class ColumnProcessors
|
|||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static ValueType getColumnType(final ColumnSelectorFactory selectorFactory, final String columnName)
|
||||
/**
|
||||
* Returns true if a given set of capabilities might indicate an underlying multi-value column. Errs on the side
|
||||
* of returning true if unknown; i.e. if this returns false, there are _definitely not_ mul.
|
||||
*/
|
||||
private static boolean mayBeMultiValue(@Nullable final ColumnCapabilities capabilities)
|
||||
{
|
||||
final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(columnName);
|
||||
return capabilities == null ? null : capabilities.getType();
|
||||
return capabilities == null || !capabilities.isComplete() || capabilities.hasMultipleValues();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -519,9 +519,4 @@ public final class DimensionHandlerUtils
|
|||
{
|
||||
return number == null ? ZERO_FLOAT : number;
|
||||
}
|
||||
|
||||
public static Number nullToZero(@Nullable Number number)
|
||||
{
|
||||
return number == null ? ZERO_DOUBLE : number;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import org.apache.druid.data.input.Row;
|
||||
|
||||
import java.util.function.Function;
|
||||
import java.util.function.ToLongFunction;
|
||||
|
||||
/**
|
||||
* Utility class for creating {@link RowAdapter}.
|
||||
*/
|
||||
public class RowAdapters
|
||||
{
|
||||
private static final RowAdapter<? extends Row> STANDARD_ROW_ADAPTER = new RowAdapter<Row>()
|
||||
{
|
||||
@Override
|
||||
public ToLongFunction<Row> timestampFunction()
|
||||
{
|
||||
return Row::getTimestampFromEpoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Row, Object> columnFunction(String columnName)
|
||||
{
|
||||
return r -> r.getRaw(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
private RowAdapters()
|
||||
{
|
||||
// No instantiation.
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link RowAdapter} that handles any kind of input {@link Row}.
|
||||
*/
|
||||
public static <RowType extends Row> RowAdapter<RowType> standardRow()
|
||||
{
|
||||
//noinspection unchecked
|
||||
return (RowAdapter<RowType>) STANDARD_ROW_ADAPTER;
|
||||
}
|
||||
}
|
|
@ -22,7 +22,6 @@ package org.apache.druid.segment;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.data.input.Rows;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
|
@ -43,54 +42,48 @@ import java.util.function.Function;
|
|||
import java.util.function.Supplier;
|
||||
import java.util.function.ToLongFunction;
|
||||
|
||||
/**
|
||||
* A {@link ColumnSelectorFactory} that is based on an object supplier and a {@link RowAdapter} for that type of object.
|
||||
*/
|
||||
public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
||||
{
|
||||
private static final RowAdapter<? extends Row> STANDARD_ROW_ADAPTER = new RowAdapter<Row>()
|
||||
{
|
||||
@Override
|
||||
public ToLongFunction<Row> timestampFunction()
|
||||
{
|
||||
return Row::getTimestampFromEpoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Row, Object> columnFunction(String columnName)
|
||||
{
|
||||
return r -> r.getRaw(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
private final Supplier<T> supplier;
|
||||
private final RowAdapter<T> adapter;
|
||||
private final Map<String, ValueType> rowSignature;
|
||||
private final boolean throwParseExceptions;
|
||||
|
||||
private RowBasedColumnSelectorFactory(
|
||||
final Supplier<T> supplier,
|
||||
final RowAdapter<T> adapter,
|
||||
@Nullable final Map<String, ValueType> rowSignature
|
||||
@Nullable final Map<String, ValueType> rowSignature,
|
||||
final boolean throwParseExceptions
|
||||
)
|
||||
{
|
||||
this.supplier = supplier;
|
||||
this.adapter = adapter;
|
||||
this.rowSignature = rowSignature != null ? rowSignature : ImmutableMap.of();
|
||||
this.throwParseExceptions = throwParseExceptions;
|
||||
}
|
||||
|
||||
public static <RowType extends Row> RowBasedColumnSelectorFactory create(
|
||||
final Supplier<RowType> supplier,
|
||||
@Nullable final Map<String, ValueType> signature
|
||||
)
|
||||
{
|
||||
//noinspection unchecked
|
||||
return new RowBasedColumnSelectorFactory<>(supplier, (RowAdapter<RowType>) STANDARD_ROW_ADAPTER, signature);
|
||||
}
|
||||
|
||||
public static <RowType> RowBasedColumnSelectorFactory create(
|
||||
/**
|
||||
* Create an instance based on any object, along with a {@link RowAdapter} for that object.
|
||||
*
|
||||
* @param adapter adapter for these row objects
|
||||
* @param supplier supplier of row objects
|
||||
* @param signature will be used for reporting available columns and their capabilities. Note that the this
|
||||
* factory will still allow creation of selectors on any field in the rows, even if it
|
||||
* doesn't appear in "rowSignature".
|
||||
* @param throwParseExceptions whether numeric selectors should throw parse exceptions or use a default/null value
|
||||
* when their inputs are not actually numeric
|
||||
*/
|
||||
public static <RowType> RowBasedColumnSelectorFactory<RowType> create(
|
||||
final RowAdapter<RowType> adapter,
|
||||
final Supplier<RowType> supplier,
|
||||
@Nullable final Map<String, ValueType> signature
|
||||
@Nullable final Map<String, ValueType> signature,
|
||||
final boolean throwParseExceptions
|
||||
)
|
||||
{
|
||||
return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature);
|
||||
return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature, throwParseExceptions);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -106,7 +99,8 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
final ValueType valueType = rowSignature.get(columnName);
|
||||
|
||||
// Do _not_ set isDictionaryEncoded or hasBitmapIndexes, because Row-based columns do not have those things.
|
||||
return valueType != null ? new ColumnCapabilitiesImpl().setType(valueType) : null;
|
||||
// Do set hasMultipleValues, because we might return multiple values.
|
||||
return valueType != null ? new ColumnCapabilitiesImpl().setType(valueType).setHasMultipleValues(true) : null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -365,47 +359,47 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
} else {
|
||||
final Function<T, Object> columnFunction = adapter.columnFunction(columnName);
|
||||
|
||||
return new ColumnValueSelector()
|
||||
return new ColumnValueSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return columnFunction.apply(supplier.get()) == null;
|
||||
return !NullHandling.replaceWithDefault() && getCurrentValueAsNumber() == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
Number metric = Rows.objectToNumber(columnName, columnFunction.apply(supplier.get()));
|
||||
assert NullHandling.replaceWithDefault() || metric != null;
|
||||
return DimensionHandlerUtils.nullToZero(metric).doubleValue();
|
||||
final Number n = getCurrentValueAsNumber();
|
||||
assert NullHandling.replaceWithDefault() || n != null;
|
||||
return n != null ? n.doubleValue() : 0d;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
Number metric = Rows.objectToNumber(columnName, columnFunction.apply(supplier.get()));
|
||||
assert NullHandling.replaceWithDefault() || metric != null;
|
||||
return DimensionHandlerUtils.nullToZero(metric).floatValue();
|
||||
final Number n = getCurrentValueAsNumber();
|
||||
assert NullHandling.replaceWithDefault() || n != null;
|
||||
return n != null ? n.floatValue() : 0f;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
Number metric = Rows.objectToNumber(columnName, columnFunction.apply(supplier.get()));
|
||||
assert NullHandling.replaceWithDefault() || metric != null;
|
||||
return DimensionHandlerUtils.nullToZero(metric).longValue();
|
||||
final Number n = getCurrentValueAsNumber();
|
||||
assert NullHandling.replaceWithDefault() || n != null;
|
||||
return n != null ? n.longValue() : 0L;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
return columnFunction.apply(supplier.get());
|
||||
return getCurrentValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
public Class<Object> classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
@ -415,6 +409,22 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
{
|
||||
inspector.visit("row", supplier);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private Object getCurrentValue()
|
||||
{
|
||||
return columnFunction.apply(supplier.get());
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private Number getCurrentValueAsNumber()
|
||||
{
|
||||
return Rows.objectToNumber(
|
||||
columnName,
|
||||
getCurrentValue(),
|
||||
throwParseExceptions
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,9 +34,10 @@ public interface ColumnCapabilities
|
|||
|
||||
/**
|
||||
* This property indicates that this {@link ColumnCapabilities} is "complete" in that all properties can be expected
|
||||
* to supply valid responses. Not all {@link ColumnCapabilities} are created equal. Some, such as those provided by
|
||||
* {@link org.apache.druid.segment.RowBasedColumnSelectorFactory} only have type information, if even that, and
|
||||
* cannot supply information like {@link ColumnCapabilities#hasMultipleValues}, and will report as false.
|
||||
* to supply valid responses. This is mostly a hack to work around {@link ColumnCapabilities} generators that
|
||||
* fail to set {@link #hasMultipleValues()} even when the associated column really could have multiple values.
|
||||
* Until this situation is sorted out, if this method returns false, callers are encouraged to ignore
|
||||
* {@link #hasMultipleValues()} and treat that property as if it were unknown.
|
||||
*/
|
||||
boolean isComplete();
|
||||
}
|
||||
|
|
|
@ -21,34 +21,36 @@ package org.apache.druid.segment.filter;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.ColumnSelectorPlus;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.filter.BitmapIndexSelector;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueGetter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.filter.ValueMatcherColumnSelectorStrategy;
|
||||
import org.apache.druid.query.filter.ValueMatcherColumnSelectorStrategyFactory;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnProcessorFactory;
|
||||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ColumnComparisonFilter implements Filter
|
||||
{
|
||||
private final List<DimensionSpec> dimensions;
|
||||
|
||||
public ColumnComparisonFilter(
|
||||
final List<DimensionSpec> dimensions
|
||||
)
|
||||
public ColumnComparisonFilter(final List<DimensionSpec> dimensions)
|
||||
{
|
||||
this.dimensions = Preconditions.checkNotNull(dimensions, "dimensions");
|
||||
}
|
||||
|
@ -62,37 +64,31 @@ public class ColumnComparisonFilter implements Filter
|
|||
@Override
|
||||
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||
{
|
||||
final ValueGetter[] valueGetters = new ValueGetter[dimensions.size()];
|
||||
final List<Supplier<String[]>> valueGetters = new ArrayList<>(dimensions.size());
|
||||
|
||||
for (int i = 0; i < dimensions.size(); i++) {
|
||||
final ColumnSelectorPlus<ValueMatcherColumnSelectorStrategy> selector =
|
||||
DimensionHandlerUtils.createColumnSelectorPlus(
|
||||
ValueMatcherColumnSelectorStrategyFactory.instance(),
|
||||
dimensions.get(i),
|
||||
factory
|
||||
);
|
||||
|
||||
valueGetters[i] = selector.getColumnSelectorStrategy().makeValueGetter(selector.getSelector());
|
||||
for (final DimensionSpec dimension : dimensions) {
|
||||
valueGetters.add(ColumnProcessors.makeProcessor(dimension, ColumnComparisonReaderFactory.INSTANCE, factory));
|
||||
}
|
||||
|
||||
return makeValueMatcher(valueGetters);
|
||||
}
|
||||
|
||||
public static ValueMatcher makeValueMatcher(final ValueGetter[] valueGetters)
|
||||
public static ValueMatcher makeValueMatcher(final List<Supplier<String[]>> valueGetters)
|
||||
{
|
||||
if (valueGetters.length == 0) {
|
||||
if (valueGetters.isEmpty()) {
|
||||
return BooleanValueMatcher.of(true);
|
||||
}
|
||||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
// Keep all values to compare against each other.
|
||||
String[][] values = new String[valueGetters.length][];
|
||||
String[][] values = new String[valueGetters.size()][];
|
||||
|
||||
for (int i = 0; i < valueGetters.length; i++) {
|
||||
values[i] = valueGetters[i].get();
|
||||
for (int i = 0; i < valueGetters.size(); i++) {
|
||||
values[i] = valueGetters.get(i).get();
|
||||
// Compare the new values to the values we already got.
|
||||
for (int j = 0; j < i; j++) {
|
||||
if (!overlap(values[i], values[j])) {
|
||||
|
@ -107,7 +103,7 @@ public class ColumnComparisonFilter implements Filter
|
|||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
// All value getters are likely the same or similar (in terms of runtime shape), so inspecting only one of them.
|
||||
inspector.visit("oneValueGetter", valueGetters[0]);
|
||||
inspector.visit("oneValueGetter", valueGetters.get(0));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -166,4 +162,73 @@ public class ColumnComparisonFilter implements Filter
|
|||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
private static class ColumnComparisonReaderFactory implements ColumnProcessorFactory<Supplier<String[]>>
|
||||
{
|
||||
private static final ColumnComparisonReaderFactory INSTANCE = new ColumnComparisonReaderFactory();
|
||||
private static final String[] NULL_VALUE = new String[]{null};
|
||||
|
||||
@Override
|
||||
public ValueType defaultType()
|
||||
{
|
||||
return ValueType.STRING;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
return () -> {
|
||||
final IndexedInts row = selector.getRow();
|
||||
final int size = row.size();
|
||||
if (size == 0) {
|
||||
return NULL_VALUE;
|
||||
} else {
|
||||
String[] values = new String[size];
|
||||
for (int i = 0; i < size; ++i) {
|
||||
values[i] = selector.lookupName(row.get(i));
|
||||
}
|
||||
return values;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeFloatProcessor(BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return () -> {
|
||||
if (selector.isNull()) {
|
||||
return NULL_VALUE;
|
||||
}
|
||||
return new String[]{Float.toString(selector.getFloat())};
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return () -> {
|
||||
if (selector.isNull()) {
|
||||
return NULL_VALUE;
|
||||
}
|
||||
return new String[]{Double.toString(selector.getDouble())};
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeLongProcessor(BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return () -> {
|
||||
if (selector.isNull()) {
|
||||
return NULL_VALUE;
|
||||
}
|
||||
return new String[]{Long.toString(selector.getLong())};
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String[]> makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
return () -> NULL_VALUE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.filter;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.SelectorPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnProcessorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Creates {@link ValueMatcher} that match constants.
|
||||
*/
|
||||
public class ConstantValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
|
||||
{
|
||||
@Nullable
|
||||
private final String matchValue;
|
||||
|
||||
ConstantValueMatcherFactory(@Nullable String matchValue)
|
||||
{
|
||||
this.matchValue = NullHandling.emptyToNullIfNeeded(matchValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType defaultType()
|
||||
{
|
||||
return ValueType.COMPLEX;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
return ValueMatchers.makeStringValueMatcher(selector, matchValue, multiValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeFloatValueMatcher(selector, matchValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeDoubleValueMatcher(selector, matchValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeLongValueMatcher(selector, matchValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
return new PredicateValueMatcherFactory(new SelectorPredicateFactory(matchValue)).makeComplexProcessor(selector);
|
||||
}
|
||||
}
|
|
@ -30,9 +30,7 @@ import it.unimi.dsi.fastutil.ints.IntList;
|
|||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.java.util.common.guava.FunctionalIterable;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.ColumnSelectorPlus;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.filter.BitmapIndexSelector;
|
||||
import org.apache.druid.query.filter.BooleanFilter;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
|
@ -40,11 +38,9 @@ import org.apache.druid.query.filter.DruidPredicateFactory;
|
|||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.filter.ValueMatcherColumnSelectorStrategy;
|
||||
import org.apache.druid.query.filter.ValueMatcherColumnSelectorStrategyFactory;
|
||||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.IntIteratorUtils;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
|
@ -60,6 +56,7 @@ import java.util.List;
|
|||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class Filters
|
||||
{
|
||||
|
@ -120,14 +117,11 @@ public class Filters
|
|||
final String value
|
||||
)
|
||||
{
|
||||
final ColumnSelectorPlus<ValueMatcherColumnSelectorStrategy> selector =
|
||||
DimensionHandlerUtils.createColumnSelectorPlus(
|
||||
ValueMatcherColumnSelectorStrategyFactory.instance(),
|
||||
DefaultDimensionSpec.of(columnName),
|
||||
columnSelectorFactory
|
||||
);
|
||||
|
||||
return selector.getColumnSelectorStrategy().makeValueMatcher(selector.getSelector(), value);
|
||||
return ColumnProcessors.makeProcessor(
|
||||
columnName,
|
||||
new ConstantValueMatcherFactory(value),
|
||||
columnSelectorFactory
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -151,14 +145,11 @@ public class Filters
|
|||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final ColumnSelectorPlus<ValueMatcherColumnSelectorStrategy> selector =
|
||||
DimensionHandlerUtils.createColumnSelectorPlus(
|
||||
ValueMatcherColumnSelectorStrategyFactory.instance(),
|
||||
DefaultDimensionSpec.of(columnName),
|
||||
columnSelectorFactory
|
||||
);
|
||||
|
||||
return selector.getColumnSelectorStrategy().makeValueMatcher(selector.getSelector(), predicateFactory);
|
||||
return ColumnProcessors.makeProcessor(
|
||||
columnName,
|
||||
new PredicateValueMatcherFactory(predicateFactory),
|
||||
columnSelectorFactory
|
||||
);
|
||||
}
|
||||
|
||||
public static ImmutableBitmap allFalse(final BitmapIndexSelector selector)
|
||||
|
@ -217,10 +208,11 @@ public class Filters
|
|||
/**
|
||||
* Return the union of bitmaps for all values matching a particular predicate.
|
||||
*
|
||||
* @param dimension dimension to look at
|
||||
* @param selector bitmap selector
|
||||
* @param dimension dimension to look at
|
||||
* @param selector bitmap selector
|
||||
* @param bitmapResultFactory
|
||||
* @param predicate predicate to use
|
||||
* @param predicate predicate to use
|
||||
*
|
||||
* @return bitmap of matching rows
|
||||
*
|
||||
* @see #estimateSelectivity(String, BitmapIndexSelector, Predicate)
|
||||
|
@ -616,9 +608,9 @@ public class Filters
|
|||
/**
|
||||
* This method provides a "standard" implementation of {@link Filter#shouldUseBitmapIndex(BitmapIndexSelector)} which takes
|
||||
* a {@link Filter}, a {@link BitmapIndexSelector}, and {@link FilterTuning} to determine if:
|
||||
* a) the filter supports bitmap indexes for all required columns
|
||||
* b) the filter tuning specifies that it should use the index
|
||||
* c) the cardinality of the column is above the minimum threshold and below the maximum threshold to use the index
|
||||
* a) the filter supports bitmap indexes for all required columns
|
||||
* b) the filter tuning specifies that it should use the index
|
||||
* c) the cardinality of the column is above the minimum threshold and below the maximum threshold to use the index
|
||||
*
|
||||
* If all these things are true, {@link org.apache.druid.segment.QueryableIndexStorageAdapter} will utilize the
|
||||
* indexes.
|
||||
|
@ -646,9 +638,10 @@ public class Filters
|
|||
* Create a filter representing an AND relationship across a list of filters.
|
||||
*
|
||||
* @param filterList List of filters
|
||||
*
|
||||
* @return If filterList has more than one element, return an AND filter composed of the filters from filterList
|
||||
* If filterList has a single element, return that element alone
|
||||
* If filterList is empty, return null
|
||||
* If filterList has a single element, return that element alone
|
||||
* If filterList is empty, return null
|
||||
*/
|
||||
@Nullable
|
||||
public static Filter and(List<Filter> filterList)
|
||||
|
|
|
@ -0,0 +1,205 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.Rows;
|
||||
import org.apache.druid.query.filter.DruidDoublePredicate;
|
||||
import org.apache.druid.query.filter.DruidFloatPredicate;
|
||||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnProcessorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.NilColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Creates {@link ValueMatcher} that apply a predicate to each value.
|
||||
*/
|
||||
public class PredicateValueMatcherFactory implements ColumnProcessorFactory<ValueMatcher>
|
||||
{
|
||||
private final DruidPredicateFactory predicateFactory;
|
||||
|
||||
PredicateValueMatcherFactory(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
this.predicateFactory = predicateFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType defaultType()
|
||||
{
|
||||
// Set default type to COMPLEX, so when the underlying type is unknown, we go into "makeComplexProcessor", which
|
||||
// uses per-row type detection.
|
||||
return ValueType.COMPLEX;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
return ValueMatchers.makeStringValueMatcher(selector, predicateFactory, multiValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeFloatValueMatcher(selector, predicateFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeDoubleValueMatcher(selector, predicateFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return ValueMatchers.makeLongValueMatcher(selector, predicateFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
if (selector instanceof NilColumnValueSelector || !mayBeFilterable(selector.classOfObject())) {
|
||||
// Column does not exist, or is unfilterable. Treat it as all nulls.
|
||||
return BooleanValueMatcher.of(predicateFactory.makeStringPredicate().apply(null));
|
||||
} else {
|
||||
// Column exists but the type of value is unknown (we might have got here because "defaultType" is COMPLEX).
|
||||
// Return a ValueMatcher that inspects the object and does type-based comparison.
|
||||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
private Predicate<String> stringPredicate;
|
||||
private DruidLongPredicate longPredicate;
|
||||
private DruidFloatPredicate floatPredicate;
|
||||
private DruidDoublePredicate doublePredicate;
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
final Object rowValue = selector.getObject();
|
||||
|
||||
if (rowValue == null) {
|
||||
return getStringPredicate().apply(null);
|
||||
} else if (rowValue instanceof Integer) {
|
||||
return getLongPredicate().applyLong((int) rowValue);
|
||||
} else if (rowValue instanceof Long) {
|
||||
return getLongPredicate().applyLong((long) rowValue);
|
||||
} else if (rowValue instanceof Float) {
|
||||
return getFloatPredicate().applyFloat((float) rowValue);
|
||||
} else if (rowValue instanceof Number) {
|
||||
// Double or some other non-int, non-long, non-float number.
|
||||
return getDoublePredicate().applyDouble((double) rowValue);
|
||||
} else if (rowValue instanceof String || rowValue instanceof List) {
|
||||
// String or list-of-something. Cast to list of strings and evaluate them as strings.
|
||||
final List<String> rowValueStrings = Rows.objectToStrings(rowValue);
|
||||
|
||||
if (rowValueStrings.isEmpty()) {
|
||||
return getStringPredicate().apply(null);
|
||||
}
|
||||
|
||||
for (String rowValueString : rowValueStrings) {
|
||||
if (getStringPredicate().apply(NullHandling.emptyToNullIfNeeded(rowValueString))) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
} else {
|
||||
// Unfilterable type. Treat as null.
|
||||
return getStringPredicate().apply(null);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("value", predicateFactory);
|
||||
}
|
||||
|
||||
private Predicate<String> getStringPredicate()
|
||||
{
|
||||
if (stringPredicate == null) {
|
||||
stringPredicate = predicateFactory.makeStringPredicate();
|
||||
}
|
||||
|
||||
return stringPredicate;
|
||||
}
|
||||
|
||||
private DruidLongPredicate getLongPredicate()
|
||||
{
|
||||
if (longPredicate == null) {
|
||||
longPredicate = predicateFactory.makeLongPredicate();
|
||||
}
|
||||
|
||||
return longPredicate;
|
||||
}
|
||||
|
||||
private DruidFloatPredicate getFloatPredicate()
|
||||
{
|
||||
if (floatPredicate == null) {
|
||||
floatPredicate = predicateFactory.makeFloatPredicate();
|
||||
}
|
||||
|
||||
return floatPredicate;
|
||||
}
|
||||
|
||||
private DruidDoublePredicate getDoublePredicate()
|
||||
{
|
||||
if (doublePredicate == null) {
|
||||
doublePredicate = predicateFactory.makeDoublePredicate();
|
||||
}
|
||||
|
||||
return doublePredicate;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether a {@link BaseObjectColumnValueSelector} with object class {@code clazz} might be filterable, i.e.,
|
||||
* whether it might return numbers or strings.
|
||||
*
|
||||
* @param clazz class of object
|
||||
*/
|
||||
private static <T> boolean mayBeFilterable(final Class<T> clazz)
|
||||
{
|
||||
if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz)) {
|
||||
// clazz is a Number or String.
|
||||
return true;
|
||||
} else if (clazz.isAssignableFrom(Number.class) || clazz.isAssignableFrom(String.class)) {
|
||||
// clazz is a superclass of Number or String.
|
||||
return true;
|
||||
} else {
|
||||
// Instances of clazz cannot possibly be Numbers or Strings.
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,365 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidDoublePredicate;
|
||||
import org.apache.druid.query.filter.DruidFloatPredicate;
|
||||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseNullableColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionDictionarySelector;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link ConstantValueMatcherFactory}
|
||||
* and {@link PredicateValueMatcherFactory}.
|
||||
*/
|
||||
public class ValueMatchers
|
||||
{
|
||||
private ValueMatchers()
|
||||
{
|
||||
// No instantiation.
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a string-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
* @param hasMultipleValues whether the column selector *might* have multiple values
|
||||
*/
|
||||
public static ValueMatcher makeStringValueMatcher(
|
||||
final DimensionSelector selector,
|
||||
final String value,
|
||||
final boolean hasMultipleValues
|
||||
)
|
||||
{
|
||||
final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(
|
||||
selector,
|
||||
hasMultipleValues,
|
||||
s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
|
||||
);
|
||||
|
||||
if (booleanMatcher != null) {
|
||||
return booleanMatcher;
|
||||
} else {
|
||||
return selector.makeValueMatcher(value);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a predicate-based {@link ValueMatcher} for a string-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param predicateFactory predicate to match
|
||||
* @param hasMultipleValues whether the column selector *might* have multiple values
|
||||
*/
|
||||
public static ValueMatcher makeStringValueMatcher(
|
||||
final DimensionSelector selector,
|
||||
final DruidPredicateFactory predicateFactory,
|
||||
final boolean hasMultipleValues
|
||||
)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate);
|
||||
|
||||
if (booleanMatcher != null) {
|
||||
return booleanMatcher;
|
||||
} else {
|
||||
return selector.makeValueMatcher(predicate);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a float-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
*/
|
||||
public static ValueMatcher makeFloatValueMatcher(
|
||||
final BaseFloatColumnValueSelector selector,
|
||||
final String value
|
||||
)
|
||||
{
|
||||
final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
|
||||
if (matchVal == null) {
|
||||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
// Use "floatToIntBits" to canonicalize NaN values.
|
||||
final int matchValIntBits = Float.floatToIntBits(matchVal);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return Float.floatToIntBits(selector.getFloat()) == matchValIntBits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value)
|
||||
{
|
||||
final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
if (matchVal == null) {
|
||||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
final long matchValLong = matchVal;
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return selector.getLong() == matchValLong;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static ValueMatcher makeLongValueMatcher(
|
||||
final BaseLongColumnValueSelector selector,
|
||||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyLong(selector.getLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a predicate-based {@link ValueMatcher} for a float-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param predicateFactory predicate to match
|
||||
*/
|
||||
public static ValueMatcher makeFloatValueMatcher(
|
||||
final BaseFloatColumnValueSelector selector,
|
||||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyFloat(selector.getFloat());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a double-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
*/
|
||||
public static ValueMatcher makeDoubleValueMatcher(
|
||||
final BaseDoubleColumnValueSelector selector,
|
||||
final String value
|
||||
)
|
||||
{
|
||||
final Double matchVal = DimensionHandlerUtils.convertObjectToDouble(value);
|
||||
if (matchVal == null) {
|
||||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
// Use "doubleToLongBits" to canonicalize NaN values.
|
||||
final long matchValLongBits = Double.doubleToLongBits(matchVal);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return Double.doubleToLongBits(selector.getDouble()) == matchValLongBits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a predicate-based {@link ValueMatcher} for a double-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param predicateFactory predicate to match
|
||||
*/
|
||||
public static ValueMatcher makeDoubleValueMatcher(
|
||||
final BaseDoubleColumnValueSelector selector,
|
||||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidDoublePredicate predicate = predicateFactory.makeDoublePredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyDouble(selector.getDouble());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* If applying {@code predicate} to {@code selector} would always return a constant, returns that constant.
|
||||
* Otherwise, returns null.
|
||||
*
|
||||
* This method would have been private, except it's also used by
|
||||
* {@link org.apache.druid.query.filter.vector.SingleValueStringVectorValueMatcher}.
|
||||
*
|
||||
* @param selector string selector
|
||||
* @param hasMultipleValues whether the selector *might* have multiple values
|
||||
* @param predicate predicate to apply
|
||||
*/
|
||||
@Nullable
|
||||
public static Boolean toBooleanIfPossible(
|
||||
final DimensionDictionarySelector selector,
|
||||
final boolean hasMultipleValues,
|
||||
final Predicate<String> predicate
|
||||
)
|
||||
{
|
||||
if (selector.getValueCardinality() == 0) {
|
||||
// Column has no values (it doesn't exist, or it's all empty arrays).
|
||||
// Match if and only if "predicate" matches null.
|
||||
return predicate.apply(null);
|
||||
} else if (!hasMultipleValues && selector.getValueCardinality() == 1 && selector.nameLookupPossibleInAdvance()) {
|
||||
// Every row has the same value. Match if and only if "predicate" matches the possible value.
|
||||
return predicate.apply(selector.lookupName(0));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If {@link #toBooleanIfPossible} would return nonnull, this returns a {@link BooleanValueMatcher} that always
|
||||
* returns that value. Otherwise, this returns null.
|
||||
*
|
||||
* @param selector string selector
|
||||
* @param hasMultipleValues whether the selector *might* have multiple values
|
||||
* @param predicate predicate to apply
|
||||
*/
|
||||
@Nullable
|
||||
private static ValueMatcher toBooleanMatcherIfPossible(
|
||||
final DimensionSelector selector,
|
||||
final boolean hasMultipleValues,
|
||||
final Predicate<String> predicate
|
||||
)
|
||||
{
|
||||
final Boolean booleanValue = ValueMatchers.toBooleanIfPossible(
|
||||
selector,
|
||||
hasMultipleValues,
|
||||
predicate
|
||||
);
|
||||
return booleanValue == null ? null : BooleanValueMatcher.of(booleanValue);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a ValueMatcher that matches when the primitive numeric (long, double, or float) value from
|
||||
* {@code selector} should be treated as null.
|
||||
*/
|
||||
private static ValueMatcher makeNumericNullValueMatcher(BaseNullableColumnValueSelector selector)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return selector.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -62,6 +62,7 @@ import org.apache.druid.segment.LongColumnSelector;
|
|||
import org.apache.druid.segment.Metadata;
|
||||
import org.apache.druid.segment.NilColumnValueSelector;
|
||||
import org.apache.druid.segment.ObjectColumnSelector;
|
||||
import org.apache.druid.segment.RowAdapters;
|
||||
import org.apache.druid.segment.RowBasedColumnSelectorFactory;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
|
@ -130,7 +131,12 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
final boolean deserializeComplexMetrics
|
||||
)
|
||||
{
|
||||
final RowBasedColumnSelectorFactory baseSelectorFactory = RowBasedColumnSelectorFactory.create(in::get, null);
|
||||
final RowBasedColumnSelectorFactory<InputRow> baseSelectorFactory = RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
in::get,
|
||||
null,
|
||||
true
|
||||
);
|
||||
|
||||
class IncrementalIndexInputRowColumnSelectorFactory implements ColumnSelectorFactory
|
||||
{
|
||||
|
|
|
@ -64,7 +64,7 @@ public class LookupJoinMatcher implements JoinMatcher
|
|||
}
|
||||
|
||||
@Override
|
||||
public Supplier<String> makeDimensionProcessor(DimensionSelector selector)
|
||||
public Supplier<String> makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
return () -> {
|
||||
final IndexedInts row = selector.getRow();
|
||||
|
|
|
@ -298,7 +298,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher
|
|||
}
|
||||
|
||||
@Override
|
||||
public Supplier<IntIterator> makeDimensionProcessor(DimensionSelector selector)
|
||||
public Supplier<IntIterator> makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
// NOTE: The slow (cardinality unknown) and fast (cardinality known) code paths below only differ in the calls to
|
||||
// getRowNumbers() and getAndCacheRowNumbers(), respectively. The majority of the code path is duplicated to avoid
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.data.input.Row;
|
|||
import org.apache.druid.data.input.Rows;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.segment.RowAdapters;
|
||||
import org.apache.druid.segment.RowBasedColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -55,8 +56,10 @@ public class Transformer
|
|||
valueMatcher = transformSpec.getFilter().toFilter()
|
||||
.makeMatcher(
|
||||
RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
rowSupplierForValueMatcher::get,
|
||||
null
|
||||
null,
|
||||
false
|
||||
)
|
||||
);
|
||||
} else {
|
||||
|
@ -152,7 +155,7 @@ public class Transformer
|
|||
{
|
||||
final RowFunction transform = transforms.get(ColumnHolder.TIME_COLUMN_NAME);
|
||||
if (transform != null) {
|
||||
return Rows.objectToNumber(ColumnHolder.TIME_COLUMN_NAME, transform.eval(row)).longValue();
|
||||
return Rows.objectToNumber(ColumnHolder.TIME_COLUMN_NAME, transform.eval(row), true).longValue();
|
||||
} else {
|
||||
return row.getTimestampFromEpoch();
|
||||
}
|
||||
|
@ -196,7 +199,7 @@ public class Transformer
|
|||
{
|
||||
final RowFunction transform = transforms.get(metric);
|
||||
if (transform != null) {
|
||||
return Rows.objectToNumber(metric, transform.eval(row));
|
||||
return Rows.objectToNumber(metric, transform.eval(row), true);
|
||||
} else {
|
||||
return row.getMetric(metric);
|
||||
}
|
||||
|
|
|
@ -95,7 +95,7 @@ public class RowBasedExpressionColumnValueSelector extends ExpressionColumnValue
|
|||
{
|
||||
Object binding = bindings.get(x);
|
||||
if (binding != null) {
|
||||
if (binding instanceof String[] && ((String[]) binding).length > 1) {
|
||||
if (binding instanceof String[]) {
|
||||
return true;
|
||||
} else if (binding instanceof Number) {
|
||||
ignoredColumns.add(x);
|
||||
|
|
|
@ -5357,7 +5357,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final String field = "idx_subpostagg";
|
||||
final int p = query.getResultRowPositionLookup().getInt(field);
|
||||
return (Rows.objectToNumber(field, row.get(p)).floatValue() < 3800);
|
||||
return (Rows.objectToNumber(field, row.get(p), true).floatValue() < 3800);
|
||||
}
|
||||
}
|
||||
)
|
||||
|
@ -5651,7 +5651,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final String field = "idx_subpostagg";
|
||||
final int p = query.getResultRowPositionLookup().getInt(field);
|
||||
return (Rows.objectToNumber(field, row.get(p)).floatValue() < 3800);
|
||||
return (Rows.objectToNumber(field, row.get(p), true).floatValue() < 3800);
|
||||
}
|
||||
}
|
||||
)
|
||||
|
|
|
@ -65,6 +65,7 @@ import org.apache.druid.segment.IndexBuilder;
|
|||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||
import org.apache.druid.segment.RowAdapters;
|
||||
import org.apache.druid.segment.RowBasedColumnSelectorFactory;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
|
@ -154,7 +155,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
@Nullable String timeDim,
|
||||
@Nullable Double d0,
|
||||
@Nullable Float f0,
|
||||
@Nullable Long l0)
|
||||
@Nullable Long l0
|
||||
)
|
||||
{
|
||||
// for row selector to work correctly as part of the test matrix, default value coercion needs to happen to columns
|
||||
Map<String, Object> mapRow = Maps.newHashMapWithExpectedSize(6);
|
||||
|
@ -638,7 +640,14 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
// Perform test
|
||||
final SettableSupplier<InputRow> rowSupplier = new SettableSupplier<>();
|
||||
final ValueMatcher matcher = makeFilter(filter).makeMatcher(
|
||||
VIRTUAL_COLUMNS.wrap(RowBasedColumnSelectorFactory.create(rowSupplier::get, rowSignature))
|
||||
VIRTUAL_COLUMNS.wrap(
|
||||
RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
rowSupplier::get,
|
||||
rowSignature,
|
||||
false
|
||||
)
|
||||
)
|
||||
);
|
||||
final List<String> values = new ArrayList<>();
|
||||
for (InputRow row : rows) {
|
||||
|
|
|
@ -84,7 +84,10 @@ public class ExpressionFilterTest extends BaseFilterTest
|
|||
ImmutableMap.of("dim0", "6", "dim1", 6L, "dim2", 6.0f, "dim3", "1"),
|
||||
ImmutableMap.of("dim0", "7", "dim1", 7L, "dim2", 7.0f, "dim3", "a"),
|
||||
ImmutableMap.of("dim0", "8", "dim1", 8L, "dim2", 8.0f, "dim3", 8L),
|
||||
ImmutableMap.of("dim0", "9", "dim1", 9L, "dim2", 9.0f, "dim3", 1.234f, "dim4", 1.234f)
|
||||
|
||||
// Note: the "dim3 == 1.234" check in "testOneSingleValuedStringColumn" fails if dim3 is 1.234f instead of 1.234d,
|
||||
// because the literal 1.234 is interpreted as a double, and 1.234f cast to double is not equivalent to 1.234d.
|
||||
ImmutableMap.of("dim0", "9", "dim1", 9L, "dim2", 9.0f, "dim3", 1.234d, "dim4", 1.234d)
|
||||
).stream().map(e -> PARSER.parseBatch(e).get(0)).collect(Collectors.toList());
|
||||
|
||||
public ExpressionFilterTest(
|
||||
|
|
|
@ -107,11 +107,10 @@ public class SelectorFilterTest extends BaseFilterTest
|
|||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", null, null), ImmutableList.of("0"));
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", "", null), ImmutableList.of("0"));
|
||||
} else {
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", null, null), ImmutableList.of());
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", "", null), ImmutableList.of("0"));
|
||||
}
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", "", null), ImmutableList.of("0"));
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", "10", null), ImmutableList.of("1"));
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", "2", null), ImmutableList.of("2"));
|
||||
assertFilterMatches(new SelectorDimFilter("dim1", "1", null), ImmutableList.of("3"));
|
||||
|
|
|
@ -110,7 +110,7 @@ public class JoinTestHelper
|
|||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Object> makeDimensionProcessor(DimensionSelector selector)
|
||||
public Supplier<Object> makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
return selector::defaultGetObject;
|
||||
}
|
||||
|
|
|
@ -84,7 +84,7 @@ public class IndexedTableJoinMatcherTest
|
|||
ValueType.STRING,
|
||||
IndexedTableJoinMatcherTest::createSingletonIntList
|
||||
);
|
||||
return conditionMatcherFactory.makeDimensionProcessor(new TestDimensionSelector(KEY, valueCardinality));
|
||||
return conditionMatcherFactory.makeDimensionProcessor(new TestDimensionSelector(KEY, valueCardinality), false);
|
||||
}
|
||||
|
||||
private static class TestDimensionSelector extends ConstantDimensionSelector
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.druid.segment.ColumnSelectorFactory;
|
|||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.RowAdapters;
|
||||
import org.apache.druid.segment.RowBasedColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
|
@ -198,8 +199,10 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
|
||||
private static final ThreadLocal<Row> CURRENT_ROW = new ThreadLocal<>();
|
||||
private static final ColumnSelectorFactory COLUMN_SELECTOR_FACTORY = RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
CURRENT_ROW::get,
|
||||
null
|
||||
null,
|
||||
false
|
||||
);
|
||||
|
||||
@Test
|
||||
|
@ -230,21 +233,33 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
{
|
||||
DimensionSpec spec = new DefaultDimensionSpec("expr", "expr");
|
||||
|
||||
final BaseObjectColumnValueSelector selectorImplicit = SCALE_LIST_IMPLICIT.makeDimensionSelector(spec, COLUMN_SELECTOR_FACTORY);
|
||||
final BaseObjectColumnValueSelector selectorImplicit = SCALE_LIST_IMPLICIT.makeDimensionSelector(
|
||||
spec,
|
||||
COLUMN_SELECTOR_FACTORY
|
||||
);
|
||||
CURRENT_ROW.set(ROWMULTI);
|
||||
Assert.assertEquals(ImmutableList.of("2.0", "4.0", "6.0"), selectorImplicit.getObject());
|
||||
CURRENT_ROW.set(ROWMULTI2);
|
||||
Assert.assertEquals(ImmutableList.of("6.0", "8.0", "10.0"), selectorImplicit.getObject());
|
||||
CURRENT_ROW.set(ROWMULTI3);
|
||||
Assert.assertEquals(Arrays.asList("6.0", NullHandling.replaceWithDefault() ? "0.0" : null, "10.0"), selectorImplicit.getObject());
|
||||
Assert.assertEquals(
|
||||
Arrays.asList("6.0", NullHandling.replaceWithDefault() ? "0.0" : null, "10.0"),
|
||||
selectorImplicit.getObject()
|
||||
);
|
||||
|
||||
final BaseObjectColumnValueSelector selectorExplicit = SCALE_LIST_EXPLICIT.makeDimensionSelector(spec, COLUMN_SELECTOR_FACTORY);
|
||||
final BaseObjectColumnValueSelector selectorExplicit = SCALE_LIST_EXPLICIT.makeDimensionSelector(
|
||||
spec,
|
||||
COLUMN_SELECTOR_FACTORY
|
||||
);
|
||||
CURRENT_ROW.set(ROWMULTI);
|
||||
Assert.assertEquals(ImmutableList.of("2.0", "4.0", "6.0"), selectorExplicit.getObject());
|
||||
CURRENT_ROW.set(ROWMULTI2);
|
||||
Assert.assertEquals(ImmutableList.of("6.0", "8.0", "10.0"), selectorExplicit.getObject());
|
||||
CURRENT_ROW.set(ROWMULTI3);
|
||||
Assert.assertEquals(Arrays.asList("6.0", NullHandling.replaceWithDefault() ? "0.0" : null, "10.0"), selectorExplicit.getObject());
|
||||
Assert.assertEquals(
|
||||
Arrays.asList("6.0", NullHandling.replaceWithDefault() ? "0.0" : null, "10.0"),
|
||||
selectorExplicit.getObject()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -725,8 +740,10 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final ColumnValueSelector<ExprEval> selector = ExpressionSelectors.makeExprEvalSelector(
|
||||
RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
CURRENT_ROW::get,
|
||||
ImmutableMap.of("x", ValueType.LONG)
|
||||
ImmutableMap.of("x", ValueType.LONG),
|
||||
false
|
||||
),
|
||||
Parser.parse(SCALE_LONG.getExpression(), TestExprMacroTable.INSTANCE)
|
||||
);
|
||||
|
@ -746,8 +763,10 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final ColumnValueSelector<ExprEval> selector = ExpressionSelectors.makeExprEvalSelector(
|
||||
RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
CURRENT_ROW::get,
|
||||
ImmutableMap.of("x", ValueType.DOUBLE)
|
||||
ImmutableMap.of("x", ValueType.DOUBLE),
|
||||
false
|
||||
),
|
||||
Parser.parse(SCALE_FLOAT.getExpression(), TestExprMacroTable.INSTANCE)
|
||||
);
|
||||
|
@ -767,8 +786,10 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final ColumnValueSelector<ExprEval> selector = ExpressionSelectors.makeExprEvalSelector(
|
||||
RowBasedColumnSelectorFactory.create(
|
||||
RowAdapters.standardRow(),
|
||||
CURRENT_ROW::get,
|
||||
ImmutableMap.of("x", ValueType.FLOAT)
|
||||
ImmutableMap.of("x", ValueType.FLOAT),
|
||||
false
|
||||
),
|
||||
Parser.parse(SCALE_FLOAT.getExpression(), TestExprMacroTable.INSTANCE)
|
||||
);
|
||||
|
|
Loading…
Reference in New Issue