mirror of https://github.com/apache/druid.git
overhaul DruidPredicateFactory to better handle 3VL (#15629)
* overhaul DruidPredicateFactory to better handle 3VL fixes some bugs caused by some limitations of the original design of how DruidPredicateFactory interacts with 3-value logic. The primary impacted area was with how filters on values transformed with expressions or extractionFn which turn non-null values into nulls, which were not possible to be modelled with the 'isNullInputUnknown' method changes: * adds DruidObjectPredicate to specialize string, array, and object based predicates instead of using guava Predicate * DruidPredicateFactory now uses DruidObjectPredicate * introduces DruidPredicateMatch enum, which all predicates returned from DruidPredicateFactory now use instead of booleans to indicate match. This means DruidLongPredicate, DruidFloatPredicate, DruidDoublePredicate, and the newly added DruidObjectPredicate apply methods all now return DruidPredicateMatch. This allows matchers and indexes * isNullInputUnknown has been removed from DruidPredicateFactory * rename, fix test * adjust * style * npe * more test * fix default value mode to not match new test
This commit is contained in:
parent
62964e99b1
commit
c221a2634b
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.benchmark;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.FluentIterable;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
|
@ -32,7 +31,9 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
|
@ -73,37 +74,32 @@ public class DimensionPredicateFilterBenchmark
|
|||
new DruidPredicateFactory()
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
if (input == null) {
|
||||
return false;
|
||||
}
|
||||
return Integer.parseInt(input) % 2 == 0;
|
||||
return (DruidObjectPredicate<String>) input -> {
|
||||
if (input == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(Integer.parseInt(input) % 2 == 0);
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
return DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
},
|
||||
null
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.benchmark;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
|
@ -41,6 +40,7 @@ import org.apache.druid.query.filter.DimFilter;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
@ -93,7 +93,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
|
@ -543,34 +542,27 @@ public class FilterPartitionBenchmark
|
|||
final DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
return Objects.equals(valueOrNull, input);
|
||||
}
|
||||
};
|
||||
return valueOrNull == null ? DruidObjectPredicate.isNull() : DruidObjectPredicate.equalTo(valueOrNull);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
return DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -88,15 +88,14 @@ final class StringTypeMapVirtualColumnDimensionSelector extends MapVirtualColumn
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final String rowValue = (String) getObject();
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
return (matchNull && rowValue == null) || predicate.apply(rowValue);
|
||||
return predicate.apply(rowValue).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonInclude;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.hash.HashCode;
|
||||
|
@ -91,16 +90,14 @@ public class BloomDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
dimension,
|
||||
new DruidPredicateFactory()
|
||||
{
|
||||
private final boolean isNullUnknown = !bloomKFilter.testBytes(null, 0, 0);
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return str -> {
|
||||
if (str == null) {
|
||||
return bloomKFilter.testBytes(null, 0, 0);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testBytes(null, 0, 0));
|
||||
}
|
||||
return bloomKFilter.testString(str);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testString(str));
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -110,15 +107,15 @@ public class BloomDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
return new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long input)
|
||||
public DruidPredicateMatch applyLong(long input)
|
||||
{
|
||||
return bloomKFilter.testLong(input);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testLong(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return bloomKFilter.testBytes(null, 0, 0);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testBytes(null, 0, 0));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -129,15 +126,15 @@ public class BloomDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
return new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyFloat(float input)
|
||||
public DruidPredicateMatch applyFloat(float input)
|
||||
{
|
||||
return bloomKFilter.testFloat(input);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testFloat(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return bloomKFilter.testBytes(null, 0, 0);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testBytes(null, 0, 0));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -148,24 +145,18 @@ public class BloomDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
return new DruidDoublePredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyDouble(double input)
|
||||
public DruidPredicateMatch applyDouble(double input)
|
||||
{
|
||||
return bloomKFilter.testDouble(input);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testDouble(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return bloomKFilter.testBytes(null, 0, 0);
|
||||
return DruidPredicateMatch.of(bloomKFilter.testBytes(null, 0, 0));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullUnknown;
|
||||
}
|
||||
},
|
||||
extractionFn,
|
||||
filterTuning
|
||||
|
|
|
@ -209,6 +209,9 @@ public interface Expr extends Cacheable
|
|||
}
|
||||
final ColumnCapabilities capabilities = holder.getCapabilities();
|
||||
final ColumnIndexSupplier delegateIndexSupplier = holder.getIndexSupplier();
|
||||
if (delegateIndexSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
final DictionaryEncodedValueIndex<?> delegateRawIndex = delegateIndexSupplier.as(
|
||||
DictionaryEncodedValueIndex.class
|
||||
);
|
||||
|
|
|
@ -19,13 +19,11 @@
|
|||
|
||||
package org.apache.druid.math.expr;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -35,8 +33,8 @@ import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex;
|
|||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
public class ExpressionPredicateIndexSupplier implements ColumnIndexSupplier
|
||||
{
|
||||
|
@ -77,211 +75,190 @@ public class ExpressionPredicateIndexSupplier implements ColumnIndexSupplier
|
|||
@Override
|
||||
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
|
||||
{
|
||||
final Supplier<ValueAndUnknownIndexes> bitmapsSupplier;
|
||||
final java.util.function.Function<Object, ExprEval<?>> evalFunction =
|
||||
inputValue -> expr.eval(InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue));
|
||||
|
||||
switch (outputType.getType()) {
|
||||
case STRING:
|
||||
bitmapsSupplier = Suppliers.memoize(() -> computeStringBitmaps(matcherFactory, evalFunction));
|
||||
break;
|
||||
case LONG:
|
||||
bitmapsSupplier = Suppliers.memoize(() -> computeLongBitmaps(matcherFactory, evalFunction));
|
||||
break;
|
||||
case DOUBLE:
|
||||
bitmapsSupplier = Suppliers.memoize(() -> computeDoubleBitmaps(matcherFactory, evalFunction));
|
||||
break;
|
||||
case FLOAT:
|
||||
bitmapsSupplier = Suppliers.memoize(() -> computeFloatBitmaps(matcherFactory, evalFunction));
|
||||
break;
|
||||
case ARRAY:
|
||||
bitmapsSupplier = Suppliers.memoize(() -> computeArrayBitmaps(matcherFactory, evalFunction));
|
||||
break;
|
||||
default:
|
||||
bitmapsSupplier = Suppliers.memoize(() -> computeComplexBitmaps(matcherFactory, evalFunction));
|
||||
break;
|
||||
}
|
||||
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return bitmapsSupplier.get().getMatches();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
return inputColumnIndexes.getBitmapFactory().union(bitmapsSupplier.get().getUnknowns());
|
||||
switch (outputType.getType()) {
|
||||
case STRING:
|
||||
return computeStringBitmaps(matcherFactory.makeStringPredicate(), evalFunction, includeUnknown);
|
||||
case LONG:
|
||||
return computeLongBitmaps(matcherFactory.makeLongPredicate(), evalFunction, includeUnknown);
|
||||
case DOUBLE:
|
||||
return computeDoubleBitmaps(matcherFactory.makeDoublePredicate(), evalFunction, includeUnknown);
|
||||
case FLOAT:
|
||||
return computeFloatBitmaps(matcherFactory.makeFloatPredicate(), evalFunction, includeUnknown);
|
||||
case ARRAY:
|
||||
return computeArrayBitmaps(matcherFactory.makeArrayPredicate(outputType), evalFunction, includeUnknown);
|
||||
default:
|
||||
return computeObjectBitmaps(matcherFactory.makeObjectPredicate(), evalFunction, includeUnknown);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
private ValueAndUnknownIndexes computeStringBitmaps(
|
||||
DruidPredicateFactory matcherFactory,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction
|
||||
private Iterable<ImmutableBitmap> computeStringBitmaps(
|
||||
DruidObjectPredicate<String> predicate,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
final Predicate<String> predicate = matcherFactory.makeStringPredicate();
|
||||
final List<ImmutableBitmap> matches = new ArrayList<>();
|
||||
final List<ImmutableBitmap> unknowns = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < inputColumnIndexes.getCardinality(); i++) {
|
||||
final Object inputValue = inputColumnIndexes.getValue(i);
|
||||
final String result = evalFunction.apply(inputValue).asString();
|
||||
if (result == null && matcherFactory.isNullInputUnknown()) {
|
||||
unknowns.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (predicate.apply(result)) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
}
|
||||
}
|
||||
|
||||
return new ValueAndUnknownIndexes(matches, unknowns);
|
||||
}
|
||||
|
||||
private ValueAndUnknownIndexes computeLongBitmaps(
|
||||
DruidPredicateFactory matcherFactory,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction
|
||||
)
|
||||
{
|
||||
final DruidLongPredicate predicate = matcherFactory.makeLongPredicate();
|
||||
final List<ImmutableBitmap> matches = new ArrayList<>();
|
||||
final List<ImmutableBitmap> unknowns = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < inputColumnIndexes.getCardinality(); i++) {
|
||||
final Object inputValue = inputColumnIndexes.getValue(i);
|
||||
final ExprEval<?> result = evalFunction.apply(inputValue);
|
||||
if (result.isNumericNull() && matcherFactory.isNullInputUnknown()) {
|
||||
unknowns.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (result.isNumericNull() && predicate.applyNull()) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (!result.isNumericNull() && predicate.applyLong(result.asLong())) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
}
|
||||
}
|
||||
|
||||
return new ValueAndUnknownIndexes(matches, unknowns);
|
||||
}
|
||||
|
||||
private ValueAndUnknownIndexes computeDoubleBitmaps(
|
||||
DruidPredicateFactory matcherFactory,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction
|
||||
)
|
||||
{
|
||||
final DruidDoublePredicate predicate = matcherFactory.makeDoublePredicate();
|
||||
final List<ImmutableBitmap> matches = new ArrayList<>();
|
||||
final List<ImmutableBitmap> unknowns = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < inputColumnIndexes.getCardinality(); i++) {
|
||||
final Object inputValue = inputColumnIndexes.getValue(i);
|
||||
final ExprEval<?> result = evalFunction.apply(inputValue);
|
||||
if (result.isNumericNull() && matcherFactory.isNullInputUnknown()) {
|
||||
unknowns.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (result.isNumericNull() && predicate.applyNull()) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (!result.isNumericNull() && predicate.applyDouble(result.asDouble())) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
}
|
||||
}
|
||||
|
||||
return new ValueAndUnknownIndexes(matches, unknowns);
|
||||
}
|
||||
|
||||
private ValueAndUnknownIndexes computeFloatBitmaps(
|
||||
DruidPredicateFactory matcherFactory,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction
|
||||
)
|
||||
{
|
||||
final DruidFloatPredicate predicate = matcherFactory.makeFloatPredicate();
|
||||
final List<ImmutableBitmap> matches = new ArrayList<>();
|
||||
final List<ImmutableBitmap> unknowns = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < inputColumnIndexes.getCardinality(); i++) {
|
||||
final Object inputValue = inputColumnIndexes.getValue(i);
|
||||
final ExprEval<?> result = evalFunction.apply(inputValue);
|
||||
if (result.isNumericNull() && matcherFactory.isNullInputUnknown()) {
|
||||
unknowns.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (result.isNumericNull() && predicate.applyNull()) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (!result.isNumericNull() && predicate.applyFloat((float) result.asDouble())) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
}
|
||||
}
|
||||
|
||||
return new ValueAndUnknownIndexes(matches, unknowns);
|
||||
}
|
||||
|
||||
private ValueAndUnknownIndexes computeArrayBitmaps(
|
||||
DruidPredicateFactory matcherFactory,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction
|
||||
)
|
||||
{
|
||||
final Predicate<Object[]> predicate = matcherFactory.makeArrayPredicate(outputType);
|
||||
final List<ImmutableBitmap> matches = new ArrayList<>();
|
||||
final List<ImmutableBitmap> unknowns = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < inputColumnIndexes.getCardinality(); i++) {
|
||||
final Object inputValue = inputColumnIndexes.getValue(i);
|
||||
final Object[] result = evalFunction.apply(inputValue).asArray();
|
||||
if (result == null && matcherFactory.isNullInputUnknown()) {
|
||||
unknowns.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (predicate.apply(result)) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
}
|
||||
}
|
||||
|
||||
return new ValueAndUnknownIndexes(matches, unknowns);
|
||||
}
|
||||
|
||||
private ValueAndUnknownIndexes computeComplexBitmaps(
|
||||
DruidPredicateFactory matcherFactory,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction
|
||||
)
|
||||
{
|
||||
final Predicate<Object> predicate = matcherFactory.makeObjectPredicate();
|
||||
final List<ImmutableBitmap> matches = new ArrayList<>();
|
||||
final List<ImmutableBitmap> unknowns = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < inputColumnIndexes.getCardinality(); i++) {
|
||||
final Object inputValue = inputColumnIndexes.getValue(i);
|
||||
final Object result = evalFunction.apply(inputValue).valueOrDefault();
|
||||
if (result == null && matcherFactory.isNullInputUnknown()) {
|
||||
unknowns.add(inputColumnIndexes.getBitmap(i));
|
||||
} else if (predicate.apply(result)) {
|
||||
matches.add(inputColumnIndexes.getBitmap(i));
|
||||
}
|
||||
}
|
||||
|
||||
return new ValueAndUnknownIndexes(matches, unknowns);
|
||||
}
|
||||
|
||||
/**
|
||||
* Holder for two sets of {@link ImmutableBitmap}, the first set representing values that match the predicate after
|
||||
* computing the expression, and the second for values which computing the expression evaluates to null and are
|
||||
* considered 'unknown'
|
||||
*/
|
||||
private static class ValueAndUnknownIndexes
|
||||
{
|
||||
private final List<ImmutableBitmap> matches;
|
||||
private final List<ImmutableBitmap> unknowns;
|
||||
|
||||
private ValueAndUnknownIndexes(List<ImmutableBitmap> matches, List<ImmutableBitmap> unknowns)
|
||||
return () -> new BitmapIterator(inputColumnIndexes)
|
||||
{
|
||||
this.matches = matches;
|
||||
this.unknowns = unknowns;
|
||||
@Override
|
||||
boolean nextMatches(@Nullable Object nextValue)
|
||||
{
|
||||
return predicate.apply(evalFunction.apply(nextValue).asString())
|
||||
.matches(includeUnknown);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Iterable<ImmutableBitmap> computeLongBitmaps(
|
||||
DruidLongPredicate predicate,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
return () -> new BitmapIterator(inputColumnIndexes)
|
||||
{
|
||||
@Override
|
||||
boolean nextMatches(@Nullable Object nextValue)
|
||||
{
|
||||
final ExprEval<?> result = evalFunction.apply(nextValue);
|
||||
if (result.isNumericNull()) {
|
||||
return predicate.applyNull().matches(includeUnknown);
|
||||
}
|
||||
return predicate.applyLong(result.asLong()).matches(includeUnknown);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Iterable<ImmutableBitmap> computeDoubleBitmaps(
|
||||
DruidDoublePredicate predicate,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
return () -> new BitmapIterator(inputColumnIndexes)
|
||||
{
|
||||
@Override
|
||||
boolean nextMatches(@Nullable Object nextValue)
|
||||
{
|
||||
final ExprEval<?> result = evalFunction.apply(nextValue);
|
||||
if (result.isNumericNull()) {
|
||||
return predicate.applyNull().matches(includeUnknown);
|
||||
}
|
||||
return predicate.applyDouble(result.asDouble()).matches(includeUnknown);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Iterable<ImmutableBitmap> computeFloatBitmaps(
|
||||
DruidFloatPredicate predicate,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
return () -> new BitmapIterator(inputColumnIndexes)
|
||||
{
|
||||
@Override
|
||||
boolean nextMatches(@Nullable Object nextValue)
|
||||
{
|
||||
final ExprEval<?> result = evalFunction.apply(nextValue);
|
||||
if (result.isNumericNull()) {
|
||||
return predicate.applyNull().matches(includeUnknown);
|
||||
}
|
||||
return predicate.applyFloat((float) result.asDouble()).matches(includeUnknown);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Iterable<ImmutableBitmap> computeArrayBitmaps(
|
||||
DruidObjectPredicate<Object[]> predicate,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
return () -> new BitmapIterator(inputColumnIndexes)
|
||||
{
|
||||
@Override
|
||||
boolean nextMatches(@Nullable Object nextValue)
|
||||
{
|
||||
final Object[] result = evalFunction.apply(nextValue).asArray();
|
||||
return predicate.apply(result).matches(includeUnknown);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Iterable<ImmutableBitmap> computeObjectBitmaps(
|
||||
DruidObjectPredicate<Object> predicate,
|
||||
java.util.function.Function<Object, ExprEval<?>> evalFunction,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
return () -> new BitmapIterator(inputColumnIndexes)
|
||||
{
|
||||
@Override
|
||||
boolean nextMatches(@Nullable Object nextValue)
|
||||
{
|
||||
final Object result = evalFunction.apply(nextValue).valueOrDefault();
|
||||
return predicate.apply(result).matches(includeUnknown);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private abstract static class BitmapIterator implements Iterator<ImmutableBitmap>
|
||||
{
|
||||
private final DictionaryEncodedValueIndex<?> inputColumnIndexes;
|
||||
int next;
|
||||
int index = 0;
|
||||
boolean nextSet = false;
|
||||
|
||||
private BitmapIterator(DictionaryEncodedValueIndex<?> inputColumnIndexes)
|
||||
{
|
||||
this.inputColumnIndexes = inputColumnIndexes;
|
||||
}
|
||||
|
||||
public List<ImmutableBitmap> getMatches()
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return matches;
|
||||
if (!nextSet) {
|
||||
findNext();
|
||||
}
|
||||
return nextSet;
|
||||
}
|
||||
|
||||
public List<ImmutableBitmap> getUnknowns()
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap next()
|
||||
{
|
||||
return unknowns;
|
||||
if (!nextSet) {
|
||||
findNext();
|
||||
if (!nextSet) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
}
|
||||
nextSet = false;
|
||||
return inputColumnIndexes.getBitmap(next);
|
||||
}
|
||||
|
||||
private void findNext()
|
||||
{
|
||||
while (!nextSet && index < inputColumnIndexes.getCardinality()) {
|
||||
Object nextValue = inputColumnIndexes.getValue(index);
|
||||
nextSet = nextMatches(nextValue);
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
}
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
abstract boolean nextMatches(@Nullable Object nextValue);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,10 @@
|
|||
package org.apache.druid.query.dimension;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -129,9 +130,13 @@ final class ForwardingFilteredDimensionSelector extends AbstractDimensionSelecto
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final BitSet valueIds = DimensionSelectorUtils.makePredicateMatchingSet(this, predicate);
|
||||
final boolean predicateMatchesNull = predicate.apply(null);
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final Supplier<BitSet> valueIds = Suppliers.memoize(
|
||||
() -> DimensionSelectorUtils.makePredicateMatchingSet(this, predicate, false)
|
||||
);
|
||||
final Supplier<BitSet> valueIdsWithUnknown = Suppliers.memoize(
|
||||
() -> DimensionSelectorUtils.makePredicateMatchingSet(this, predicate, true)
|
||||
);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
|
@ -142,20 +147,20 @@ final class ForwardingFilteredDimensionSelector extends AbstractDimensionSelecto
|
|||
boolean nullRow = true;
|
||||
for (int i = 0; i < baseRowSize; ++i) {
|
||||
final int baseId = baseRow.get(i);
|
||||
|
||||
if (includeUnknown && NullHandling.isNullOrEquivalent(selector.lookupName(baseId))) {
|
||||
return true;
|
||||
}
|
||||
int forwardedValue = idMapping.getForwardedId(baseId);
|
||||
if (forwardedValue >= 0) {
|
||||
if (valueIds.get(forwardedValue)) {
|
||||
if (includeUnknown) {
|
||||
if (valueIdsWithUnknown.get().get(forwardedValue)) {
|
||||
return true;
|
||||
}
|
||||
} else if (valueIds.get().get(forwardedValue)) {
|
||||
return true;
|
||||
}
|
||||
nullRow = false;
|
||||
}
|
||||
}
|
||||
// null should match empty rows in multi-value columns
|
||||
return nullRow && (includeUnknown || predicateMatchesNull);
|
||||
return nullRow && predicate.apply(null).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.dimension;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -101,31 +102,26 @@ final class PredicateFilteredDimensionSelector extends AbstractDimensionSelector
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> matcherPredicate = predicateFactory.makeStringPredicate();
|
||||
final boolean predicateMatchesNull = matcherPredicate.apply(null);
|
||||
final DruidObjectPredicate<String> matcherPredicate = predicateFactory.makeStringPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
final IndexedInts baseRow = selector.getRow();
|
||||
final int baseRowSize = baseRow.size();
|
||||
boolean nullRow = true;
|
||||
for (int i = 0; i < baseRowSize; ++i) {
|
||||
String rowValue = lookupName(baseRow.get(i));
|
||||
if (matchNull && rowValue == null) {
|
||||
return true;
|
||||
}
|
||||
if (predicate.apply(rowValue)) {
|
||||
if (matcherPredicate.apply(rowValue)) {
|
||||
if (matcherPredicate.apply(rowValue).matches(includeUnknown)) {
|
||||
return true;
|
||||
}
|
||||
nullRow = false;
|
||||
}
|
||||
}
|
||||
// null should match empty rows in multi-value columns
|
||||
return nullRow && (includeUnknown || predicateMatchesNull);
|
||||
return nullRow && matcherPredicate.apply(null).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.math.expr.Expr;
|
|||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.LikeDimFilter;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
@ -80,7 +81,11 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
return ExprEval.ofLongBoolean(likeMatcher.matches(arg.eval(bindings).asString()));
|
||||
final DruidPredicateMatch match = likeMatcher.matches(arg.eval(bindings).asString());
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ExprEval.ofLong(null);
|
||||
}
|
||||
return ExprEval.ofLongBoolean(match.matches(false));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.druid.query.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -307,13 +305,13 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
{
|
||||
private final ExprEval<?> elementMatchValue;
|
||||
private final EqualityFilter.EqualityPredicateFactory equalityPredicateFactory;
|
||||
private final Supplier<Predicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidObjectPredicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidLongPredicate> longPredicateSupplier;
|
||||
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
|
||||
private final Supplier<DruidDoublePredicate> doublePredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, Predicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<Predicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
private final Supplier<Predicate<Object>> objectPredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, DruidObjectPredicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<DruidObjectPredicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
private final Supplier<DruidObjectPredicate<Object>> objectPredicateSupplier;
|
||||
|
||||
public ArrayContainsPredicateFactory(ExprEval<?> elementMatchValue)
|
||||
{
|
||||
|
@ -322,10 +320,10 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
// if element match value is an array, scalar matches can never be true
|
||||
final Object matchVal = elementMatchValue.valueOrDefault();
|
||||
if (matchVal == null || (elementMatchValue.isArray() && elementMatchValue.asArray().length > 1)) {
|
||||
this.stringPredicateSupplier = Predicates::alwaysFalse;
|
||||
this.longPredicateSupplier = () -> DruidLongPredicate.ALWAYS_FALSE;
|
||||
this.doublePredicateSupplier = () -> DruidDoublePredicate.ALWAYS_FALSE;
|
||||
this.floatPredicateSupplier = () -> DruidFloatPredicate.ALWAYS_FALSE;
|
||||
this.stringPredicateSupplier = DruidObjectPredicate::alwaysFalseWithNullUnknown;
|
||||
this.longPredicateSupplier = () -> DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
this.doublePredicateSupplier = () -> DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
this.floatPredicateSupplier = () -> DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
this.stringPredicateSupplier = equalityPredicateFactory::makeStringPredicate;
|
||||
this.longPredicateSupplier = equalityPredicateFactory::makeLongPredicate;
|
||||
|
@ -338,7 +336,7 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return stringPredicateSupplier.get();
|
||||
}
|
||||
|
@ -362,7 +360,7 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
public DruidObjectPredicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
if (arrayType == null) {
|
||||
// fall back to per row detection if input array type is unknown
|
||||
|
@ -373,19 +371,19 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
public DruidObjectPredicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return objectPredicateSupplier.get();
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object>> makeObjectPredicateSupplier()
|
||||
private Supplier<DruidObjectPredicate<Object>> makeObjectPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final ExprEval<?> inputEval = ExprEval.bestEffortOf(StructuredData.unwrap(input));
|
||||
final Predicate<Object[]> matcher = new FallbackPredicate<>(
|
||||
final DruidObjectPredicate<Object[]> matcher = new FallbackPredicate<>(
|
||||
computeArrayPredicate(ExpressionType.toColumnType(inputEval.asArrayType())),
|
||||
inputEval.asArrayType()
|
||||
);
|
||||
|
@ -393,24 +391,24 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
});
|
||||
}
|
||||
|
||||
private Predicate<Object[]> computeArrayPredicate(TypeSignature<ValueType> arrayType)
|
||||
private DruidObjectPredicate<Object[]> computeArrayPredicate(TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
return arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType));
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
private Supplier<DruidObjectPredicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
{
|
||||
return Suppliers.memoize(() -> input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
// just use object predicate logic
|
||||
final Predicate<Object> objectPredicate = objectPredicateSupplier.get();
|
||||
final DruidObjectPredicate<Object> objectPredicate = objectPredicateSupplier.get();
|
||||
return objectPredicate.apply(input);
|
||||
});
|
||||
}
|
||||
|
||||
private Predicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
|
||||
private DruidObjectPredicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
|
||||
|
||||
|
@ -421,18 +419,18 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
|
|||
(ExpressionType) expressionType.getElementType()
|
||||
);
|
||||
if (castForComparison == null) {
|
||||
return Predicates.alwaysFalse();
|
||||
return DruidObjectPredicate.alwaysFalseWithNullUnknown();
|
||||
}
|
||||
final Object matchVal = castForComparison.value();
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
boolean anyMatch = false;
|
||||
for (Object elem : input) {
|
||||
anyMatch = anyMatch || elementComparator.compare(elem, matchVal) == 0;
|
||||
}
|
||||
return anyMatch;
|
||||
return DruidPredicateMatch.of(anyMatch);
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -434,7 +434,7 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of(
|
||||
hasLowerLongBound,
|
||||
|
@ -491,7 +491,7 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
return DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
|
||||
final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of(
|
||||
|
@ -553,7 +553,7 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of(
|
||||
hasLowerBound,
|
||||
|
|
|
@ -28,29 +28,29 @@ package org.apache.druid.query.filter;
|
|||
@SuppressWarnings("unused")
|
||||
public interface DruidDoublePredicate
|
||||
{
|
||||
DruidDoublePredicate ALWAYS_FALSE = input -> false;
|
||||
DruidDoublePredicate ALWAYS_FALSE_WITH_NULL_UNKNOWN = input -> DruidPredicateMatch.FALSE;
|
||||
|
||||
DruidDoublePredicate ALWAYS_TRUE = input -> true;
|
||||
DruidDoublePredicate ALWAYS_TRUE = input -> DruidPredicateMatch.TRUE;
|
||||
|
||||
DruidDoublePredicate MATCH_NULL_ONLY = new DruidDoublePredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyDouble(double input)
|
||||
public DruidPredicateMatch applyDouble(double input)
|
||||
{
|
||||
return false;
|
||||
return DruidPredicateMatch.FALSE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return true;
|
||||
return DruidPredicateMatch.TRUE;
|
||||
}
|
||||
};
|
||||
|
||||
boolean applyDouble(double input);
|
||||
DruidPredicateMatch applyDouble(double input);
|
||||
|
||||
default boolean applyNull()
|
||||
default DruidPredicateMatch applyNull()
|
||||
{
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,30 +28,30 @@ package org.apache.druid.query.filter;
|
|||
@SuppressWarnings("unused")
|
||||
public interface DruidFloatPredicate
|
||||
{
|
||||
DruidFloatPredicate ALWAYS_FALSE = input -> false;
|
||||
DruidFloatPredicate ALWAYS_FALSE_WITH_NULL_UNKNOWN = input -> DruidPredicateMatch.FALSE;
|
||||
|
||||
DruidFloatPredicate ALWAYS_TRUE = input -> true;
|
||||
DruidFloatPredicate ALWAYS_TRUE = input -> DruidPredicateMatch.TRUE;
|
||||
|
||||
DruidFloatPredicate MATCH_NULL_ONLY = new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyFloat(float input)
|
||||
public DruidPredicateMatch applyFloat(float input)
|
||||
{
|
||||
return false;
|
||||
return DruidPredicateMatch.FALSE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return true;
|
||||
return DruidPredicateMatch.TRUE;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
boolean applyFloat(float input);
|
||||
DruidPredicateMatch applyFloat(float input);
|
||||
|
||||
default boolean applyNull()
|
||||
default DruidPredicateMatch applyNull()
|
||||
{
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,29 +25,29 @@ package org.apache.druid.query.filter;
|
|||
*/
|
||||
public interface DruidLongPredicate
|
||||
{
|
||||
DruidLongPredicate ALWAYS_FALSE = input -> false;
|
||||
DruidLongPredicate ALWAYS_FALSE_WITH_NULL_UNKNOWN = input -> DruidPredicateMatch.FALSE;
|
||||
|
||||
DruidLongPredicate ALWAYS_TRUE = input -> true;
|
||||
DruidLongPredicate ALWAYS_TRUE = input -> DruidPredicateMatch.TRUE;
|
||||
|
||||
DruidLongPredicate MATCH_NULL_ONLY = new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long input)
|
||||
public DruidPredicateMatch applyLong(long input)
|
||||
{
|
||||
return false;
|
||||
return DruidPredicateMatch.FALSE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return true;
|
||||
return DruidPredicateMatch.TRUE;
|
||||
}
|
||||
};
|
||||
|
||||
boolean applyLong(long input);
|
||||
DruidPredicateMatch applyLong(long input);
|
||||
|
||||
default boolean applyNull()
|
||||
default DruidPredicateMatch applyNull()
|
||||
{
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* 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 DruidObjectPredicate<T>
|
||||
{
|
||||
static <T> DruidObjectPredicate<T> alwaysFalseWithNullUnknown()
|
||||
{
|
||||
return value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.FALSE;
|
||||
}
|
||||
|
||||
static <T> DruidObjectPredicate<T> alwaysTrue()
|
||||
{
|
||||
return value -> DruidPredicateMatch.TRUE;
|
||||
}
|
||||
|
||||
static <T> DruidObjectPredicate<T> equalTo(T val)
|
||||
{
|
||||
return value -> {
|
||||
if (value == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(val.equals(value));
|
||||
};
|
||||
}
|
||||
|
||||
static <T> DruidObjectPredicate<T> notEqualTo(T val)
|
||||
{
|
||||
return value -> {
|
||||
if (value == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(!val.equals(value));
|
||||
};
|
||||
}
|
||||
|
||||
static <T> DruidObjectPredicate<T> isNull()
|
||||
{
|
||||
return value -> DruidPredicateMatch.of(value == null);
|
||||
}
|
||||
|
||||
static <T> DruidObjectPredicate<T> notNull()
|
||||
{
|
||||
return value -> DruidPredicateMatch.of(value != null);
|
||||
}
|
||||
|
||||
DruidPredicateMatch apply(@Nullable T value);
|
||||
}
|
|
@ -19,11 +19,8 @@
|
|||
|
||||
package org.apache.druid.query.filter;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.vector.ReadableVectorMatch;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
|
@ -32,7 +29,7 @@ import javax.annotation.Nullable;
|
|||
@SubclassesMustOverrideEqualsAndHashCode
|
||||
public interface DruidPredicateFactory
|
||||
{
|
||||
Predicate<String> makeStringPredicate();
|
||||
DruidObjectPredicate<String> makeStringPredicate();
|
||||
|
||||
DruidLongPredicate makeLongPredicate();
|
||||
|
||||
|
@ -40,7 +37,7 @@ public interface DruidPredicateFactory
|
|||
|
||||
DruidDoublePredicate makeDoublePredicate();
|
||||
|
||||
default Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> inputType)
|
||||
default DruidObjectPredicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> inputType)
|
||||
{
|
||||
throw new UOE("Predicate does not support ARRAY types");
|
||||
}
|
||||
|
@ -55,24 +52,9 @@ public interface DruidPredicateFactory
|
|||
*
|
||||
* @see org.apache.druid.segment.VectorColumnProcessorFactory#makeObjectProcessor
|
||||
*/
|
||||
default Predicate<Object> makeObjectPredicate()
|
||||
default DruidObjectPredicate<Object> makeObjectPredicate()
|
||||
{
|
||||
final Predicate<String> stringPredicate = makeStringPredicate();
|
||||
final DruidObjectPredicate<String> stringPredicate = makeStringPredicate();
|
||||
return o -> stringPredicate.apply(null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicator for if null inputs should be considered 'unknown' matches when used for filter matching with
|
||||
* {@link ValueMatcher#matches(boolean)},
|
||||
* {@link org.apache.druid.query.filter.vector.VectorValueMatcher#match(ReadableVectorMatch, boolean)}, or
|
||||
* {@link org.apache.druid.segment.index.BitmapColumnIndex#computeBitmapResult(BitmapResultFactory, boolean)}.
|
||||
*
|
||||
* If returns true, unknown (null) inputs can automatically be considered matches if {@code includeUnknown} is set
|
||||
* to true on these methods, else null inputs should be evaluated against the predicate as any other value to
|
||||
* determine a match
|
||||
*/
|
||||
default boolean isNullInputUnknown()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Three-value logic result for matching values with predicates produced by {@link DruidPredicateFactory}
|
||||
*
|
||||
* @see DruidObjectPredicate
|
||||
* @see DruidLongPredicate
|
||||
* @see DruidFloatPredicate
|
||||
* @see DruidDoublePredicate
|
||||
*/
|
||||
public enum DruidPredicateMatch
|
||||
{
|
||||
/**
|
||||
* Value does not match
|
||||
*/
|
||||
FALSE,
|
||||
/**
|
||||
* Value matches
|
||||
*/
|
||||
TRUE,
|
||||
/**
|
||||
* Value is unknown to match, for example from a null input to a predicate which does not match null as true or false
|
||||
*/
|
||||
UNKNOWN;
|
||||
|
||||
/**
|
||||
* Convenience method for {@link ValueMatcher} and {@link org.apache.druid.query.filter.vector.VectorValueMatcher}
|
||||
* implementations to pass through the 'includeUnknown' parameter to the predicate match result.
|
||||
*/
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
return this == TRUE || (includeUnknown && this == UNKNOWN);
|
||||
}
|
||||
|
||||
public static DruidPredicateMatch of(boolean val)
|
||||
{
|
||||
if (val) {
|
||||
return TRUE;
|
||||
}
|
||||
return FALSE;
|
||||
}
|
||||
}
|
|
@ -22,8 +22,6 @@ package org.apache.druid.query.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -340,13 +338,13 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
public static class EqualityPredicateFactory implements DruidPredicateFactory
|
||||
{
|
||||
private final ExprEval<?> matchValue;
|
||||
private final Supplier<Predicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidObjectPredicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidLongPredicate> longPredicateSupplier;
|
||||
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
|
||||
private final Supplier<DruidDoublePredicate> doublePredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, Predicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<Predicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
private final Supplier<Predicate<Object>> objectPredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, DruidObjectPredicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<DruidObjectPredicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
private final Supplier<DruidObjectPredicate<Object>> objectPredicateSupplier;
|
||||
|
||||
public EqualityPredicateFactory(ExprEval<?> matchValue)
|
||||
{
|
||||
|
@ -361,7 +359,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return stringPredicateSupplier.get();
|
||||
}
|
||||
|
@ -385,10 +383,10 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
public DruidObjectPredicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
if (!matchValue.isArray()) {
|
||||
return Predicates.alwaysFalse();
|
||||
return DruidObjectPredicate.alwaysFalseWithNullUnknown();
|
||||
}
|
||||
if (arrayType == null) {
|
||||
// fall back to per row detection if input array type is unknown
|
||||
|
@ -402,19 +400,19 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
public DruidObjectPredicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return objectPredicateSupplier.get();
|
||||
}
|
||||
|
||||
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
||||
private Supplier<DruidObjectPredicate<String>> makeStringPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.STRING);
|
||||
if (castForComparison == null) {
|
||||
return Predicates.alwaysFalse();
|
||||
return DruidObjectPredicate.alwaysFalseWithNullUnknown();
|
||||
}
|
||||
return Predicates.equalTo(castForComparison.asString());
|
||||
return DruidObjectPredicate.equalTo(castForComparison.asString());
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -423,11 +421,11 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
return Suppliers.memoize(() -> {
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.LONG);
|
||||
if (castForComparison == null) {
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = castForComparison.asLong();
|
||||
return input -> input == unboxedLong;
|
||||
return input -> DruidPredicateMatch.of(input == unboxedLong);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -437,11 +435,11 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
return Suppliers.memoize(() -> {
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||
if (castForComparison == null) {
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
return DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
// Compare with floatToIntBits instead of == to canonicalize NaNs.
|
||||
final int floatBits = Float.floatToIntBits((float) castForComparison.asDouble());
|
||||
return input -> Float.floatToIntBits(input) == floatBits;
|
||||
return input -> DruidPredicateMatch.of(Float.floatToIntBits(input) == floatBits);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -451,50 +449,53 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
return Suppliers.memoize(() -> {
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||
if (castForComparison == null) {
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
// Compare with doubleToLongBits instead of == to canonicalize NaNs.
|
||||
final long bits = Double.doubleToLongBits(castForComparison.asDouble());
|
||||
return input -> Double.doubleToLongBits(input) == bits;
|
||||
return input -> DruidPredicateMatch.of(Double.doubleToLongBits(input) == bits);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object>> makeObjectPredicateSupplier()
|
||||
private Supplier<DruidObjectPredicate<Object>> makeObjectPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
if (matchValue.type().equals(ExpressionType.NESTED_DATA)) {
|
||||
return input -> Objects.equals(StructuredData.unwrap(input), StructuredData.unwrap(matchValue.value()));
|
||||
return input -> input == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.of(Objects.equals(StructuredData.unwrap(input), StructuredData.unwrap(matchValue.value())));
|
||||
}
|
||||
return Predicates.equalTo(matchValue.valueOrDefault());
|
||||
return DruidObjectPredicate.equalTo(matchValue.valueOrDefault());
|
||||
});
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
private Supplier<DruidObjectPredicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
{
|
||||
return Suppliers.memoize(() -> input -> {
|
||||
if (input == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final ExprEval<?> eval = ExprEval.bestEffortOf(input);
|
||||
final Comparator<Object[]> arrayComparator = eval.type().getNullableStrategy();
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, eval.type());
|
||||
if (castForComparison == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final Object[] matchArray = castForComparison.asArray();
|
||||
return arrayComparator.compare(input, matchArray) == 0;
|
||||
return DruidPredicateMatch.of(arrayComparator.compare(input, matchArray) == 0);
|
||||
});
|
||||
}
|
||||
|
||||
private Predicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
|
||||
private DruidObjectPredicate<Object[]> makeArrayPredicateInternal(TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType);
|
||||
final Comparator<Object[]> arrayComparator = arrayType.getNullableStrategy();
|
||||
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, expressionType);
|
||||
if (castForComparison == null) {
|
||||
return Predicates.alwaysFalse();
|
||||
return DruidObjectPredicate.alwaysFalseWithNullUnknown();
|
||||
}
|
||||
final Object[] matchArray = castForComparison.asArray();
|
||||
return input -> arrayComparator.compare(input, matchArray) == 0;
|
||||
return input -> input == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.of(arrayComparator.compare(input, matchArray) == 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -549,7 +550,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
{
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.STRING);
|
||||
if (castForComparison == null) {
|
||||
return ValueMatchers.makeAlwaysFalseDimensionMatcher(selector, multiValue);
|
||||
return ValueMatchers.makeAlwaysFalseWithNullUnknownDimensionMatcher(selector, multiValue);
|
||||
}
|
||||
return ValueMatchers.makeStringValueMatcher(selector, castForComparison.asString(), multiValue);
|
||||
}
|
||||
|
@ -559,7 +560,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
{
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||
if (castForComparison == null) {
|
||||
return ValueMatchers.makeAlwaysFalseNumericMatcher(selector);
|
||||
return ValueMatchers.makeAlwaysFalseWithNullUnknownNumericMatcher(selector);
|
||||
}
|
||||
return ValueMatchers.makeFloatValueMatcher(selector, (float) castForComparison.asDouble());
|
||||
}
|
||||
|
@ -569,7 +570,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
{
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE);
|
||||
if (castForComparison == null) {
|
||||
return ValueMatchers.makeAlwaysFalseNumericMatcher(selector);
|
||||
return ValueMatchers.makeAlwaysFalseWithNullUnknownNumericMatcher(selector);
|
||||
}
|
||||
return ValueMatchers.makeDoubleValueMatcher(selector, castForComparison.asDouble());
|
||||
}
|
||||
|
@ -579,7 +580,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
|
|||
{
|
||||
final ExprEval<?> castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.LONG);
|
||||
if (castForComparison == null) {
|
||||
return ValueMatchers.makeAlwaysFalseNumericMatcher(selector);
|
||||
return ValueMatchers.makeAlwaysFalseWithNullUnknownNumericMatcher(selector);
|
||||
}
|
||||
return ValueMatchers.makeLongValueMatcher(selector, castForComparison.asLong());
|
||||
}
|
||||
|
|
|
@ -31,20 +31,20 @@ import javax.annotation.Nullable;
|
|||
* is encountered. Useful when processing data that might be mixed types, despite what the column capabilities might
|
||||
* claim the type is, such as variant 'auto' types. This class is not thread-safe.
|
||||
*/
|
||||
public class FallbackPredicate<T> implements Predicate<T>
|
||||
public class FallbackPredicate<T> implements DruidObjectPredicate<T>
|
||||
{
|
||||
private final Predicate<T> delegate;
|
||||
private final DruidObjectPredicate<T> delegate;
|
||||
private final ExpressionType expectedType;
|
||||
private boolean needsCast = false;
|
||||
|
||||
public FallbackPredicate(Predicate<T> delegate, ExpressionType expectedType)
|
||||
public FallbackPredicate(DruidObjectPredicate<T> delegate, ExpressionType expectedType)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
this.expectedType = expectedType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean apply(@Nullable T input)
|
||||
public DruidPredicateMatch apply(@Nullable T input)
|
||||
{
|
||||
if (needsCast) {
|
||||
return castApply(input);
|
||||
|
@ -58,7 +58,7 @@ public class FallbackPredicate<T> implements Predicate<T>
|
|||
}
|
||||
}
|
||||
|
||||
private boolean castApply(@Nullable T input)
|
||||
private DruidPredicateMatch castApply(@Nullable T input)
|
||||
{
|
||||
final ExprEval<T> castEval = ExprEval.bestEffortOf(input).castTo(expectedType);
|
||||
return delegate.apply(castEval.value());
|
||||
|
|
|
@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ForwardingSortedSet;
|
||||
|
@ -302,10 +301,8 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
|
||||
if (indexSupplier == null) {
|
||||
// column doesn't exist, match against null
|
||||
return Filters.makeMissingColumnNullIndex(
|
||||
predicateFactory.makeStringPredicate().apply(null),
|
||||
selector
|
||||
);
|
||||
DruidPredicateMatch match = predicateFactory.makeStringPredicate().apply(null);
|
||||
return Filters.makeMissingColumnNullIndex(match, selector);
|
||||
}
|
||||
|
||||
final Utf8ValueSetIndexes utf8ValueSetIndexes = indexSupplier.as(Utf8ValueSetIndexes.class);
|
||||
|
@ -535,10 +532,15 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
}
|
||||
|
||||
@SuppressWarnings("ReturnValueIgnored")
|
||||
private static Predicate<String> createStringPredicate(final Set<String> values)
|
||||
private static DruidObjectPredicate<String> createStringPredicate(final Set<String> values)
|
||||
{
|
||||
Preconditions.checkNotNull(values, "values");
|
||||
return values::contains;
|
||||
return value -> {
|
||||
if (value == null) {
|
||||
return values.contains(null) ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(values.contains(value));
|
||||
};
|
||||
}
|
||||
|
||||
private static DruidLongPredicate createLongPredicate(final Set<String> values)
|
||||
|
@ -556,15 +558,15 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
return new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long n)
|
||||
public DruidPredicateMatch applyLong(long n)
|
||||
{
|
||||
return longHashSet.contains(n);
|
||||
return DruidPredicateMatch.of(longHashSet.contains(n));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return matchNull;
|
||||
return matchNull ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -584,15 +586,15 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
return new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyFloat(float n)
|
||||
public DruidPredicateMatch applyFloat(float n)
|
||||
{
|
||||
return floatBitsHashSet.contains(Float.floatToIntBits(n));
|
||||
return DruidPredicateMatch.of(floatBitsHashSet.contains(Float.floatToIntBits(n)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return matchNull;
|
||||
return matchNull ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -612,15 +614,15 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
return new DruidDoublePredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyDouble(double n)
|
||||
public DruidPredicateMatch applyDouble(double n)
|
||||
{
|
||||
return doubleBitsHashSet.contains(Double.doubleToLongBits(n));
|
||||
return DruidPredicateMatch.of(doubleBitsHashSet.contains(Double.doubleToLongBits(n)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return matchNull;
|
||||
return matchNull ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -630,11 +632,10 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
{
|
||||
private final ExtractionFn extractionFn;
|
||||
private final Set<String> values;
|
||||
private final Supplier<Predicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidObjectPredicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidLongPredicate> longPredicateSupplier;
|
||||
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
|
||||
private final Supplier<DruidDoublePredicate> doublePredicateSupplier;
|
||||
private final boolean hasNull;
|
||||
|
||||
public InFilterDruidPredicateFactory(
|
||||
final ExtractionFn extractionFn,
|
||||
|
@ -643,7 +644,6 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
{
|
||||
this.extractionFn = extractionFn;
|
||||
this.values = values;
|
||||
this.hasNull = values.contains(null);
|
||||
|
||||
// As the set of filtered values can be large, parsing them as numbers 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
|
||||
|
@ -656,10 +656,10 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
final Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
final DruidObjectPredicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
return input -> stringPredicate.apply(extractionFn.apply(input));
|
||||
} else {
|
||||
return stringPredicateSupplier.get();
|
||||
|
@ -670,7 +670,7 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
final Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
final DruidObjectPredicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
return input -> stringPredicate.apply(extractionFn.apply(input));
|
||||
} else {
|
||||
return longPredicateSupplier.get();
|
||||
|
@ -681,7 +681,7 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
final Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
final DruidObjectPredicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
return input -> stringPredicate.apply(extractionFn.apply(input));
|
||||
} else {
|
||||
return floatPredicateSupplier.get();
|
||||
|
@ -692,19 +692,13 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
final Predicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
final DruidObjectPredicate<String> stringPredicate = stringPredicateSupplier.get();
|
||||
return input -> stringPredicate.apply(extractionFn.apply(input));
|
||||
} else {
|
||||
return doublePredicateSupplier.get();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return !hasNull;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonInclude;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -241,7 +240,7 @@ public class JavaScriptDimFilter extends AbstractOptimizableDimFilter implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return this::applyObject;
|
||||
}
|
||||
|
@ -253,13 +252,13 @@ public class JavaScriptDimFilter extends AbstractOptimizableDimFilter implements
|
|||
return new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long input)
|
||||
public DruidPredicateMatch applyLong(long input)
|
||||
{
|
||||
return JavaScriptPredicateFactory.this.applyObject(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return JavaScriptPredicateFactory.this.applyObject(null);
|
||||
}
|
||||
|
@ -273,13 +272,13 @@ public class JavaScriptDimFilter extends AbstractOptimizableDimFilter implements
|
|||
return new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyFloat(float input)
|
||||
public DruidPredicateMatch applyFloat(float input)
|
||||
{
|
||||
return JavaScriptPredicateFactory.this.applyObject(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return JavaScriptPredicateFactory.this.applyObject(null);
|
||||
}
|
||||
|
@ -293,25 +292,25 @@ public class JavaScriptDimFilter extends AbstractOptimizableDimFilter implements
|
|||
return new DruidDoublePredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyDouble(double input)
|
||||
public DruidPredicateMatch applyDouble(double input)
|
||||
{
|
||||
return JavaScriptPredicateFactory.this.applyObject(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return JavaScriptPredicateFactory.this.applyObject(null);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public boolean applyObject(final Object input)
|
||||
public DruidPredicateMatch applyObject(final Object input)
|
||||
{
|
||||
// one and only one context per thread
|
||||
final Context cx = Context.enter();
|
||||
try {
|
||||
return applyInContext(cx, input);
|
||||
return DruidPredicateMatch.of(applyInContext(cx, input));
|
||||
}
|
||||
finally {
|
||||
Context.exit();
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonInclude;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.io.BaseEncoding;
|
||||
|
@ -276,27 +275,30 @@ public class LikeDimFilter extends AbstractOptimizableDimFilter implements DimFi
|
|||
}
|
||||
}
|
||||
|
||||
public boolean matches(@Nullable final String s)
|
||||
public DruidPredicateMatch matches(@Nullable final String s)
|
||||
{
|
||||
return matches(s, pattern);
|
||||
}
|
||||
|
||||
private static boolean matches(@Nullable final String s, Pattern pattern)
|
||||
private static DruidPredicateMatch matches(@Nullable final String s, Pattern pattern)
|
||||
{
|
||||
String val = NullHandling.nullToEmptyIfNeeded(s);
|
||||
return val != null && pattern.matcher(val).matches();
|
||||
if (val == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(pattern.matcher(val).matches());
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the suffix of "value" matches the suffix of this matcher. The first prefix.length() characters
|
||||
* of "value" are ignored. This method is useful if you've already independently verified the prefix.
|
||||
*/
|
||||
public boolean matchesSuffixOnly(@Nullable String value)
|
||||
public DruidPredicateMatch matchesSuffixOnly(@Nullable String value)
|
||||
{
|
||||
if (suffixMatch == SuffixMatch.MATCH_ANY) {
|
||||
return true;
|
||||
return DruidPredicateMatch.TRUE;
|
||||
} else if (suffixMatch == SuffixMatch.MATCH_EMPTY) {
|
||||
return value == null ? matches(null) : value.length() == prefix.length();
|
||||
return value == null ? matches(null) : DruidPredicateMatch.of(value.length() == prefix.length());
|
||||
} else {
|
||||
// suffixMatch is MATCH_PATTERN
|
||||
return matches(value);
|
||||
|
@ -331,7 +333,7 @@ public class LikeDimFilter extends AbstractOptimizableDimFilter implements DimFi
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
return input -> matches(extractionFn.apply(input), pattern);
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.druid.query.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Range;
|
||||
import com.google.common.collect.RangeSet;
|
||||
|
@ -228,9 +226,9 @@ public class NullFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return Predicates.isNull();
|
||||
return DruidObjectPredicate.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -252,22 +250,15 @@ public class NullFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
public DruidObjectPredicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
return Predicates.isNull();
|
||||
return DruidObjectPredicate.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
public DruidObjectPredicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return Predicates.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
// this filter only matches null inputs
|
||||
return false;
|
||||
return DruidObjectPredicate.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.druid.query.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.BoundType;
|
||||
|
@ -82,12 +80,12 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
private final boolean upperOpen;
|
||||
@Nullable
|
||||
private final FilterTuning filterTuning;
|
||||
private final Supplier<Predicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidObjectPredicate<String>> stringPredicateSupplier;
|
||||
private final Supplier<DruidLongPredicate> longPredicateSupplier;
|
||||
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
|
||||
private final Supplier<DruidDoublePredicate> doublePredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, Predicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<Predicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
private final ConcurrentHashMap<TypeSignature<ValueType>, DruidObjectPredicate<Object[]>> arrayPredicates;
|
||||
private final Supplier<DruidObjectPredicate<Object[]>> typeDetectingArrayPredicateSupplier;
|
||||
|
||||
@JsonCreator
|
||||
public RangeFilter(
|
||||
|
@ -506,7 +504,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
// lower value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||
// but just in case...
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
if (lowerOpen) {
|
||||
// lower bound is open, so take the floor of the value so that x > 1.1 can match 2 but not 1
|
||||
|
@ -528,7 +526,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
// upper value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||
// but just in case...
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
if (upperOpen) {
|
||||
// upper bound is open, take the ceil so that x < 1.1 can match 1 but not 2
|
||||
|
@ -570,7 +568,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
// lower value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||
// but just in case...
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
lowerBound = lowerCast.asDouble();
|
||||
hasLowerBound = true;
|
||||
|
@ -586,7 +584,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
// upper value is not null, but isn't convertible to a long so is effectively null, nothing matches
|
||||
// this shouldn't be possible because we only use numeric predicates when the match value type is numeric
|
||||
// but just in case...
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
hasUpperBound = true;
|
||||
upperBound = upperCast.asDouble();
|
||||
|
@ -601,7 +599,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
});
|
||||
}
|
||||
|
||||
private Supplier<Predicate<String>> makeStringPredicateSupplier()
|
||||
private Supplier<DruidObjectPredicate<String>> makeStringPredicateSupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
final Comparator<String> stringComparator =
|
||||
|
@ -615,7 +613,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
});
|
||||
}
|
||||
|
||||
private Predicate<Object[]> makeArrayPredicate(TypeSignature<ValueType> inputType)
|
||||
private DruidObjectPredicate<Object[]> makeArrayPredicate(TypeSignature<ValueType> inputType)
|
||||
{
|
||||
final Comparator<Object[]> arrayComparator;
|
||||
if (inputType.getElementType().is(ValueType.STRING) && Types.isNumericOrNumericArray(matchValueType)) {
|
||||
|
@ -649,7 +647,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
return makeComparatorPredicate(rangeType, arrayComparator, lowerBound, upperBound);
|
||||
}
|
||||
|
||||
private Supplier<Predicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
private Supplier<DruidObjectPredicate<Object[]>> makeTypeDetectingArrayPredicate()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
RangeType rangeType = RangeType.of(hasLowerBound(), lowerOpen, hasUpperBound(), upperOpen);
|
||||
|
@ -657,7 +655,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
case OPEN:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
||||
|
@ -665,12 +663,12 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
final Comparator<Object[]> comparator = val.type().getNullableStrategy();
|
||||
final int lowerComparing = comparator.compare(val.asArray(), lowerBound);
|
||||
final int upperComparing = comparator.compare(upperBound, val.asArray());
|
||||
return ((lowerComparing > 0)) && (upperComparing > 0);
|
||||
return DruidPredicateMatch.of(((lowerComparing > 0)) && (upperComparing > 0));
|
||||
};
|
||||
case LOWER_OPEN_UPPER_CLOSED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
||||
|
@ -678,12 +676,12 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||
return (lowerComparing > 0) && (upperComparing >= 0);
|
||||
return DruidPredicateMatch.of((lowerComparing > 0) && (upperComparing >= 0));
|
||||
};
|
||||
case LOWER_CLOSED_UPPER_OPEN:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType());
|
||||
|
@ -691,12 +689,12 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||
return (lowerComparing >= 0) && (upperComparing > 0);
|
||||
return DruidPredicateMatch.of((lowerComparing >= 0) && (upperComparing > 0));
|
||||
};
|
||||
case CLOSED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType());
|
||||
|
@ -704,55 +702,55 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound);
|
||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||
return (lowerComparing >= 0) && (upperComparing >= 0);
|
||||
return DruidPredicateMatch.of((lowerComparing >= 0) && (upperComparing >= 0));
|
||||
};
|
||||
case LOWER_UNBOUNDED_UPPER_OPEN:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] upperBound = castArrayForComparisonWithCeilIfNeeded(upperEval, val.asArrayType());
|
||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||
return upperComparing > 0;
|
||||
return DruidPredicateMatch.of(upperComparing > 0);
|
||||
};
|
||||
case LOWER_UNBOUNDED_UPPER_CLOSED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] upperBound = upperEval.castTo(val.type()).asArray();
|
||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int upperComparing = arrayComparator.compare(upperBound, val.asArray());
|
||||
return upperComparing >= 0;
|
||||
return DruidPredicateMatch.of(upperComparing >= 0);
|
||||
};
|
||||
case LOWER_OPEN_UPPER_UNBOUNDED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] lowerBound = lowerEval.castTo(val.type()).asArray();
|
||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray());
|
||||
return lowerComparing > 0;
|
||||
return DruidPredicateMatch.of(lowerComparing > 0);
|
||||
};
|
||||
case LOWER_CLOSED_UPPER_UNBOUNDED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
ExprEval<?> val = ExprEval.bestEffortOf(input);
|
||||
final Object[] lowerBound = castArrayForComparisonWithCeilIfNeeded(lowerEval, val.asArrayType());
|
||||
final Comparator<Object[]> arrayComparator = val.type().getNullableStrategy();
|
||||
final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray());
|
||||
return lowerComparing >= 0;
|
||||
return DruidPredicateMatch.of(lowerComparing >= 0);
|
||||
};
|
||||
case UNBOUNDED:
|
||||
default:
|
||||
return Predicates.notNull();
|
||||
return DruidObjectPredicate.notNull();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -767,7 +765,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return new FallbackPredicate<>(
|
||||
stringPredicateSupplier.get(),
|
||||
|
@ -781,7 +779,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
if (matchValueType.isNumeric()) {
|
||||
return longPredicateSupplier.get();
|
||||
}
|
||||
Predicate<String> stringPredicate = makeStringPredicate();
|
||||
DruidObjectPredicate<String> stringPredicate = makeStringPredicate();
|
||||
return input -> stringPredicate.apply(Evals.asString(input));
|
||||
}
|
||||
|
||||
|
@ -791,7 +789,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
if (matchValueType.isNumeric()) {
|
||||
return floatPredicateSupplier.get();
|
||||
}
|
||||
Predicate<String> stringPredicate = makeStringPredicate();
|
||||
DruidObjectPredicate<String> stringPredicate = makeStringPredicate();
|
||||
return input -> stringPredicate.apply(Evals.asString(input));
|
||||
}
|
||||
|
||||
|
@ -801,12 +799,12 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
if (matchValueType.isNumeric()) {
|
||||
return doublePredicateSupplier.get();
|
||||
}
|
||||
Predicate<String> stringPredicate = makeStringPredicate();
|
||||
DruidObjectPredicate<String> stringPredicate = makeStringPredicate();
|
||||
return input -> stringPredicate.apply(Evals.asString(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> inputType)
|
||||
public DruidObjectPredicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> inputType)
|
||||
{
|
||||
if (inputType == null) {
|
||||
return typeDetectingArrayPredicateSupplier.get();
|
||||
|
@ -874,21 +872,21 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
{
|
||||
switch (rangeType) {
|
||||
case OPEN:
|
||||
return input -> input > lowerLongBound && input < upperLongBound;
|
||||
return input -> DruidPredicateMatch.of(input > lowerLongBound && input < upperLongBound);
|
||||
case LOWER_OPEN_UPPER_CLOSED:
|
||||
return input -> input > lowerLongBound && input <= upperLongBound;
|
||||
return input -> DruidPredicateMatch.of(input > lowerLongBound && input <= upperLongBound);
|
||||
case LOWER_CLOSED_UPPER_OPEN:
|
||||
return input -> input >= lowerLongBound && input < upperLongBound;
|
||||
return input -> DruidPredicateMatch.of(input >= lowerLongBound && input < upperLongBound);
|
||||
case CLOSED:
|
||||
return input -> input >= lowerLongBound && input <= upperLongBound;
|
||||
return input -> DruidPredicateMatch.of(input >= lowerLongBound && input <= upperLongBound);
|
||||
case LOWER_UNBOUNDED_UPPER_OPEN:
|
||||
return input -> input < upperLongBound;
|
||||
return input -> DruidPredicateMatch.of(input < upperLongBound);
|
||||
case LOWER_UNBOUNDED_UPPER_CLOSED:
|
||||
return input -> input <= upperLongBound;
|
||||
return input -> DruidPredicateMatch.of(input <= upperLongBound);
|
||||
case LOWER_OPEN_UPPER_UNBOUNDED:
|
||||
return input -> input > lowerLongBound;
|
||||
return input -> DruidPredicateMatch.of(input > lowerLongBound);
|
||||
case LOWER_CLOSED_UPPER_UNBOUNDED:
|
||||
return input -> input >= lowerLongBound;
|
||||
return input -> DruidPredicateMatch.of(input >= lowerLongBound);
|
||||
case UNBOUNDED:
|
||||
default:
|
||||
return DruidLongPredicate.ALWAYS_TRUE;
|
||||
|
@ -906,45 +904,45 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return ((lowerComparing > 0)) && (upperComparing > 0);
|
||||
return DruidPredicateMatch.of(((lowerComparing > 0)) && (upperComparing > 0));
|
||||
};
|
||||
case LOWER_OPEN_UPPER_CLOSED:
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return (lowerComparing > 0) && (upperComparing >= 0);
|
||||
return DruidPredicateMatch.of((lowerComparing > 0) && (upperComparing >= 0));
|
||||
};
|
||||
case LOWER_CLOSED_UPPER_OPEN:
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return (lowerComparing >= 0) && (upperComparing > 0);
|
||||
return DruidPredicateMatch.of((lowerComparing >= 0) && (upperComparing > 0));
|
||||
};
|
||||
case CLOSED:
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return (lowerComparing >= 0) && (upperComparing >= 0);
|
||||
return DruidPredicateMatch.of((lowerComparing >= 0) && (upperComparing >= 0));
|
||||
};
|
||||
case LOWER_UNBOUNDED_UPPER_OPEN:
|
||||
return input -> {
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return upperComparing > 0;
|
||||
return DruidPredicateMatch.of(upperComparing > 0);
|
||||
};
|
||||
case LOWER_UNBOUNDED_UPPER_CLOSED:
|
||||
return input -> {
|
||||
final int upperComparing = Double.compare(upperDoubleBound, input);
|
||||
return upperComparing >= 0;
|
||||
return DruidPredicateMatch.of(upperComparing >= 0);
|
||||
};
|
||||
case LOWER_OPEN_UPPER_UNBOUNDED:
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
return lowerComparing > 0;
|
||||
return DruidPredicateMatch.of(lowerComparing > 0);
|
||||
};
|
||||
case LOWER_CLOSED_UPPER_UNBOUNDED:
|
||||
return input -> {
|
||||
final int lowerComparing = Double.compare(input, lowerDoubleBound);
|
||||
return lowerComparing >= 0;
|
||||
return DruidPredicateMatch.of(lowerComparing >= 0);
|
||||
};
|
||||
case UNBOUNDED:
|
||||
default:
|
||||
|
@ -952,7 +950,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
}
|
||||
}
|
||||
|
||||
public static <T> Predicate<T> makeComparatorPredicate(
|
||||
public static <T> DruidObjectPredicate<T> makeComparatorPredicate(
|
||||
RangeType rangeType,
|
||||
Comparator<T> comparator,
|
||||
@Nullable T lowerBound,
|
||||
|
@ -963,74 +961,74 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
|
|||
case OPEN:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int lowerComparing = comparator.compare(input, lowerBound);
|
||||
final int upperComparing = comparator.compare(upperBound, input);
|
||||
return ((lowerComparing > 0)) && (upperComparing > 0);
|
||||
return DruidPredicateMatch.of(((lowerComparing > 0)) && (upperComparing > 0));
|
||||
};
|
||||
case LOWER_OPEN_UPPER_CLOSED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int lowerComparing = comparator.compare(input, lowerBound);
|
||||
final int upperComparing = comparator.compare(upperBound, input);
|
||||
return (lowerComparing > 0) && (upperComparing >= 0);
|
||||
return DruidPredicateMatch.of((lowerComparing > 0) && (upperComparing >= 0));
|
||||
};
|
||||
case LOWER_CLOSED_UPPER_OPEN:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int lowerComparing = comparator.compare(input, lowerBound);
|
||||
final int upperComparing = comparator.compare(upperBound, input);
|
||||
return (lowerComparing >= 0) && (upperComparing > 0);
|
||||
return DruidPredicateMatch.of((lowerComparing >= 0) && (upperComparing > 0));
|
||||
};
|
||||
case CLOSED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int lowerComparing = comparator.compare(input, lowerBound);
|
||||
final int upperComparing = comparator.compare(upperBound, input);
|
||||
return (lowerComparing >= 0) && (upperComparing >= 0);
|
||||
return DruidPredicateMatch.of((lowerComparing >= 0) && (upperComparing >= 0));
|
||||
};
|
||||
case LOWER_UNBOUNDED_UPPER_OPEN:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int upperComparing = comparator.compare(upperBound, input);
|
||||
return upperComparing > 0;
|
||||
return DruidPredicateMatch.of(upperComparing > 0);
|
||||
};
|
||||
case LOWER_UNBOUNDED_UPPER_CLOSED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int upperComparing = comparator.compare(upperBound, input);
|
||||
return upperComparing >= 0;
|
||||
return DruidPredicateMatch.of(upperComparing >= 0);
|
||||
};
|
||||
case LOWER_OPEN_UPPER_UNBOUNDED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int lowerComparing = comparator.compare(input, lowerBound);
|
||||
return lowerComparing > 0;
|
||||
return DruidPredicateMatch.of(lowerComparing > 0);
|
||||
};
|
||||
case LOWER_CLOSED_UPPER_UNBOUNDED:
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final int lowerComparing = comparator.compare(input, lowerBound);
|
||||
return lowerComparing >= 0;
|
||||
return DruidPredicateMatch.of(lowerComparing >= 0);
|
||||
};
|
||||
case UNBOUNDED:
|
||||
default:
|
||||
return Predicates.notNull();
|
||||
return DruidObjectPredicate.notNull();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
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;
|
||||
|
@ -40,18 +38,19 @@ public class SelectorPredicateFactory implements DruidPredicateFactory
|
|||
private volatile DruidLongPredicate longPredicate;
|
||||
private volatile DruidFloatPredicate floatPredicate;
|
||||
private volatile DruidDoublePredicate doublePredicate;
|
||||
private final boolean isNullUnknown;
|
||||
|
||||
public SelectorPredicateFactory(@Nullable String value)
|
||||
{
|
||||
this.value = value;
|
||||
this.isNullUnknown = value != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return Predicates.equalTo(value);
|
||||
if (value == null) {
|
||||
return DruidObjectPredicate.isNull();
|
||||
}
|
||||
return DruidObjectPredicate.equalTo(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -75,12 +74,6 @@ public class SelectorPredicateFactory implements DruidPredicateFactory
|
|||
return doublePredicate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullUnknown;
|
||||
}
|
||||
|
||||
private void initLongPredicate()
|
||||
{
|
||||
if (longPredicate != null) {
|
||||
|
@ -97,11 +90,11 @@ public class SelectorPredicateFactory implements DruidPredicateFactory
|
|||
final Long valueAsLong = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
|
||||
if (valueAsLong == null) {
|
||||
longPredicate = DruidLongPredicate.ALWAYS_FALSE;
|
||||
longPredicate = DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = valueAsLong;
|
||||
longPredicate = input -> input == unboxedLong;
|
||||
longPredicate = input -> DruidPredicateMatch.of(input == unboxedLong);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -123,11 +116,11 @@ public class SelectorPredicateFactory implements DruidPredicateFactory
|
|||
final Float valueAsFloat = DimensionHandlerUtils.convertObjectToFloat(value);
|
||||
|
||||
if (valueAsFloat == null) {
|
||||
floatPredicate = DruidFloatPredicate.ALWAYS_FALSE;
|
||||
floatPredicate = DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
// Compare with floatToIntBits instead of == to canonicalize NaNs.
|
||||
final int floatBits = Float.floatToIntBits(valueAsFloat);
|
||||
floatPredicate = input -> Float.floatToIntBits(input) == floatBits;
|
||||
floatPredicate = input -> DruidPredicateMatch.of(Float.floatToIntBits(input) == floatBits);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -148,11 +141,11 @@ public class SelectorPredicateFactory implements DruidPredicateFactory
|
|||
final Double aDouble = DimensionHandlerUtils.convertObjectToDouble(value);
|
||||
|
||||
if (aDouble == null) {
|
||||
doublePredicate = DruidDoublePredicate.ALWAYS_FALSE;
|
||||
doublePredicate = DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
} else {
|
||||
// Compare with doubleToLongBits instead of == to canonicalize NaNs.
|
||||
final long bits = Double.doubleToLongBits(aDouble);
|
||||
doublePredicate = input -> Double.doubleToLongBits(input) == bits;
|
||||
doublePredicate = input -> DruidPredicateMatch.of(Double.doubleToLongBits(input) == bits);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.filter;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import org.apache.druid.error.DruidException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -30,26 +28,27 @@ public class StringPredicateDruidPredicateFactory implements DruidPredicateFacto
|
|||
{
|
||||
public static StringPredicateDruidPredicateFactory equalTo(@Nullable String value)
|
||||
{
|
||||
return new StringPredicateDruidPredicateFactory(Predicates.equalTo(value), value != null);
|
||||
if (value == null) {
|
||||
return new StringPredicateDruidPredicateFactory(DruidObjectPredicate.isNull());
|
||||
}
|
||||
return new StringPredicateDruidPredicateFactory(DruidObjectPredicate.equalTo(value));
|
||||
}
|
||||
|
||||
public static StringPredicateDruidPredicateFactory of(@Nullable Predicate<String> predicate, boolean isNullInputUnknown)
|
||||
public static StringPredicateDruidPredicateFactory of(@Nullable DruidObjectPredicate<String> predicate)
|
||||
{
|
||||
return new StringPredicateDruidPredicateFactory(predicate, isNullInputUnknown);
|
||||
return new StringPredicateDruidPredicateFactory(predicate);
|
||||
}
|
||||
|
||||
private final boolean isNullInputUnknown;
|
||||
@Nullable
|
||||
private final Predicate<String> predicate;
|
||||
private final DruidObjectPredicate<String> predicate;
|
||||
|
||||
public StringPredicateDruidPredicateFactory(Predicate<String> predicate, boolean isNullInputUnknown)
|
||||
private StringPredicateDruidPredicateFactory(DruidObjectPredicate<String> predicate)
|
||||
{
|
||||
this.predicate = predicate;
|
||||
this.isNullInputUnknown = isNullInputUnknown;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return predicate;
|
||||
}
|
||||
|
@ -72,12 +71,6 @@ public class StringPredicateDruidPredicateFactory implements DruidPredicateFacto
|
|||
throw DruidException.defensive("String equality predicate factory only supports string predicates");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullInputUnknown;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -88,12 +81,12 @@ public class StringPredicateDruidPredicateFactory implements DruidPredicateFacto
|
|||
return false;
|
||||
}
|
||||
StringPredicateDruidPredicateFactory that = (StringPredicateDruidPredicateFactory) o;
|
||||
return isNullInputUnknown == that.isNullInputUnknown && Objects.equals(predicate, that.predicate);
|
||||
return Objects.equals(predicate, that.predicate);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(isNullInputUnknown, predicate);
|
||||
return Objects.hash(predicate);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
|
@ -61,7 +61,7 @@ public class ArrayVectorValueMatcher implements VectorValueMatcherFactory
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<Object[]> predicate = predicateFactory.makeArrayPredicate(columnType);
|
||||
final DruidObjectPredicate<Object[]> predicate = predicateFactory.makeArrayPredicate(columnType);
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
|
||||
|
@ -77,16 +77,12 @@ public class ArrayVectorValueMatcher implements VectorValueMatcherFactory
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
Object o = vector[rowNum];
|
||||
if (includeUnknown && o == null && predicateFactory.isNullInputUnknown()) {
|
||||
if ((o == null || o instanceof Object[])) {
|
||||
if (predicate.apply((Object[]) o).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else if (predicate.apply(new Object[]{o}).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
} else if (o == null || o instanceof Object[]) {
|
||||
if (predicate.apply((Object[]) o)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else {
|
||||
if (predicate.apply(new Object[]{o})) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -121,10 +121,10 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
if (hasNulls && nulls[rowNum]) {
|
||||
if ((includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.applyNull()) {
|
||||
if (predicate.applyNull().matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else if (predicate.applyDouble(vector[rowNum])) {
|
||||
} else if (predicate.applyDouble(vector[rowNum]).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -123,10 +123,10 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
if (hasNulls && nulls[rowNum]) {
|
||||
if ((includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.applyNull()) {
|
||||
if (predicate.applyNull().matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else if (predicate.applyFloat(vector[rowNum])) {
|
||||
} else if (predicate.applyFloat(vector[rowNum]).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -122,10 +122,10 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
if (hasNulls && nulls[rowNum]) {
|
||||
if ((includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.applyNull()) {
|
||||
if (predicate.applyNull().matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else if (predicate.applyLong(vector[rowNum])) {
|
||||
} else if (predicate.applyLong(vector[rowNum]).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -31,7 +31,6 @@ import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.BitSet;
|
||||
import java.util.Objects;
|
||||
|
||||
public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFactory
|
||||
{
|
||||
|
@ -98,7 +97,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
}
|
||||
};
|
||||
} else {
|
||||
return makeMatcher(s -> Objects.equals(s, etnValue), true);
|
||||
return makeMatcher(etnValue == null ? DruidObjectPredicate.isNull() : DruidObjectPredicate.equalTo(etnValue));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -116,13 +115,11 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
return makeMatcher(predicateFactory.makeStringPredicate(), predicateFactory.isNullInputUnknown());
|
||||
return makeMatcher(predicateFactory.makeStringPredicate());
|
||||
}
|
||||
|
||||
private VectorValueMatcher makeMatcher(final Predicate<String> predicate, boolean isNullInputUnknown)
|
||||
private VectorValueMatcher makeMatcher(final DruidObjectPredicate<String> predicate)
|
||||
{
|
||||
final boolean matchNull = predicate.apply(null);
|
||||
|
||||
if (selector.getValueCardinality() > 0) {
|
||||
final BitSet checkedIds = new BitSet(selector.getValueCardinality());
|
||||
final BitSet matchingIds = new BitSet(selector.getValueCardinality());
|
||||
|
@ -137,7 +134,6 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
{
|
||||
final IndexedInts[] vector = selector.getRowVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean includeNulls = includeUnknown && isNullInputUnknown;
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
|
@ -148,7 +144,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
|
||||
if (n == 0) {
|
||||
// null should match empty rows in multi-value columns
|
||||
if (matchNull || includeNulls) {
|
||||
if (predicate.apply(null).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else {
|
||||
|
@ -160,7 +156,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
matches = matchingIds.get(id);
|
||||
} else {
|
||||
final String val = selector.lookupName(id);
|
||||
matches = (includeNulls && val == null) || predicate.apply(val);
|
||||
matches = predicate.apply(val).matches(includeUnknown);
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
|
@ -188,7 +184,6 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
@Override
|
||||
public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown)
|
||||
{
|
||||
final boolean includeNulls = includeUnknown && isNullInputUnknown;
|
||||
final IndexedInts[] vector = selector.getRowVector();
|
||||
final int[] selection = match.getSelection();
|
||||
|
||||
|
@ -201,14 +196,14 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac
|
|||
|
||||
if (n == 0) {
|
||||
// null should match empty rows in multi-value columns
|
||||
if (matchNull || includeNulls) {
|
||||
if (predicate.apply(null).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
} else {
|
||||
for (int j = 0; j < n; j++) {
|
||||
final int id = ints.get(j);
|
||||
final String val = selector.lookupName(id);
|
||||
if ((includeNulls && val == null) || predicate.apply(val)) {
|
||||
if (predicate.apply(val).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.filter.ConstantMatcherType;
|
||||
|
@ -69,7 +69,7 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<Object> predicate = predicateFactory.makeObjectPredicate();
|
||||
final DruidObjectPredicate<Object> predicate = predicateFactory.makeObjectPredicate();
|
||||
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
|
@ -86,7 +86,7 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
final Object o = vector[rowNum];
|
||||
if ((o == null && includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.apply(o)) {
|
||||
if (predicate.apply(o).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -32,7 +32,6 @@ import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.BitSet;
|
||||
import java.util.Objects;
|
||||
|
||||
public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFactory
|
||||
{
|
||||
|
@ -51,7 +50,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
final ConstantMatcherType constantMatcherType = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
selector,
|
||||
false,
|
||||
s -> Objects.equals(s, etnValue)
|
||||
etnValue == null ? DruidObjectPredicate.isNull() : DruidObjectPredicate.equalTo(etnValue)
|
||||
);
|
||||
if (constantMatcherType != null) {
|
||||
return constantMatcherType.asVectorMatcher(selector);
|
||||
|
@ -96,7 +95,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
}
|
||||
};
|
||||
} else {
|
||||
return makeMatcher(s -> Objects.equals(s, etnValue));
|
||||
return makeMatcher(DruidObjectPredicate.equalTo(etnValue));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -117,7 +116,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
return makeMatcher(predicateFactory.makeStringPredicate());
|
||||
}
|
||||
|
||||
private VectorValueMatcher makeMatcher(final Predicate<String> predicate)
|
||||
private VectorValueMatcher makeMatcher(final DruidObjectPredicate<String> predicate)
|
||||
{
|
||||
final ConstantMatcherType constantMatcherType = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
selector,
|
||||
|
@ -155,7 +154,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
matches = matchingIds.get(id);
|
||||
} else {
|
||||
final String val = selector.lookupName(id);
|
||||
matches = (includeUnknown && val == null) || predicate.apply(val);
|
||||
matches = predicate.apply(val).matches(includeUnknown);
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
|
@ -188,7 +187,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
final String val = selector.lookupName(vector[rowNum]);
|
||||
if ((includeUnknown && val == null) || predicate.apply(val)) {
|
||||
if (predicate.apply(val).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
|
@ -83,7 +83,7 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
return new BaseVectorValueMatcher(selector)
|
||||
{
|
||||
|
@ -100,7 +100,7 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory
|
|||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
final int rowNum = mask.getSelection()[i];
|
||||
final String val = (String) vector[rowNum];
|
||||
if ((includeUnknown && val == null) || predicate.apply(val)) {
|
||||
if (predicate.apply(val).matches(includeUnknown)) {
|
||||
selection[numRows++] = rowNum;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||
|
@ -77,15 +77,14 @@ public abstract class BaseSingleValueDimensionSelector implements DimensionSelec
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final String rowValue = getValue();
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
return (matchNull && rowValue == null) || predicate.apply(rowValue);
|
||||
return predicate.apply(rowValue).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,9 +19,10 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
|
@ -73,8 +74,15 @@ public class ConstantDimensionSelector implements SingleValueHistoricalDimension
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return predicate.apply(value) ? ValueMatchers.allTrue() : ValueMatchers.allFalse();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidPredicateMatch match = predicate.apply(value);
|
||||
if (match == DruidPredicateMatch.TRUE) {
|
||||
return ValueMatchers.allTrue();
|
||||
}
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ValueMatchers.allUnknown();
|
||||
}
|
||||
return ValueMatchers.allFalse();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -111,8 +111,8 @@ public class ConstantMultiValueDimensionSelector implements HistoricalDimensionS
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return values.stream().anyMatch(predicate::apply) ? ValueMatchers.allTrue() : ValueMatchers.allFalse();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return values.stream().anyMatch(x -> predicate.apply(x).matches(false)) ? ValueMatchers.allTrue() : ValueMatchers.allFalse();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,11 +19,12 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.annotations.PublicApi;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||
import org.apache.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||
|
@ -271,12 +272,16 @@ public interface DimensionSelector extends ColumnValueSelector<Object>, Dimensio
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
if (predicate.apply(null)) {
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidPredicateMatch match = predicate.apply(null);
|
||||
|
||||
if (match == DruidPredicateMatch.TRUE) {
|
||||
return ValueMatchers.allTrue();
|
||||
}
|
||||
|
||||
return predicateFactory.isNullInputUnknown() ? ValueMatchers.allUnknown() : ValueMatchers.allFalse();
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ValueMatchers.allUnknown();
|
||||
}
|
||||
return ValueMatchers.allFalse();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -182,8 +182,7 @@ public final class DimensionSelectorUtils
|
|||
{
|
||||
final BitSet checkedIds = new BitSet(selector.getValueCardinality());
|
||||
final BitSet matchingIds = new BitSet(selector.getValueCardinality());
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final boolean predicateMatchesNull = predicate.apply(null);
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
|
@ -191,12 +190,11 @@ public final class DimensionSelectorUtils
|
|||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
final IndexedInts row = selector.getRow();
|
||||
final int size = row.size();
|
||||
if (size == 0) {
|
||||
// null should match empty rows in multi-value columns
|
||||
return matchNull || predicateMatchesNull;
|
||||
return predicate.apply(null).matches(includeUnknown);
|
||||
} else {
|
||||
for (int i = 0; i < size; ++i) {
|
||||
final int id = row.get(i);
|
||||
|
@ -206,7 +204,7 @@ public final class DimensionSelectorUtils
|
|||
matches = matchingIds.get(id);
|
||||
} else {
|
||||
final String rowValue = selector.lookupName(id);
|
||||
matches = (matchNull && rowValue == null) || predicate.apply(rowValue);
|
||||
matches = predicate.apply(rowValue).matches(includeUnknown);
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
|
@ -234,23 +232,21 @@ public final class DimensionSelectorUtils
|
|||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final boolean predicateMatchesNull = predicate.apply(null);
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
final IndexedInts row = selector.getRow();
|
||||
final int size = row.size();
|
||||
if (size == 0) {
|
||||
// null should match empty rows in multi-value columns
|
||||
return matchNull || predicateMatchesNull;
|
||||
return predicate.apply(null).matches(includeUnknown);
|
||||
} else {
|
||||
for (int i = 0; i < size; ++i) {
|
||||
final String rowValue = selector.lookupName(row.get(i));
|
||||
if ((matchNull && rowValue == null) || predicate.apply(rowValue)) {
|
||||
if (predicate.apply(rowValue).matches(includeUnknown)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -267,7 +263,7 @@ public final class DimensionSelectorUtils
|
|||
};
|
||||
}
|
||||
|
||||
public static BitSet makePredicateMatchingSet(DimensionSelector selector, Predicate<String> predicate)
|
||||
public static BitSet makePredicateMatchingSet(DimensionSelector selector, DruidObjectPredicate<String> predicate, boolean includeUnknown)
|
||||
{
|
||||
if (!selector.nameLookupPossibleInAdvance()) {
|
||||
throw new IAE("selector.nameLookupPossibleInAdvance() should return true");
|
||||
|
@ -275,7 +271,7 @@ public final class DimensionSelectorUtils
|
|||
int cardinality = selector.getValueCardinality();
|
||||
BitSet valueIds = new BitSet(cardinality);
|
||||
for (int i = 0; i < cardinality; i++) {
|
||||
if (predicate.apply(selector.lookupName(i))) {
|
||||
if (predicate.apply(selector.lookupName(i)).matches(includeUnknown)) {
|
||||
valueIds.set(i);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,11 +20,11 @@
|
|||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
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.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -262,24 +262,22 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final boolean predicateMatchNull = predicate.apply(null);
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
updateCurrentValues();
|
||||
|
||||
if (dimensionValues.isEmpty()) {
|
||||
return matchNull || predicateMatchNull;
|
||||
return predicate.apply(null).matches(includeUnknown);
|
||||
}
|
||||
|
||||
for (String dimensionValue : dimensionValues) {
|
||||
final String coerced = NullHandling.emptyToNullIfNeeded(dimensionValue);
|
||||
if ((matchNull && coerced == null) || predicate.apply(coerced)) {
|
||||
if (predicate.apply(coerced).matches(includeUnknown)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,8 +20,8 @@
|
|||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -95,15 +95,13 @@ public class SingleScanTimeDimensionSelector implements DimensionSelector
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final String rowVal = lookupName(getDimensionValueIndex());
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
return (matchNull && rowVal == null) || predicate.apply(lookupName(getDimensionValueIndex()));
|
||||
return predicate.apply(lookupName(getDimensionValueIndex())).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.primitives.Ints;
|
||||
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
|
@ -31,7 +30,9 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -422,9 +423,7 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer<int[]
|
|||
{
|
||||
final BitSet checkedIds = new BitSet(maxId);
|
||||
final BitSet matchingIds = new BitSet(maxId);
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final boolean predicateMatchesNull = predicate.apply(null);
|
||||
final int nullValueId = lookupId(null);
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
|
@ -432,30 +431,25 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer<int[]
|
|||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
Object[] dims = currEntry.get().getDims();
|
||||
if (dimIndex >= dims.length) {
|
||||
return matchNull || predicateMatchesNull;
|
||||
return predicate.apply(null).matches(includeUnknown);
|
||||
}
|
||||
|
||||
int[] dimsInt = (int[]) dims[dimIndex];
|
||||
if (dimsInt == null || dimsInt.length == 0) {
|
||||
return matchNull || predicateMatchesNull;
|
||||
return predicate.apply(null).matches(includeUnknown);
|
||||
}
|
||||
|
||||
for (int id : dimsInt) {
|
||||
if (includeUnknown && id == nullValueId) {
|
||||
checkedIds.set(id);
|
||||
return true;
|
||||
}
|
||||
if (checkedIds.get(id)) {
|
||||
if (matchingIds.get(id)) {
|
||||
return true;
|
||||
}
|
||||
} else {
|
||||
final boolean matches = predicate.apply(lookupName(id));
|
||||
final DruidPredicateMatch matches = predicate.apply(lookupName(id));
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
if (matches.matches(includeUnknown)) {
|
||||
matchingIds.set(id);
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.apache.druid.segment.column;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -331,7 +331,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
{
|
||||
final BitSet checkedIds = new BitSet(getCardinality());
|
||||
final BitSet matchingIds = new BitSet(getCardinality());
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
|
@ -344,9 +344,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
if (checkedIds.get(id)) {
|
||||
return matchingIds.get(id);
|
||||
} else {
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
final String rowValue = lookupName(id);
|
||||
final boolean matches = (matchNull && rowValue == null) || predicate.apply(rowValue);
|
||||
final boolean matches = predicate.apply(rowValue).matches(includeUnknown);
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.primitives.Doubles;
|
||||
|
@ -33,7 +32,9 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -45,8 +46,6 @@ import org.apache.druid.segment.ColumnProcessors;
|
|||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes;
|
||||
|
@ -82,9 +81,8 @@ public class BoundFilter implements Filter
|
|||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension());
|
||||
if (indexSupplier == null) {
|
||||
// missing column -> match all rows if the predicate matches null; match no rows otherwise
|
||||
return getPredicateFactory().makeStringPredicate().apply(null)
|
||||
? new AllTrueBitmapColumnIndex(selector)
|
||||
: new AllUnknownBitmapColumnIndex(selector);
|
||||
final DruidPredicateMatch match = getPredicateFactory().makeStringPredicate().apply(null);
|
||||
return Filters.makeMissingColumnNullIndex(match, selector);
|
||||
}
|
||||
|
||||
if (supportStringShortCircuit()) {
|
||||
|
@ -327,7 +325,6 @@ public class BoundFilter implements Filter
|
|||
private final Supplier<DruidLongPredicate> longPredicateSupplier;
|
||||
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
|
||||
private final Supplier<DruidDoublePredicate> doublePredicateSupplier;
|
||||
private final boolean isNullUnknown;
|
||||
|
||||
BoundDimFilterDruidPredicateFactory(ExtractionFn extractionFn, BoundDimFilter boundDimFilter)
|
||||
{
|
||||
|
@ -336,20 +333,15 @@ public class BoundFilter implements Filter
|
|||
this.longPredicateSupplier = boundDimFilter.getLongPredicateSupplier();
|
||||
this.floatPredicateSupplier = boundDimFilter.getFloatPredicateSupplier();
|
||||
this.doublePredicateSupplier = boundDimFilter.getDoublePredicateSupplier();
|
||||
if (extractionFn != null) {
|
||||
this.isNullUnknown = !doesMatch(extractionFn.apply(null), boundDimFilter);
|
||||
} else {
|
||||
this.isNullUnknown = !doesMatch(null, boundDimFilter);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
return input -> doesMatch(extractionFn.apply(input), boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(extractionFn.apply(input), boundDimFilter));
|
||||
}
|
||||
return input -> doesMatch(input, boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(input, boundDimFilter));
|
||||
|
||||
}
|
||||
|
||||
|
@ -357,42 +349,36 @@ public class BoundFilter implements Filter
|
|||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
return input -> doesMatch(extractionFn.apply(input), boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(extractionFn.apply(input), boundDimFilter));
|
||||
}
|
||||
if (boundDimFilter.getOrdering().equals(StringComparators.NUMERIC)) {
|
||||
return longPredicateSupplier.get();
|
||||
}
|
||||
return input -> doesMatch(String.valueOf(input), boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(String.valueOf(input), boundDimFilter));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
return input -> doesMatch(extractionFn.apply(input), boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(extractionFn.apply(input), boundDimFilter));
|
||||
}
|
||||
if (boundDimFilter.getOrdering().equals(StringComparators.NUMERIC)) {
|
||||
return floatPredicateSupplier.get();
|
||||
}
|
||||
return input -> doesMatch(String.valueOf(input), boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(String.valueOf(input), boundDimFilter));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
return input -> doesMatch(extractionFn.apply(input), boundDimFilter);
|
||||
return input -> DruidPredicateMatch.of(doesMatch(extractionFn.apply(input), boundDimFilter));
|
||||
}
|
||||
if (boundDimFilter.getOrdering().equals(StringComparators.NUMERIC)) {
|
||||
return doublePredicateSupplier.get();
|
||||
}
|
||||
return input -> doesMatch(String.valueOf(input), boundDimFilter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullUnknown;
|
||||
return input -> DruidPredicateMatch.of(doesMatch(String.valueOf(input), boundDimFilter));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.segment.filter;
|
|||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
|
@ -29,7 +28,9 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -136,21 +137,19 @@ public class DimensionPredicateFilter implements Filter
|
|||
@VisibleForTesting
|
||||
static class DelegatingStringPredicateFactory implements DruidPredicateFactory
|
||||
{
|
||||
private final Predicate<String> baseStringPredicate;
|
||||
private final DruidObjectPredicate<String> baseStringPredicate;
|
||||
private final DruidPredicateFactory predicateFactory;
|
||||
private final ExtractionFn extractionFn;
|
||||
private final boolean isNullUnknown;
|
||||
|
||||
DelegatingStringPredicateFactory(DruidPredicateFactory predicateFactory, ExtractionFn extractionFn)
|
||||
{
|
||||
this.predicateFactory = predicateFactory;
|
||||
this.baseStringPredicate = predicateFactory.makeStringPredicate();
|
||||
this.extractionFn = extractionFn;
|
||||
this.isNullUnknown = !baseStringPredicate.apply(extractionFn.apply(null));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return input -> baseStringPredicate.apply(extractionFn.apply(input));
|
||||
}
|
||||
|
@ -161,13 +160,13 @@ public class DimensionPredicateFilter implements Filter
|
|||
return new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long input)
|
||||
public DruidPredicateMatch applyLong(long input)
|
||||
{
|
||||
return baseStringPredicate.apply(extractionFn.apply(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return baseStringPredicate.apply(extractionFn.apply(null));
|
||||
}
|
||||
|
@ -180,13 +179,13 @@ public class DimensionPredicateFilter implements Filter
|
|||
return new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyFloat(float input)
|
||||
public DruidPredicateMatch applyFloat(float input)
|
||||
{
|
||||
return baseStringPredicate.apply(extractionFn.apply(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return baseStringPredicate.apply(extractionFn.apply(null));
|
||||
}
|
||||
|
@ -199,25 +198,19 @@ public class DimensionPredicateFilter implements Filter
|
|||
return new DruidDoublePredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyDouble(double input)
|
||||
public DruidPredicateMatch applyDouble(double input)
|
||||
{
|
||||
return baseStringPredicate.apply(extractionFn.apply(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return baseStringPredicate.apply(extractionFn.apply(null));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullUnknown;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
@ -34,7 +33,9 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -49,6 +50,9 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
|||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.TypeSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.virtual.ExpressionSelectors;
|
||||
|
@ -93,7 +97,7 @@ public class ExpressionFilter implements Filter
|
|||
// evaluate the expression, just in case it does actually match nulls
|
||||
final ExprEval<?> constantEval = theExpr.eval(InputBindings.nilBindings());
|
||||
final ConstantMatcherType constantMatcherType;
|
||||
if (constantEval.value() == null) {
|
||||
if (constantEval.valueOrDefault() == null) {
|
||||
constantMatcherType = ConstantMatcherType.ALL_UNKNOWN;
|
||||
} else {
|
||||
constantMatcherType = constantEval.asBoolean() ? ConstantMatcherType.ALL_TRUE : ConstantMatcherType.ALL_FALSE;
|
||||
|
@ -203,10 +207,14 @@ public class ExpressionFilter implements Filter
|
|||
final Expr.BindingAnalysis details = bindingDetails.get();
|
||||
if (details.getRequiredBindings().isEmpty()) {
|
||||
// Constant expression.
|
||||
return Filters.makeMissingColumnNullIndex(
|
||||
expr.get().eval(InputBindings.nilBindings()).asBoolean(),
|
||||
selector
|
||||
);
|
||||
final ExprEval<?> eval = expr.get().eval(InputBindings.nilBindings());
|
||||
if (eval.valueOrDefault() == null) {
|
||||
return new AllUnknownBitmapColumnIndex(selector);
|
||||
}
|
||||
if (eval.asBoolean()) {
|
||||
return new AllTrueBitmapColumnIndex(selector);
|
||||
}
|
||||
return new AllFalseBitmapColumnIndex(selector.getBitmapFactory());
|
||||
} else if (details.getRequiredBindings().size() == 1) {
|
||||
// Single-column expression. We can use bitmap indexes if this column has an index and the expression can
|
||||
// map over the values of the index.
|
||||
|
@ -280,37 +288,46 @@ public class ExpressionFilter implements Filter
|
|||
*/
|
||||
private DruidPredicateFactory getPredicateFactory()
|
||||
{
|
||||
final boolean isNullUnknown = expr.get().eval(InputBindings.nilBindings()).value() == null;
|
||||
return new DruidPredicateFactory()
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return Evals::asBoolean;
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(Evals.asBoolean(input));
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return Evals::asBoolean;
|
||||
return input -> DruidPredicateMatch.of(Evals.asBoolean(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return Evals::asBoolean;
|
||||
return input -> DruidPredicateMatch.of(Evals.asBoolean(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return Evals::asBoolean;
|
||||
return input -> DruidPredicateMatch.of(Evals.asBoolean(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
public DruidObjectPredicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return Evals::objectAsBoolean;
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(Evals.objectAsBoolean(input));
|
||||
};
|
||||
}
|
||||
|
||||
// The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
|
||||
|
@ -329,12 +346,6 @@ public class ExpressionFilter implements Filter
|
|||
{
|
||||
return super.equals(obj);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullUnknown;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -344,18 +355,23 @@ public class ExpressionFilter implements Filter
|
|||
*/
|
||||
private DruidPredicateFactory getBitmapPredicateFactory(@Nullable ColumnCapabilities inputCapabilites)
|
||||
{
|
||||
final boolean isNullUnknown = expr.get().eval(InputBindings.nilBindings()).value() == null;
|
||||
return new DruidPredicateFactory()
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return value -> expr.get().eval(
|
||||
InputBindings.forInputSupplier(
|
||||
ExpressionType.STRING,
|
||||
() -> NullHandling.nullToEmptyIfNeeded(value)
|
||||
)
|
||||
).asBoolean();
|
||||
return value -> {
|
||||
final ExprEval<?> eval = expr.get().eval(
|
||||
InputBindings.forInputSupplier(
|
||||
ExpressionType.STRING,
|
||||
() -> NullHandling.nullToEmptyIfNeeded(value)
|
||||
)
|
||||
);
|
||||
if (eval.value() == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -364,15 +380,23 @@ public class ExpressionFilter implements Filter
|
|||
return new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long input)
|
||||
public DruidPredicateMatch applyLong(long input)
|
||||
{
|
||||
return expr.get().eval(InputBindings.forInputSupplier(ExpressionType.LONG, () -> input)).asBoolean();
|
||||
final ExprEval<?> eval = expr.get().eval(InputBindings.forInputSupplier(ExpressionType.LONG, () -> input));
|
||||
if (eval.isNumericNull()) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return expr.get().eval(InputBindings.nilBindings()).asBoolean();
|
||||
final ExprEval<?> eval = expr.get().eval(InputBindings.nilBindings());
|
||||
if (eval.isNumericNull()) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -383,17 +407,26 @@ public class ExpressionFilter implements Filter
|
|||
return new DruidFloatPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyFloat(float input)
|
||||
public DruidPredicateMatch applyFloat(float input)
|
||||
{
|
||||
return expr.get().eval(
|
||||
final ExprEval<?> eval = expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.DOUBLE, () -> input)
|
||||
).asBoolean();
|
||||
);
|
||||
if (eval.isNumericNull()) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return expr.get().eval(InputBindings.nilBindings()).asBoolean();
|
||||
|
||||
final ExprEval<?> eval = expr.get().eval(InputBindings.nilBindings());
|
||||
if (eval.isNumericNull()) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -404,38 +437,53 @@ public class ExpressionFilter implements Filter
|
|||
return new DruidDoublePredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyDouble(double input)
|
||||
public DruidPredicateMatch applyDouble(double input)
|
||||
{
|
||||
return expr.get().eval(
|
||||
final ExprEval<?> eval = expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.DOUBLE, () -> input)
|
||||
).asBoolean();
|
||||
);
|
||||
if (eval.isNumericNull()) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean applyNull()
|
||||
public DruidPredicateMatch applyNull()
|
||||
{
|
||||
return expr.get().eval(InputBindings.nilBindings()).asBoolean();
|
||||
final ExprEval<?> eval = expr.get().eval(InputBindings.nilBindings());
|
||||
if (eval.isNumericNull()) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
public DruidObjectPredicate<Object[]> makeArrayPredicate(@Nullable TypeSignature<ValueType> arrayType)
|
||||
{
|
||||
if (inputCapabilites == null) {
|
||||
return input -> expr.get()
|
||||
.eval(InputBindings.forInputSupplier(ExpressionType.STRING_ARRAY, () -> input))
|
||||
.asBoolean();
|
||||
return input -> {
|
||||
final ExprEval<?> eval = expr.get()
|
||||
.eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.STRING_ARRAY, () -> input)
|
||||
);
|
||||
if (eval.value() == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
};
|
||||
}
|
||||
return input -> expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.fromColumnType(inputCapabilites), () -> input)
|
||||
).asBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullInputUnknown()
|
||||
{
|
||||
return isNullUnknown;
|
||||
return input -> {
|
||||
ExprEval<?> eval = expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.fromColumnType(inputCapabilites), () -> input)
|
||||
);
|
||||
if (eval.value() == null) {
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
return DruidPredicateMatch.of(eval.asBoolean());
|
||||
};
|
||||
}
|
||||
|
||||
// The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.query.filter.BooleanFilter;
|
|||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -39,6 +40,7 @@ import org.apache.druid.segment.column.ColumnIndexSupplier;
|
|||
import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException;
|
||||
import org.apache.druid.segment.filter.cnf.CalciteCnfHelper;
|
||||
import org.apache.druid.segment.filter.cnf.HiveCnfHelper;
|
||||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
@ -128,14 +130,22 @@ public class Filters
|
|||
return null;
|
||||
}
|
||||
// missing column -> match all rows if the predicate matches null; match no rows otherwise
|
||||
return predicateFactory.makeStringPredicate().apply(null)
|
||||
? new AllTrueBitmapColumnIndex(selector)
|
||||
: new AllUnknownBitmapColumnIndex(selector);
|
||||
final DruidPredicateMatch match = predicateFactory.makeStringPredicate().apply(null);
|
||||
return makeMissingColumnNullIndex(match, selector);
|
||||
}
|
||||
|
||||
public static BitmapColumnIndex makeMissingColumnNullIndex(boolean matchesNull, final ColumnIndexSelector selector)
|
||||
public static BitmapColumnIndex makeMissingColumnNullIndex(
|
||||
DruidPredicateMatch match,
|
||||
final ColumnIndexSelector selector
|
||||
)
|
||||
{
|
||||
return matchesNull ? new AllTrueBitmapColumnIndex(selector) : new AllUnknownBitmapColumnIndex(selector);
|
||||
if (match == DruidPredicateMatch.TRUE) {
|
||||
return new AllTrueBitmapColumnIndex(selector);
|
||||
}
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return new AllUnknownBitmapColumnIndex(selector);
|
||||
}
|
||||
return new AllFalseBitmapColumnIndex(selector.getBitmapFactory());
|
||||
}
|
||||
|
||||
public static ImmutableBitmap computeDefaultBitmapResults(Filter filter, ColumnIndexSelector selector)
|
||||
|
|
|
@ -76,7 +76,7 @@ public class LikeFilter implements Filter
|
|||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(dimension);
|
||||
if (indexSupplier == null) {
|
||||
// Treat this as a column full of nulls
|
||||
return likeMatcher.matches(null)
|
||||
return likeMatcher.matches(null).matches(false)
|
||||
? new AllTrueBitmapColumnIndex(selector)
|
||||
: new AllUnknownBitmapColumnIndex(selector);
|
||||
}
|
||||
|
|
|
@ -19,14 +19,15 @@
|
|||
|
||||
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.math.expr.ExprEval;
|
||||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
|
@ -95,30 +96,33 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
if (selector instanceof NilColumnValueSelector) {
|
||||
// Column does not exist, or is unfilterable. Treat it as all nulls.
|
||||
|
||||
final boolean matchesNull = predicateFactory.makeArrayPredicate(columnCapabilities).apply(null);
|
||||
if (matchesNull) {
|
||||
final DruidPredicateMatch match = predicateFactory.makeArrayPredicate(columnCapabilities).apply(null);
|
||||
if (match == DruidPredicateMatch.TRUE) {
|
||||
return ValueMatchers.allTrue();
|
||||
}
|
||||
return ValueMatchers.makeAlwaysFalseObjectMatcher(selector);
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ValueMatchers.makeAlwaysFalseWithNullUnknownObjectMatcher(selector);
|
||||
}
|
||||
// predicate matches null as false, there are no unknowns
|
||||
return ValueMatchers.allFalse();
|
||||
} else {
|
||||
// use the object predicate
|
||||
final Predicate<Object[]> predicate = predicateFactory.makeArrayPredicate(columnCapabilities);
|
||||
// use the array predicate
|
||||
final DruidObjectPredicate<Object[]> predicate = predicateFactory.makeArrayPredicate(columnCapabilities);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
Object o = selector.getObject();
|
||||
if (o == null || o instanceof Object[]) {
|
||||
return (matchNull && o == null) || predicate.apply((Object[]) o);
|
||||
return predicate.apply((Object[]) o).matches(includeUnknown);
|
||||
}
|
||||
if (o instanceof List) {
|
||||
ExprEval<?> oEval = ExprEval.bestEffortArray((List<?>) o);
|
||||
return predicate.apply(oEval.asArray());
|
||||
return predicate.apply(oEval.asArray()).matches(includeUnknown);
|
||||
}
|
||||
// upcast non-array to a single element array to behave consistently with expressions.. idk if this is cool
|
||||
return predicate.apply(new Object[]{o});
|
||||
return predicate.apply(new Object[]{o}).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -136,22 +140,25 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
{
|
||||
if (selector instanceof NilColumnValueSelector) {
|
||||
// Column does not exist, or is unfilterable. Treat it as all nulls.
|
||||
final boolean predicateMatches = predicateFactory.makeStringPredicate().apply(null);
|
||||
if (predicateMatches) {
|
||||
final DruidPredicateMatch match = predicateFactory.makeStringPredicate().apply(null);
|
||||
if (match == DruidPredicateMatch.TRUE) {
|
||||
return ValueMatchers.allTrue();
|
||||
}
|
||||
return ValueMatchers.makeAlwaysFalseObjectMatcher(selector);
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ValueMatchers.makeAlwaysFalseWithNullUnknownObjectMatcher(selector);
|
||||
}
|
||||
// predicate matches null as false, there are no unknowns
|
||||
return ValueMatchers.allFalse();
|
||||
} else if (!isNumberOrString(selector.classOfObject())) {
|
||||
// if column is definitely not a number of string, use the object predicate
|
||||
final Predicate<Object> predicate = predicateFactory.makeObjectPredicate();
|
||||
final DruidObjectPredicate<Object> predicate = predicateFactory.makeObjectPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
final Object val = selector.getObject();
|
||||
return (matchNull && val == null) || predicate.apply(val);
|
||||
return predicate.apply(val).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -167,31 +174,30 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
private Predicate<String> stringPredicate;
|
||||
private DruidObjectPredicate<String> stringPredicate;
|
||||
private DruidLongPredicate longPredicate;
|
||||
private DruidFloatPredicate floatPredicate;
|
||||
private DruidDoublePredicate doublePredicate;
|
||||
private Predicate<Object[]> arrayPredicate;
|
||||
private DruidObjectPredicate<Object[]> arrayPredicate;
|
||||
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final Object rowValue = selector.getObject();
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
|
||||
if (rowValue == null) {
|
||||
return matchNull || getStringPredicate().apply(null);
|
||||
return getStringPredicate().apply(null).matches(includeUnknown);
|
||||
} else if (rowValue instanceof Integer) {
|
||||
return getLongPredicate().applyLong((int) rowValue);
|
||||
return getLongPredicate().applyLong((int) rowValue).matches(includeUnknown);
|
||||
} else if (rowValue instanceof Long) {
|
||||
return getLongPredicate().applyLong((long) rowValue);
|
||||
return getLongPredicate().applyLong((long) rowValue).matches(includeUnknown);
|
||||
} else if (rowValue instanceof Float) {
|
||||
return getFloatPredicate().applyFloat((float) rowValue);
|
||||
return getFloatPredicate().applyFloat((float) rowValue).matches(includeUnknown);
|
||||
} else if (rowValue instanceof Number) {
|
||||
// Double or some other non-int, non-long, non-float number.
|
||||
return getDoublePredicate().applyDouble(((Number) rowValue).doubleValue());
|
||||
return getDoublePredicate().applyDouble(((Number) rowValue).doubleValue()).matches(includeUnknown);
|
||||
} else if (rowValue instanceof Object[]) {
|
||||
return getArrayPredicate().apply((Object[]) rowValue);
|
||||
return getArrayPredicate().apply((Object[]) rowValue).matches(includeUnknown);
|
||||
} else {
|
||||
// Other types. Cast to list of strings and evaluate them as strings.
|
||||
// Boolean values are handled here as well since it is not a known type in Druid.
|
||||
|
@ -199,12 +205,12 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
|
||||
if (rowValueStrings.isEmpty()) {
|
||||
// Empty list is equivalent to null.
|
||||
return matchNull || getStringPredicate().apply(null);
|
||||
return getStringPredicate().apply(null).matches(includeUnknown);
|
||||
}
|
||||
|
||||
for (String rowValueString : rowValueStrings) {
|
||||
final String coerced = NullHandling.emptyToNullIfNeeded(rowValueString);
|
||||
if ((matchNull && coerced == null) || getStringPredicate().apply(coerced)) {
|
||||
if (getStringPredicate().apply(coerced).matches(includeUnknown)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -220,7 +226,7 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
inspector.visit("factory", predicateFactory);
|
||||
}
|
||||
|
||||
private Predicate<String> getStringPredicate()
|
||||
private DruidObjectPredicate<String> getStringPredicate()
|
||||
{
|
||||
if (stringPredicate == null) {
|
||||
stringPredicate = predicateFactory.makeStringPredicate();
|
||||
|
@ -256,7 +262,7 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory<Valu
|
|||
return doublePredicate;
|
||||
}
|
||||
|
||||
private Predicate<Object[]> getArrayPredicate()
|
||||
private DruidObjectPredicate<Object[]> getArrayPredicate()
|
||||
{
|
||||
if (arrayPredicate == null) {
|
||||
arrayPredicate = predicateFactory.makeArrayPredicate(null);
|
||||
|
|
|
@ -20,13 +20,14 @@
|
|||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
|
||||
|
@ -67,27 +68,27 @@ public class RegexFilter extends DimensionPredicateFilter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return input -> (input != null) && pattern.matcher(input).find();
|
||||
return input -> input == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.of(pattern.matcher(input).find());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return input -> pattern.matcher(String.valueOf(input)).find();
|
||||
return input -> DruidPredicateMatch.of(pattern.matcher(String.valueOf(input)).find());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return input -> pattern.matcher(String.valueOf(input)).find();
|
||||
return input -> DruidPredicateMatch.of(pattern.matcher(String.valueOf(input)).find());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return input -> pattern.matcher(String.valueOf(input)).find();
|
||||
return input -> DruidPredicateMatch.of(pattern.matcher(String.valueOf(input)).find());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,13 +22,14 @@ package org.apache.druid.segment.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.search.SearchQuerySpec;
|
||||
|
@ -128,27 +129,27 @@ public class SearchQueryFilter extends DimensionPredicateFilter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return input -> query.accept(input);
|
||||
return input -> input == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.of(query.accept(input));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return input -> query.accept(String.valueOf(input));
|
||||
return input -> DruidPredicateMatch.of(query.accept(String.valueOf(input)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return input -> query.accept(String.valueOf(input));
|
||||
return input -> DruidPredicateMatch.of(query.accept(String.valueOf(input)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return input -> query.accept(String.valueOf(input));
|
||||
return input -> DruidPredicateMatch.of(query.accept(String.valueOf(input)));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnProcessors;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.semantic.NullValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.StringValueSetIndexes;
|
||||
|
@ -86,7 +88,10 @@ public class SelectorFilter implements Filter
|
|||
final boolean isNull = NullHandling.isNullOrEquivalent(value);
|
||||
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(dimension);
|
||||
if (indexSupplier == null) {
|
||||
return Filters.makeMissingColumnNullIndex(isNull, selector);
|
||||
if (isNull) {
|
||||
return new AllTrueBitmapColumnIndex(selector);
|
||||
}
|
||||
return new AllUnknownBitmapColumnIndex(selector);
|
||||
}
|
||||
if (isNull) {
|
||||
final NullValueIndex nullValueIndex = indexSupplier.as(NullValueIndex.class);
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.segment.filter;
|
|||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.spatial.search.Bound;
|
||||
|
@ -30,7 +29,9 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -147,14 +148,14 @@ public class SpatialFilter implements Filter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return input -> {
|
||||
if (input == null) {
|
||||
return false;
|
||||
return DruidPredicateMatch.UNKNOWN;
|
||||
}
|
||||
final float[] coordinate = SpatialDimensionRowTransformer.decode(input);
|
||||
return bound.contains(coordinate);
|
||||
return DruidPredicateMatch.of(bound.contains(coordinate));
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -162,21 +163,21 @@ public class SpatialFilter implements Filter
|
|||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
// SpatialFilter does not currently support longs
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
// SpatialFilter does not currently support floats
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
return DruidFloatPredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
// SpatialFilter does not currently support doubles
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE_WITH_NULL_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,12 +19,13 @@
|
|||
|
||||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
|
@ -39,7 +40,6 @@ import org.apache.druid.segment.IdLookup;
|
|||
import org.apache.druid.segment.data.IndexedInts;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link StringConstantValueMatcherFactory}
|
||||
|
@ -90,10 +90,11 @@ public class ValueMatchers
|
|||
final boolean hasMultipleValues
|
||||
)
|
||||
{
|
||||
final String constant = NullHandling.emptyToNullIfNeeded(value);
|
||||
final ConstantMatcherType matcherType = toConstantMatcherTypeIfPossible(
|
||||
selector,
|
||||
hasMultipleValues,
|
||||
s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
|
||||
constant == null ? DruidObjectPredicate.isNull() : DruidObjectPredicate.equalTo(constant)
|
||||
);
|
||||
if (matcherType != null) {
|
||||
return matcherType.asValueMatcher();
|
||||
|
@ -115,7 +116,7 @@ public class ValueMatchers
|
|||
final boolean hasMultipleValues
|
||||
)
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final ConstantMatcherType constantMatcherType = toConstantMatcherTypeIfPossible(
|
||||
selector,
|
||||
hasMultipleValues,
|
||||
|
@ -197,11 +198,10 @@ public class ValueMatchers
|
|||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
if (selector.isNull()) {
|
||||
return matchNull || predicate.applyNull();
|
||||
return predicate.applyNull().matches(includeUnknown);
|
||||
}
|
||||
return predicate.applyFloat(selector.getFloat());
|
||||
return predicate.applyFloat(selector.getFloat()).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -272,11 +272,10 @@ public class ValueMatchers
|
|||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
if (selector.isNull()) {
|
||||
return matchNull || predicate.applyNull();
|
||||
return predicate.applyNull().matches(includeUnknown);
|
||||
}
|
||||
return predicate.applyLong(selector.getLong());
|
||||
return predicate.applyLong(selector.getLong()).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -357,11 +356,10 @@ public class ValueMatchers
|
|||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
if (selector.isNull()) {
|
||||
return matchNull || predicate.applyNull();
|
||||
return predicate.applyNull().matches(includeUnknown);
|
||||
}
|
||||
return predicate.applyDouble(selector.getDouble());
|
||||
return predicate.applyDouble(selector.getDouble()).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -378,7 +376,7 @@ public class ValueMatchers
|
|||
* null values will be matched. This is typically used when the filter should never match any actual values, but
|
||||
* still needs to be able to report 'unknown' matches.
|
||||
*/
|
||||
public static ValueMatcher makeAlwaysFalseDimensionMatcher(final DimensionSelector selector, boolean multiValue)
|
||||
public static ValueMatcher makeAlwaysFalseWithNullUnknownDimensionMatcher(final DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
final IdLookup lookup = selector.idLookup();
|
||||
// if the column doesn't have null
|
||||
|
@ -466,7 +464,7 @@ public class ValueMatchers
|
|||
* null values will be matched. This is typically used when the filter should never match any actual values, but
|
||||
* still needs to be able to report 'unknown' matches.
|
||||
*/
|
||||
public static ValueMatcher makeAlwaysFalseNumericMatcher(BaseNullableColumnValueSelector selector)
|
||||
public static ValueMatcher makeAlwaysFalseWithNullUnknownNumericMatcher(BaseNullableColumnValueSelector selector)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
|
@ -489,7 +487,7 @@ public class ValueMatchers
|
|||
* null values will be matched. This is typically used when the filter should never match any actual values, but
|
||||
* still needs to be able to report 'unknown' matches.
|
||||
*/
|
||||
public static ValueMatcher makeAlwaysFalseObjectMatcher(BaseObjectColumnValueSelector<?> selector)
|
||||
public static ValueMatcher makeAlwaysFalseWithNullUnknownObjectMatcher(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
|
@ -522,23 +520,28 @@ public class ValueMatchers
|
|||
public static ConstantMatcherType toConstantMatcherTypeIfPossible(
|
||||
final DimensionDictionarySelector selector,
|
||||
final boolean hasMultipleValues,
|
||||
final Predicate<String> predicate
|
||||
final DruidObjectPredicate<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.
|
||||
if (predicate.apply(null)) {
|
||||
final DruidPredicateMatch match = predicate.apply(null);
|
||||
if (match.matches(false)) {
|
||||
return ConstantMatcherType.ALL_TRUE;
|
||||
}
|
||||
return ConstantMatcherType.ALL_UNKNOWN;
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ConstantMatcherType.ALL_UNKNOWN;
|
||||
}
|
||||
return ConstantMatcherType.ALL_FALSE;
|
||||
} else if (!hasMultipleValues && selector.getValueCardinality() == 1 && selector.nameLookupPossibleInAdvance()) {
|
||||
// Every row has the same value. Match if and only if "predicate" matches the possible value.
|
||||
final String constant = selector.lookupName(0);
|
||||
if (predicate.apply(constant)) {
|
||||
final DruidPredicateMatch match = predicate.apply(constant);
|
||||
if (match == DruidPredicateMatch.TRUE) {
|
||||
return ConstantMatcherType.ALL_TRUE;
|
||||
}
|
||||
if (constant == null) {
|
||||
if (match == DruidPredicateMatch.UNKNOWN) {
|
||||
return ConstantMatcherType.ALL_UNKNOWN;
|
||||
}
|
||||
return ConstantMatcherType.ALL_FALSE;
|
||||
|
|
|
@ -19,10 +19,9 @@
|
|||
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
|
@ -64,14 +63,15 @@ public final class IndexedStringDruidPredicateIndexes<TDictionary extends Indexe
|
|||
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, dictionary.size())) {
|
||||
return null;
|
||||
}
|
||||
final DruidObjectPredicate<String> stringPredicate = matcherFactory.makeStringPredicate();
|
||||
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
final Predicate<String> stringPredicate = matcherFactory.makeStringPredicate();
|
||||
final Iterator<String> iterator = dictionary.iterator();
|
||||
@Nullable
|
||||
String next = null;
|
||||
|
@ -109,7 +109,7 @@ public final class IndexedStringDruidPredicateIndexes<TDictionary extends Indexe
|
|||
{
|
||||
while (!nextSet && iterator.hasNext()) {
|
||||
String nextValue = iterator.next();
|
||||
nextSet = stringPredicate.apply(nextValue);
|
||||
nextSet = stringPredicate.apply(nextValue).matches(includeUnknown);
|
||||
if (nextSet) {
|
||||
next = nextValue;
|
||||
}
|
||||
|
@ -117,16 +117,6 @@ public final class IndexedStringDruidPredicateIndexes<TDictionary extends Indexe
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown() && NullHandling.isNullOrEquivalent(dictionary.get(0))) {
|
||||
return bitmaps.get(0);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.segment.index;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import it.unimi.dsi.fastutil.ints.IntIntImmutablePair;
|
||||
import it.unimi.dsi.fastutil.ints.IntIntPair;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
@ -28,6 +27,8 @@ import org.apache.druid.collections.bitmap.BitmapFactory;
|
|||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.segment.IntListUtils;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
|
@ -82,7 +83,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
|
|||
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) {
|
||||
return null;
|
||||
}
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -126,7 +127,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
|
|||
boolean startStrict,
|
||||
@Nullable String endValue,
|
||||
boolean endStrict,
|
||||
Predicate<String> matcher
|
||||
DruidObjectPredicate<String> matcher
|
||||
)
|
||||
{
|
||||
final IntIntPair range = getRange(startValue, startStrict, endValue, endStrict);
|
||||
|
@ -134,7 +135,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
|
|||
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) {
|
||||
return null;
|
||||
}
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -150,7 +151,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
|
|||
|
||||
private int findNext()
|
||||
{
|
||||
while (currIndex < end && !applyMatcher(dictionary.get(currIndex))) {
|
||||
while (currIndex < end && !applyMatcher(dictionary.get(currIndex)).matches(false)) {
|
||||
currIndex++;
|
||||
}
|
||||
|
||||
|
@ -192,7 +193,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
|
|||
return null;
|
||||
}
|
||||
|
||||
private boolean applyMatcher(@Nullable final ByteBuffer valueUtf8)
|
||||
private DruidPredicateMatch applyMatcher(@Nullable final ByteBuffer valueUtf8)
|
||||
{
|
||||
if (valueUtf8 == null) {
|
||||
return matcher.apply(null);
|
||||
|
|
|
@ -163,7 +163,7 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
|
|||
{
|
||||
// for large number of in-filter values in comparison to the dictionary size, use the sorted merge algorithm.
|
||||
if (size > SORTED_MERGE_RATIO_THRESHOLD * dictionary.size()) {
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -240,9 +240,9 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
|
|||
* Iterates over the value set, using binary search to look up each element. The algorithm works well for smaller
|
||||
* number of values, and must be used if the values are not sorted in the same manner as {@link #dictionary}
|
||||
*/
|
||||
private SimpleImmutableBitmapIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable<ByteBuffer> valuesUtf8, boolean valuesContainsNull)
|
||||
private SimpleImmutableBitmapDelegatingIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable<ByteBuffer> valuesUtf8, boolean valuesContainsNull)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.index;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
|
||||
public abstract class SimpleImmutableBitmapDelegatingIterableIndex extends SimpleBitmapColumnIndex
|
||||
{
|
||||
@Override
|
||||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
|
||||
{
|
||||
if (includeUnknown) {
|
||||
final ImmutableBitmap unknownsBitmap = getUnknownsBitmap();
|
||||
if (unknownsBitmap != null) {
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(
|
||||
Iterables.concat(
|
||||
getBitmapIterable(),
|
||||
Collections.singletonList(unknownsBitmap)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable());
|
||||
}
|
||||
|
||||
protected abstract Iterable<ImmutableBitmap> getBitmapIterable();
|
||||
|
||||
@Nullable
|
||||
protected abstract ImmutableBitmap getUnknownsBitmap();
|
||||
}
|
|
@ -19,13 +19,9 @@
|
|||
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* {@link SimpleBitmapColumnIndex} for anything which can compute an {@link Iterable<ImmutableBitmap>} in some manner
|
||||
*/
|
||||
|
@ -35,22 +31,8 @@ public abstract class SimpleImmutableBitmapIterableIndex extends SimpleBitmapCol
|
|||
@Override
|
||||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
|
||||
{
|
||||
if (includeUnknown) {
|
||||
final ImmutableBitmap unknownsBitmap = getUnknownsBitmap();
|
||||
if (unknownsBitmap != null) {
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(
|
||||
Iterables.concat(
|
||||
getBitmapIterable(),
|
||||
Collections.singletonList(unknownsBitmap)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable());
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable(includeUnknown));
|
||||
}
|
||||
|
||||
protected abstract Iterable<ImmutableBitmap> getBitmapIterable();
|
||||
|
||||
@Nullable
|
||||
protected abstract ImmutableBitmap getUnknownsBitmap();
|
||||
protected abstract Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.apache.druid.segment.index.semantic;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -65,6 +65,6 @@ public interface LexicographicalRangeIndexes
|
|||
boolean startStrict,
|
||||
@Nullable String endValue,
|
||||
boolean endStrict,
|
||||
Predicate<String> matcher
|
||||
DruidObjectPredicate<String> matcher
|
||||
);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -44,6 +43,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.DruidDoublePredicate;
|
||||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.IntListUtils;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
|
@ -54,6 +54,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
|
@ -290,7 +291,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, size)) {
|
||||
return null;
|
||||
}
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -401,7 +402,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -489,7 +490,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
boolean startStrict,
|
||||
@Nullable String endValue,
|
||||
boolean endStrict,
|
||||
Predicate<String> matcher
|
||||
DruidObjectPredicate<String> matcher
|
||||
)
|
||||
{
|
||||
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
|
||||
|
@ -507,7 +508,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) {
|
||||
return null;
|
||||
}
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -523,7 +524,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
|
||||
private int findNext()
|
||||
{
|
||||
while (currIndex < end && !matcher.apply(StringUtils.fromUtf8Nullable(stringDictionary.get(localDictionary.get(currIndex))))) {
|
||||
while (currIndex < end && !matcher.apply(StringUtils.fromUtf8Nullable(stringDictionary.get(localDictionary.get(currIndex)))).matches(false)) {
|
||||
currIndex++;
|
||||
}
|
||||
|
||||
|
@ -581,12 +582,12 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
final Indexed<ByteBuffer> stringDictionary = globalStringDictionarySupplier.get();
|
||||
final Predicate<String> stringPredicate = matcherFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> stringPredicate = matcherFactory.makeStringPredicate();
|
||||
|
||||
// in the future, this could use an int iterator
|
||||
final Iterator<Integer> iterator = localDictionary.iterator();
|
||||
|
@ -620,7 +621,8 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
{
|
||||
while (!nextSet && iterator.hasNext()) {
|
||||
Integer nextValue = iterator.next();
|
||||
nextSet = stringPredicate.apply(StringUtils.fromUtf8Nullable(stringDictionary.get(nextValue)));
|
||||
nextSet = stringPredicate.apply(StringUtils.fromUtf8Nullable(stringDictionary.get(nextValue)))
|
||||
.matches(includeUnknown);
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
}
|
||||
|
@ -629,16 +631,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
|
||||
return bitmaps.get(0);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -689,7 +681,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -821,7 +813,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
|
@ -862,9 +854,9 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
while (!nextSet && iterator.hasNext()) {
|
||||
Integer nextValue = iterator.next();
|
||||
if (nextValue == 0) {
|
||||
nextSet = longPredicate.applyNull();
|
||||
nextSet = longPredicate.applyNull().matches(includeUnknown);
|
||||
} else {
|
||||
nextSet = longPredicate.applyLong(longDictionary.get(nextValue - adjustLongId));
|
||||
nextSet = longPredicate.applyLong(longDictionary.get(nextValue - adjustLongId)).matches(includeUnknown);
|
||||
}
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
|
@ -874,16 +866,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
|
||||
return bitmaps.get(0);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -933,7 +915,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -1049,7 +1031,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
|
@ -1090,9 +1072,10 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
while (!nextSet && iterator.hasNext()) {
|
||||
Integer nextValue = iterator.next();
|
||||
if (nextValue == 0) {
|
||||
nextSet = doublePredicate.applyNull();
|
||||
nextSet = doublePredicate.applyNull().matches(includeUnknown);
|
||||
} else {
|
||||
nextSet = doublePredicate.applyDouble(doubleDictionary.get(nextValue - adjustDoubleId));
|
||||
nextSet = doublePredicate.applyDouble(doubleDictionary.get(nextValue - adjustDoubleId))
|
||||
.matches(includeUnknown);
|
||||
}
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
|
@ -1102,16 +1085,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
|
||||
bitmaps.get(0);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -1172,7 +1145,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
@Override
|
||||
public BitmapColumnIndex forValue(@Nullable String value)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
protected Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -1209,7 +1182,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -1279,11 +1252,11 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
final Predicate<String> stringPredicate = matcherFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> stringPredicate = matcherFactory.makeStringPredicate();
|
||||
final DruidLongPredicate longPredicate = matcherFactory.makeLongPredicate();
|
||||
final DruidDoublePredicate doublePredicate = matcherFactory.makeDoublePredicate();
|
||||
|
||||
|
@ -1320,11 +1293,14 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
while (!nextSet && iterator.hasNext()) {
|
||||
Integer nextValue = iterator.next();
|
||||
if (nextValue >= adjustDoubleId) {
|
||||
nextSet = doublePredicate.applyDouble(doubleDictionary.get(nextValue - adjustDoubleId));
|
||||
nextSet = doublePredicate.applyDouble(doubleDictionary.get(nextValue - adjustDoubleId))
|
||||
.matches(includeUnknown);
|
||||
} else if (nextValue >= adjustLongId) {
|
||||
nextSet = longPredicate.applyLong(longDictionary.get(nextValue - adjustLongId));
|
||||
nextSet = longPredicate.applyLong(longDictionary.get(nextValue - adjustLongId))
|
||||
.matches(includeUnknown);
|
||||
} else {
|
||||
nextSet = stringPredicate.apply(StringUtils.fromUtf8Nullable(stringDictionary.get(nextValue)));
|
||||
nextSet = stringPredicate.apply(StringUtils.fromUtf8Nullable(stringDictionary.get(nextValue)))
|
||||
.matches(includeUnknown);
|
||||
}
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
|
@ -1334,16 +1310,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
|
||||
return bitmaps.get(0);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
|
||||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Floats;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
|
@ -30,6 +29,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -435,7 +435,7 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
|||
{
|
||||
final BitSet checkedIds = new BitSet(getCardinality());
|
||||
final BitSet matchingIds = new BitSet(getCardinality());
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
|
@ -444,13 +444,12 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
|||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final int id = getRowValue();
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
|
||||
if (checkedIds.get(id)) {
|
||||
return matchingIds.get(id);
|
||||
} else {
|
||||
final String rowVal = lookupName(id);
|
||||
final boolean matches = (matchNull && rowVal == null) || predicate.apply(rowVal);
|
||||
final boolean matches = predicate.apply(rowVal).matches(includeUnknown);
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.druid.segment.data.VByte;
|
|||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
|
@ -319,7 +320,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -426,7 +427,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, endIndex - startIndex)) {
|
||||
return null;
|
||||
}
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -472,7 +473,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
|
@ -511,12 +512,12 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
Double nextValue = iterator.next();
|
||||
if (nextValue == null) {
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
nextSet = doublePredicate.applyNull();
|
||||
nextSet = doublePredicate.applyNull().matches(includeUnknown);
|
||||
} else {
|
||||
nextSet = doublePredicate.applyDouble(NullHandling.defaultDoubleValue());
|
||||
nextSet = doublePredicate.applyDouble(NullHandling.defaultDoubleValue()).matches(includeUnknown);
|
||||
}
|
||||
} else {
|
||||
nextSet = doublePredicate.applyDouble(nextValue);
|
||||
nextSet = doublePredicate.applyDouble(nextValue).matches(includeUnknown);
|
||||
}
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
|
@ -526,16 +527,6 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown()) {
|
||||
return nullValueBitmap;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.druid.segment.data.VByte;
|
|||
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
|
@ -319,7 +320,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -436,7 +437,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, endIndex - startIndex)) {
|
||||
return null;
|
||||
}
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -482,7 +483,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
return new SimpleImmutableBitmapIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
|
@ -522,12 +523,12 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
Long nextValue = iterator.next();
|
||||
if (nextValue == null) {
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
nextSet = longPredicate.applyNull();
|
||||
nextSet = longPredicate.applyNull().matches(includeUnknown);
|
||||
} else {
|
||||
nextSet = longPredicate.applyLong(NullHandling.defaultLongValue());
|
||||
nextSet = longPredicate.applyLong(NullHandling.defaultLongValue()).matches(includeUnknown);
|
||||
}
|
||||
} else {
|
||||
nextSet = longPredicate.applyLong(nextValue);
|
||||
nextSet = longPredicate.applyLong(nextValue).matches(includeUnknown);
|
||||
}
|
||||
if (nextSet) {
|
||||
next = index;
|
||||
|
@ -537,16 +538,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
protected ImmutableBitmap getUnknownsBitmap()
|
||||
{
|
||||
if (matcherFactory.isNullInputUnknown()) {
|
||||
return nullValueBitmap;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Floats;
|
||||
import it.unimi.dsi.fastutil.ints.IntArraySet;
|
||||
|
@ -32,6 +31,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -518,7 +518,7 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
|||
{
|
||||
final BitSet checkedIds = new BitSet(getCardinality());
|
||||
final BitSet matchingIds = new BitSet(getCardinality());
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
|
@ -528,13 +528,10 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
|||
{
|
||||
final int id = getRowValue();
|
||||
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
|
||||
if (checkedIds.get(id)) {
|
||||
return matchingIds.get(id);
|
||||
} else {
|
||||
final String val = lookupName(id);
|
||||
final boolean matches = (matchNull && val == null) || predicate.apply(lookupName(id));
|
||||
final boolean matches = predicate.apply(lookupName(id)).matches(includeUnknown);
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
|
|
|
@ -19,11 +19,11 @@
|
|||
|
||||
package org.apache.druid.segment.virtual;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -156,14 +156,13 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
|
|||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
ExprEval evaluated = getEvaluated();
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
final DruidObjectPredicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
if (evaluated.isArray()) {
|
||||
List<String> array = getArrayAsList(evaluated);
|
||||
return array.stream().anyMatch(x -> (matchNull && x == null) || predicate.apply(x));
|
||||
return array.stream().anyMatch(x -> predicate.apply(x).matches(includeUnknown));
|
||||
}
|
||||
final String rowValue = getValue(evaluated);
|
||||
return (matchNull && rowValue == null) || predicate.apply(rowValue);
|
||||
return predicate.apply(rowValue).matches(includeUnknown);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.druid.segment.virtual;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
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.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -36,6 +34,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder;
|
|||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.dimension.ListFilteredDimensionSpec;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
|
@ -51,7 +50,7 @@ import org.apache.druid.segment.column.ColumnHolder;
|
|||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex;
|
||||
import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex;
|
||||
import org.apache.druid.segment.index.semantic.DruidPredicateIndexes;
|
||||
|
@ -265,8 +264,10 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
return false;
|
||||
}
|
||||
ListFilteredVirtualColumn that = (ListFilteredVirtualColumn) o;
|
||||
return allowList == that.allowList && name.equals(that.name) && delegate.equals(that.delegate) && values.equals(
|
||||
that.values);
|
||||
return allowList == that.allowList &&
|
||||
name.equals(that.name) &&
|
||||
delegate.equals(that.delegate) &&
|
||||
values.equals(that.values);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -292,15 +293,23 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
)
|
||||
{
|
||||
final int start = NullHandling.isNullOrEquivalent(delegateIndex.getValue(idMapping.getReverseId(0))) ? 1 : 0;
|
||||
return getBitmapsInRange(delegateIndex, idMapping, v -> true, start, idMapping.getValueCardinality());
|
||||
return getBitmapsInRange(
|
||||
delegateIndex,
|
||||
idMapping,
|
||||
DruidObjectPredicate.alwaysTrue(),
|
||||
start,
|
||||
idMapping.getValueCardinality(),
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
private static Iterable<ImmutableBitmap> getBitmapsInRange(
|
||||
DictionaryEncodedStringValueIndex delegate,
|
||||
IdMapping idMapping,
|
||||
Predicate<String> matcher,
|
||||
DruidObjectPredicate<String> matcher,
|
||||
int start,
|
||||
int end
|
||||
int end,
|
||||
boolean includeUnknown
|
||||
)
|
||||
{
|
||||
return () -> new Iterator<ImmutableBitmap>()
|
||||
|
@ -314,7 +323,7 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
|
||||
private int findNext()
|
||||
{
|
||||
while (currIndex < end && !matcher.apply(delegate.getValue(idMapping.getReverseId(currIndex)))) {
|
||||
while (currIndex < end && !matcher.apply(delegate.getValue(idMapping.getReverseId(currIndex))).matches(includeUnknown)) {
|
||||
currIndex++;
|
||||
}
|
||||
|
||||
|
@ -394,9 +403,9 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
return -(minIndex + 1);
|
||||
}
|
||||
|
||||
Iterable<ImmutableBitmap> getBitmapsInRange(Predicate<String> matcher, int start, int end)
|
||||
Iterable<ImmutableBitmap> getBitmapsInRange(DruidObjectPredicate<String> matcher, int start, int end, boolean includeUnknown)
|
||||
{
|
||||
return ListFilteredVirtualColumn.getBitmapsInRange(delegate, idMapping, matcher, start, end);
|
||||
return ListFilteredVirtualColumn.getBitmapsInRange(delegate, idMapping, matcher, start, end, includeUnknown);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -470,7 +479,7 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -544,7 +553,6 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
{
|
||||
return new SimpleBitmapColumnIndex()
|
||||
{
|
||||
|
||||
@Override
|
||||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
|
||||
{
|
||||
|
@ -552,11 +560,11 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
Iterable<ImmutableBitmap> bitmaps;
|
||||
if (includeUnknown) {
|
||||
bitmaps = Iterables.concat(
|
||||
getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end),
|
||||
getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end, includeUnknown),
|
||||
Collections.singletonList(nullValueBitmapSupplier.get())
|
||||
);
|
||||
} else {
|
||||
bitmaps = getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end);
|
||||
bitmaps = getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end, includeUnknown);
|
||||
}
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(bitmaps);
|
||||
}
|
||||
|
@ -588,7 +596,7 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
boolean endStrict
|
||||
)
|
||||
{
|
||||
return forRange(startValue, startStrict, endValue, endStrict, Predicates.alwaysTrue());
|
||||
return forRange(startValue, startStrict, endValue, endStrict, DruidObjectPredicate.alwaysTrue());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -598,10 +606,10 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
boolean startStrict,
|
||||
@Nullable String endValue,
|
||||
boolean endStrict,
|
||||
Predicate<String> matcher
|
||||
DruidObjectPredicate<String> matcher
|
||||
)
|
||||
{
|
||||
return new SimpleImmutableBitmapIterableIndex()
|
||||
return new SimpleImmutableBitmapDelegatingIterableIndex()
|
||||
{
|
||||
@Override
|
||||
public Iterable<ImmutableBitmap> getBitmapIterable()
|
||||
|
@ -632,7 +640,7 @@ public class ListFilteredVirtualColumn implements VirtualColumn
|
|||
|
||||
endIndex = Math.max(startIndex, endIndex);
|
||||
final int start = startIndex, end = endIndex;
|
||||
return getBitmapsInRange(matcher, start, end);
|
||||
return getBitmapsInRange(matcher, start, end, false);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.datasketches.memory.WritableMemory;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
@ -41,8 +42,6 @@ import org.mockito.junit.MockitoJUnit;
|
|||
import org.mockito.junit.MockitoRule;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class DoubleFieldReaderTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final long MEMORY_POSITION = 1;
|
||||
|
@ -135,12 +134,12 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
|
|||
Assert.assertTrue(readSelector.makeValueMatcher("0.0").matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
} else {
|
||||
Assert.assertFalse(readSelector.makeValueMatcher("0.0").matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.datasketches.memory.WritableMemory;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
@ -41,8 +42,6 @@ import org.mockito.junit.MockitoJUnit;
|
|||
import org.mockito.junit.MockitoRule;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class FloatFieldReaderTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final long MEMORY_POSITION = 1;
|
||||
|
@ -135,12 +134,12 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
|
|||
Assert.assertTrue(readSelector.makeValueMatcher("0.0").matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
} else {
|
||||
Assert.assertFalse(readSelector.makeValueMatcher("0.0").matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.datasketches.memory.WritableMemory;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
@ -41,8 +42,6 @@ import org.mockito.junit.MockitoJUnit;
|
|||
import org.mockito.junit.MockitoRule;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class LongFieldReaderTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final long MEMORY_POSITION = 1;
|
||||
|
@ -135,12 +134,12 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
|
|||
Assert.assertTrue(readSelector.makeValueMatcher("0").matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0")).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
} else {
|
||||
Assert.assertFalse(readSelector.makeValueMatcher("0").matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches(false));
|
||||
Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0")).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false));
|
||||
Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -124,9 +124,9 @@ public class JavaScriptDimFilterTest
|
|||
null,
|
||||
JavaScriptConfig.getEnabledInstance()
|
||||
);
|
||||
Assert.assertTrue(javaScriptDimFilter.getPredicateFactory().applyObject("123"));
|
||||
Assert.assertTrue(javaScriptDimFilter.getPredicateFactory().applyObject("456"));
|
||||
Assert.assertFalse(javaScriptDimFilter.getPredicateFactory().applyObject("789"));
|
||||
Assert.assertTrue(javaScriptDimFilter.getPredicateFactory().applyObject("123").matches(false));
|
||||
Assert.assertTrue(javaScriptDimFilter.getPredicateFactory().applyObject("456").matches(false));
|
||||
Assert.assertFalse(javaScriptDimFilter.getPredicateFactory().applyObject("789").matches(false));
|
||||
|
||||
// test for return java.lang.Boolean
|
||||
JavaScriptDimFilter javaScriptDimFilter1 = new JavaScriptDimFilter(
|
||||
|
@ -135,9 +135,9 @@ public class JavaScriptDimFilterTest
|
|||
null,
|
||||
JavaScriptConfig.getEnabledInstance()
|
||||
);
|
||||
Assert.assertTrue(javaScriptDimFilter1.getPredicateFactory().applyObject("123"));
|
||||
Assert.assertTrue(javaScriptDimFilter1.getPredicateFactory().applyObject("456"));
|
||||
Assert.assertFalse(javaScriptDimFilter1.getPredicateFactory().applyObject("789"));
|
||||
Assert.assertTrue(javaScriptDimFilter1.getPredicateFactory().applyObject("123").matches(false));
|
||||
Assert.assertTrue(javaScriptDimFilter1.getPredicateFactory().applyObject("456").matches(false));
|
||||
Assert.assertFalse(javaScriptDimFilter1.getPredicateFactory().applyObject("789").matches(false));
|
||||
|
||||
// test for return other type
|
||||
JavaScriptDimFilter javaScriptDimFilter2 = new JavaScriptDimFilter(
|
||||
|
@ -146,7 +146,7 @@ public class JavaScriptDimFilterTest
|
|||
null,
|
||||
JavaScriptConfig.getEnabledInstance()
|
||||
);
|
||||
Assert.assertTrue(javaScriptDimFilter2.getPredicateFactory().applyObject("123"));
|
||||
Assert.assertTrue(javaScriptDimFilter2.getPredicateFactory().applyObject("123").matches(false));
|
||||
|
||||
// test for return null
|
||||
JavaScriptDimFilter javaScriptDimFilter3 = new JavaScriptDimFilter(
|
||||
|
@ -155,6 +155,6 @@ public class JavaScriptDimFilterTest
|
|||
null,
|
||||
JavaScriptConfig.getEnabledInstance()
|
||||
);
|
||||
Assert.assertFalse(javaScriptDimFilter3.getPredicateFactory().applyObject("123"));
|
||||
Assert.assertFalse(javaScriptDimFilter3.getPredicateFactory().applyObject("123").matches(false));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@ public class SelectorPredicateFactoryTest
|
|||
public void testEqualsContract()
|
||||
{
|
||||
EqualsVerifier.forClass(SelectorPredicateFactory.class)
|
||||
.withIgnoredFields("initLock", "longPredicate", "floatPredicate", "doublePredicate", "isNullUnknown")
|
||||
.withIgnoredFields("initLock", "longPredicate", "floatPredicate", "doublePredicate")
|
||||
.usingGetClass()
|
||||
.verify();
|
||||
}
|
||||
|
|
|
@ -22,7 +22,9 @@ package org.apache.druid.segment;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.extraction.StringFormatExtractionFn;
|
||||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
|
@ -30,8 +32,6 @@ import org.apache.druid.testing.InitializedNullHandlingTest;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class ConstantDimensionSelectorTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private final DimensionSelector NULL_SELECTOR = DimensionSelector.constant(null);
|
||||
|
@ -98,12 +98,14 @@ public class ConstantDimensionSelectorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testValueMatcherPredicates()
|
||||
{
|
||||
DruidPredicateFactory nullUnkown = new StringPredicateDruidPredicateFactory(Objects::nonNull, true);
|
||||
DruidPredicateFactory nullUnkown = StringPredicateDruidPredicateFactory.of(
|
||||
value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.TRUE
|
||||
);
|
||||
ValueMatcher matcher = NULL_SELECTOR.makeValueMatcher(nullUnkown);
|
||||
Assert.assertFalse(matcher.matches(false));
|
||||
Assert.assertTrue(matcher.matches(true));
|
||||
|
||||
DruidPredicateFactory notUnknown = new StringPredicateDruidPredicateFactory(Objects::nonNull, false);
|
||||
DruidPredicateFactory notUnknown = StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.notNull());
|
||||
matcher = NULL_SELECTOR.makeValueMatcher(notUnknown);
|
||||
Assert.assertFalse(matcher.matches(false));
|
||||
Assert.assertFalse(matcher.matches(true));
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.query.extraction.StringFormatExtractionFn;
|
||||
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -138,7 +138,7 @@ public class ConstantMultiValueDimensionSelectorTest extends InitializedNullHand
|
|||
Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("doug").matches(false));
|
||||
Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches(false));
|
||||
|
||||
Assert.assertTrue(NULL_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.isNull(), false)).matches(false));
|
||||
Assert.assertTrue(NULL_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.of(DruidObjectPredicate.isNull())).matches(false));
|
||||
Assert.assertFalse(NULL_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("billy")).matches(false));
|
||||
|
||||
Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(null)).matches(false));
|
||||
|
|
|
@ -90,6 +90,8 @@ public class IndexBuilder
|
|||
@Nullable
|
||||
private File inputSourceTmpDir = null;
|
||||
|
||||
private boolean writeNullColumns = false;
|
||||
|
||||
private IndexBuilder(ObjectMapper jsonMapper, ColumnConfig columnConfig)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -131,12 +133,13 @@ public class IndexBuilder
|
|||
public IndexBuilder segmentWriteOutMediumFactory(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory)
|
||||
{
|
||||
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
|
||||
this.indexMerger = new IndexMergerV9(jsonMapper, indexIO, segmentWriteOutMediumFactory);
|
||||
this.indexMerger = new IndexMergerV9(jsonMapper, indexIO, segmentWriteOutMediumFactory, writeNullColumns);
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexBuilder writeNullColumns(boolean shouldWriteNullColumns)
|
||||
{
|
||||
this.writeNullColumns = shouldWriteNullColumns;
|
||||
this.indexMerger = new IndexMergerV9(jsonMapper, indexIO, segmentWriteOutMediumFactory, shouldWriteNullColumns);
|
||||
return this;
|
||||
}
|
||||
|
|
|
@ -331,6 +331,7 @@ public class ArrayContainsElementFilterTests
|
|||
public void testArrayLongColumnContainsArrays()
|
||||
{
|
||||
// only auto schema supports array columns... skip other segment types
|
||||
|
||||
Assume.assumeTrue(isAutoSchema());
|
||||
|
||||
// these are not nested arrays, expect no matches
|
||||
|
|
|
@ -143,6 +143,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
new ExpressionVirtualColumn("vd0-add-sub", "d0 + (d0 - d0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE),
|
||||
new ExpressionVirtualColumn("vf0-add-sub", "f0 + (f0 - f0)", ColumnType.FLOAT, TestExprMacroTable.INSTANCE),
|
||||
new ExpressionVirtualColumn("vl0-add-sub", "l0 + (l0 - l0)", ColumnType.LONG, TestExprMacroTable.INSTANCE),
|
||||
new ExpressionVirtualColumn("vdim3-concat", "dim3 + dim3", ColumnType.LONG, TestExprMacroTable.INSTANCE),
|
||||
new ExpressionVirtualColumn("nestedArrayLong", "array(arrayLong)", ColumnType.ofArray(ColumnType.LONG_ARRAY), TestExprMacroTable.INSTANCE),
|
||||
new ListFilteredVirtualColumn("allow-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), true),
|
||||
new ListFilteredVirtualColumn("deny-dim0", DefaultDimensionSpec.of("dim0"), ImmutableSet.of("3", "4"), false),
|
||||
|
@ -626,27 +627,31 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
finishers.entrySet()) {
|
||||
for (boolean cnf : ImmutableList.of(false, true)) {
|
||||
for (boolean optimize : ImmutableList.of(false, true)) {
|
||||
for (StringEncodingStrategy encodingStrategy : stringEncoding) {
|
||||
final String testName = StringUtils.format(
|
||||
"bitmaps[%s], indexMerger[%s], finisher[%s], cnf[%s], optimize[%s], stringDictionaryEncoding[%s]",
|
||||
bitmapSerdeFactoryEntry.getKey(),
|
||||
segmentWriteOutMediumFactoryEntry.getKey(),
|
||||
finisherEntry.getKey(),
|
||||
cnf,
|
||||
optimize,
|
||||
encodingStrategy.getType()
|
||||
);
|
||||
final IndexBuilder indexBuilder = IndexBuilder
|
||||
.create()
|
||||
.schema(DEFAULT_INDEX_SCHEMA)
|
||||
.indexSpec(
|
||||
IndexSpec.builder()
|
||||
.withBitmapSerdeFactory(bitmapSerdeFactoryEntry.getValue())
|
||||
.withStringDictionaryEncoding(encodingStrategy)
|
||||
.build()
|
||||
)
|
||||
.segmentWriteOutMediumFactory(segmentWriteOutMediumFactoryEntry.getValue());
|
||||
constructors.add(new Object[]{testName, indexBuilder, finisherEntry.getValue(), cnf, optimize});
|
||||
for (boolean storeNullColumns : ImmutableList.of(false, true)) {
|
||||
for (StringEncodingStrategy encodingStrategy : stringEncoding) {
|
||||
final String testName = StringUtils.format(
|
||||
"bitmaps[%s], indexMerger[%s], finisher[%s], cnf[%s], optimize[%s], stringDictionaryEncoding[%s], storeNullColumns[%s]",
|
||||
bitmapSerdeFactoryEntry.getKey(),
|
||||
segmentWriteOutMediumFactoryEntry.getKey(),
|
||||
finisherEntry.getKey(),
|
||||
cnf,
|
||||
optimize,
|
||||
encodingStrategy.getType(),
|
||||
storeNullColumns
|
||||
);
|
||||
final IndexBuilder indexBuilder = IndexBuilder
|
||||
.create()
|
||||
.schema(DEFAULT_INDEX_SCHEMA)
|
||||
.writeNullColumns(storeNullColumns)
|
||||
.indexSpec(
|
||||
IndexSpec.builder()
|
||||
.withBitmapSerdeFactory(bitmapSerdeFactoryEntry.getValue())
|
||||
.withStringDictionaryEncoding(encodingStrategy)
|
||||
.build()
|
||||
)
|
||||
.segmentWriteOutMediumFactory(segmentWriteOutMediumFactoryEntry.getValue());
|
||||
constructors.add(new Object[]{testName, indexBuilder, finisherEntry.getValue(), cnf, optimize});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1106,6 +1111,24 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
selectColumnValuesMatchingFilter(filter, "dim0")
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0")
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator: " + filter,
|
||||
expectedRows.size(),
|
||||
selectCountUsingFilteredAggregator(filter)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"RowBasedColumnSelectorFactory: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0")
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Cursor (vectorized): " + filter,
|
||||
|
@ -1118,40 +1141,17 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorVirtualColumnCursor(filter, "vdim0", "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0")
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering (vectorized): " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorizedPostFiltering(filter, "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator: " + filter,
|
||||
expectedRows.size(),
|
||||
selectCountUsingFilteredAggregator(filter)
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator (vectorized): " + filter,
|
||||
expectedRows.size(),
|
||||
selectCountUsingVectorizedFilteredAggregator(filter)
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"RowBasedColumnSelectorFactory: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -895,8 +895,7 @@ public class BoundFilterTest extends BaseFilterTest
|
|||
.withIgnoredFields(
|
||||
"longPredicateSupplier",
|
||||
"floatPredicateSupplier",
|
||||
"doublePredicateSupplier",
|
||||
"isNullUnknown"
|
||||
"doublePredicateSupplier"
|
||||
)
|
||||
.verify();
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ public class DimensionPredicateFilterTest
|
|||
public void testEqualsContractForDelegatingStringPredicateFactory()
|
||||
{
|
||||
EqualsVerifier.forClass(DimensionPredicateFilter.DelegatingStringPredicateFactory.class)
|
||||
.withIgnoredFields("baseStringPredicate", "isNullUnknown")
|
||||
.withIgnoredFields("baseStringPredicate")
|
||||
.usingGetClass()
|
||||
.verify();
|
||||
}
|
||||
|
|
|
@ -501,6 +501,12 @@ public class EqualityFilterTests
|
|||
? ImmutableList.of()
|
||||
: ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||
);
|
||||
|
||||
assertFilterMatches(new EqualityFilter("vdim3-concat", ColumnType.STRING, "1", null), ImmutableList.of());
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(new EqualityFilter("vdim3-concat", ColumnType.STRING, "1", null)),
|
||||
NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
|
@ -34,7 +33,9 @@ import org.apache.druid.query.filter.DimFilter;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
@ -127,27 +128,27 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
final DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
return input -> Objects.equals(valueOrNull, input);
|
||||
return valueOrNull == null ? DruidObjectPredicate.isNull() : DruidObjectPredicate.equalTo(valueOrNull);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate makeLongPredicate()
|
||||
{
|
||||
return input -> Objects.equals(valueOrNull, String.valueOf(input));
|
||||
return input -> DruidPredicateMatch.of(Objects.equals(valueOrNull, String.valueOf(input)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate makeFloatPredicate()
|
||||
{
|
||||
return input -> Objects.equals(valueOrNull, String.valueOf(input));
|
||||
return input -> DruidPredicateMatch.of(Objects.equals(valueOrNull, String.valueOf(input)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate makeDoublePredicate()
|
||||
{
|
||||
return input -> Objects.equals(valueOrNull, String.valueOf(input));
|
||||
return input -> DruidPredicateMatch.of(Objects.equals(valueOrNull, String.valueOf(input)));
|
||||
}
|
||||
|
||||
};
|
||||
|
|
|
@ -377,9 +377,8 @@ public class InFilterTest extends BaseFilterTest
|
|||
);
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(toInFilterWithFn("dim3", yesNullFn, "NO")),
|
||||
NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||
ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
toInFilterWithFn("dim3", yesNullFn, "YES"),
|
||||
ImmutableList.of("a", "b", "c", "d", "e", "f")
|
||||
|
@ -563,8 +562,7 @@ public class InFilterTest extends BaseFilterTest
|
|||
"longPredicateSupplier",
|
||||
"floatPredicateSupplier",
|
||||
"doublePredicateSupplier",
|
||||
"stringPredicateSupplier",
|
||||
"hasNull"
|
||||
"stringPredicateSupplier"
|
||||
)
|
||||
.verify();
|
||||
}
|
||||
|
|
|
@ -139,7 +139,9 @@ public class LikeFilterTest extends BaseFilterTest
|
|||
);
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(new LikeDimFilter("dim1", "bar", null, new SubstringDimExtractionFn(3, 3))),
|
||||
ImmutableList.of("0", "1", "3", "5", "6")
|
||||
NullHandling.replaceWithDefault()
|
||||
? ImmutableList.of("0", "1", "3", "5", "6")
|
||||
: ImmutableList.of("5", "6")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
|
@ -150,7 +152,7 @@ public class LikeFilterTest extends BaseFilterTest
|
|||
NotDimFilter.of(new LikeDimFilter("dim2", "bbb", null, new SubstringDimExtractionFn(0, 3))),
|
||||
NullHandling.replaceWithDefault()
|
||||
? ImmutableList.of("0", "1", "2", "3", "4", "6")
|
||||
: ImmutableList.of("0", "1", "2", "4", "6")
|
||||
: ImmutableList.of("1", "2", "4", "6")
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -194,7 +196,9 @@ public class LikeFilterTest extends BaseFilterTest
|
|||
);
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(new LikeDimFilter("dim1", "a%", null, new SubstringDimExtractionFn(1, null))),
|
||||
ImmutableList.of("0", "1", "2", "4", "5", "6")
|
||||
NullHandling.replaceWithDefault()
|
||||
? ImmutableList.of("0", "1", "2", "4", "5", "6")
|
||||
: ImmutableList.of("1", "2", "4", "5", "6")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
|
@ -205,7 +209,7 @@ public class LikeFilterTest extends BaseFilterTest
|
|||
NotDimFilter.of(new LikeDimFilter("dim2", "a%", null, new SubstringDimExtractionFn(1, null))),
|
||||
NullHandling.replaceWithDefault()
|
||||
? ImmutableList.of("0", "3", "4", "5", "6")
|
||||
: ImmutableList.of("0", "4", "5", "6")
|
||||
: ImmutableList.of("4", "5", "6")
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -889,7 +889,9 @@ public class RangeFilterTests
|
|||
false,
|
||||
null
|
||||
),
|
||||
canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "2", "7") : ImmutableList.of("0")
|
||||
NullHandling.replaceWithDefault() && canTestNumericNullsAsDefaultValues
|
||||
? ImmutableList.of("0", "2", "7")
|
||||
: ImmutableList.of("0")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
|
@ -902,7 +904,9 @@ public class RangeFilterTests
|
|||
false,
|
||||
null
|
||||
),
|
||||
canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "4", "6") : ImmutableList.of("0")
|
||||
NullHandling.replaceWithDefault() && canTestNumericNullsAsDefaultValues
|
||||
? ImmutableList.of("0", "4", "6")
|
||||
: ImmutableList.of("0")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.druid.segment.filter;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.SimpleAscendingOffset;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
|
@ -84,7 +86,7 @@ public class ValueMatchersTest extends InitializedNullHandlingTest
|
|||
ConstantMatcherType resultMatchNull = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
DimensionSelector.constant(null),
|
||||
false,
|
||||
string -> string == null
|
||||
DruidObjectPredicate.isNull()
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNull);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNull);
|
||||
|
@ -92,15 +94,23 @@ public class ValueMatchersTest extends InitializedNullHandlingTest
|
|||
ConstantMatcherType resultMatchNotNull = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
DimensionSelector.constant(null),
|
||||
false,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNotNull);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_UNKNOWN, resultMatchNotNull);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_FALSE, resultMatchNotNull);
|
||||
|
||||
ConstantMatcherType resultMatchNullUnknown = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
DimensionSelector.constant(null),
|
||||
false,
|
||||
value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.of(true)
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNullUnknown);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_UNKNOWN, resultMatchNullUnknown);
|
||||
|
||||
ConstantMatcherType resultMatchNonNilConstant = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
supplierSingleConstant.get().makeDimensionSelector(new SimpleAscendingOffset(1), null),
|
||||
false,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNonNilConstant);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNonNilConstant);
|
||||
|
@ -108,14 +118,14 @@ public class ValueMatchersTest extends InitializedNullHandlingTest
|
|||
ConstantMatcherType resultMatchNonNil = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
supplierSingle.get().makeDimensionSelector(new SimpleAscendingOffset(1), null),
|
||||
false,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNull(resultMatchNonNil);
|
||||
|
||||
ConstantMatcherType resultMatchNonNilMulti = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
supplierMulti.get().makeDimensionSelector(new SimpleAscendingOffset(1), null),
|
||||
true,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNull(resultMatchNonNilMulti);
|
||||
}
|
||||
|
@ -126,7 +136,7 @@ public class ValueMatchersTest extends InitializedNullHandlingTest
|
|||
ConstantMatcherType resultMatchNull = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)),
|
||||
false,
|
||||
string -> string == null
|
||||
DruidObjectPredicate.isNull()
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNull);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNull);
|
||||
|
@ -134,15 +144,23 @@ public class ValueMatchersTest extends InitializedNullHandlingTest
|
|||
ConstantMatcherType resultMatchNotNull = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)),
|
||||
false,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNotNull);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_UNKNOWN, resultMatchNotNull);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_FALSE, resultMatchNotNull);
|
||||
|
||||
ConstantMatcherType resultMatchNullUnknown = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)),
|
||||
false,
|
||||
value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.of(true)
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNullUnknown);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_UNKNOWN, resultMatchNullUnknown);
|
||||
|
||||
ConstantMatcherType resultMatchNotNilConstant = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
supplierSingleConstant.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)),
|
||||
false,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNotNull(resultMatchNotNilConstant);
|
||||
Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNotNilConstant);
|
||||
|
@ -150,14 +168,14 @@ public class ValueMatchersTest extends InitializedNullHandlingTest
|
|||
ConstantMatcherType resultMatchNotNil = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
supplierSingle.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)),
|
||||
false,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNull(resultMatchNotNil);
|
||||
|
||||
ConstantMatcherType resultMatchNotNilMulti = ValueMatchers.toConstantMatcherTypeIfPossible(
|
||||
supplierMulti.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)),
|
||||
true,
|
||||
string -> string != null
|
||||
DruidObjectPredicate.notNull()
|
||||
);
|
||||
Assert.assertNull(resultMatchNotNilMulti);
|
||||
}
|
||||
|
|
|
@ -20,8 +20,6 @@
|
|||
package org.apache.druid.segment.incremental;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -45,6 +43,7 @@ import org.apache.druid.query.filter.DimFilters;
|
|||
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.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
|
@ -664,7 +663,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
|
|||
private class DictionaryRaceTestFilterDruidPredicateFactory implements DruidPredicateFactory
|
||||
{
|
||||
@Override
|
||||
public Predicate<String> makeStringPredicate()
|
||||
public DruidObjectPredicate<String> makeStringPredicate()
|
||||
{
|
||||
try {
|
||||
index.add(
|
||||
|
@ -679,7 +678,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
|
|||
throw new RuntimeException(isee);
|
||||
}
|
||||
|
||||
return Predicates.alwaysTrue();
|
||||
return DruidObjectPredicate.alwaysTrue();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.collections.bitmap.MutableBitmap;
|
|||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.DefaultBitmapResultFactory;
|
||||
import org.apache.druid.query.filter.DruidObjectPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
|
@ -352,7 +353,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
true,
|
||||
"g",
|
||||
true,
|
||||
s -> !"fooo".equals(s)
|
||||
DruidObjectPredicate.notEqualTo("fooo")
|
||||
);
|
||||
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
|
||||
checkBitmap(bitmap, 0, 9);
|
||||
|
@ -362,7 +363,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
true,
|
||||
"g",
|
||||
true,
|
||||
s -> "fooo".equals(s)
|
||||
DruidObjectPredicate.equalTo("fooo")
|
||||
);
|
||||
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
|
||||
checkBitmap(bitmap, 2, 5);
|
||||
|
@ -372,7 +373,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
false,
|
||||
"z",
|
||||
false,
|
||||
s -> !"fooo".equals(s)
|
||||
DruidObjectPredicate.notEqualTo("fooo")
|
||||
);
|
||||
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
|
||||
checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 8, 9);
|
||||
|
@ -382,7 +383,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
false,
|
||||
"z",
|
||||
true,
|
||||
s -> !"fooo".equals(s)
|
||||
DruidObjectPredicate.notEqualTo("fooo")
|
||||
);
|
||||
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
|
||||
checkBitmap(bitmap, 0, 1, 3, 7, 8, 9);
|
||||
|
@ -392,7 +393,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
true,
|
||||
null,
|
||||
true,
|
||||
s -> true
|
||||
DruidObjectPredicate.alwaysTrue()
|
||||
);
|
||||
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
|
||||
checkBitmap(bitmap, 0, 2, 4, 5, 6, 9);
|
||||
|
@ -1378,10 +1379,10 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
// string: [b, foo, fooo, z]
|
||||
// small enough should be cool
|
||||
Assert.assertNotNull(stringRange.forRange("fo", false, "fooo", false));
|
||||
Assert.assertNotNull(stringRange.forRange("fo", false, "fooo", false, (s) -> true));
|
||||
Assert.assertNotNull(stringRange.forRange("fo", false, "fooo", false, DruidObjectPredicate.alwaysTrue()));
|
||||
// range too big, no index
|
||||
Assert.assertNull(stringRange.forRange("fo", false, "z", false));
|
||||
Assert.assertNull(stringRange.forRange("fo", false, "z", false, (s) -> true));
|
||||
Assert.assertNull(stringRange.forRange("fo", false, "z", false, DruidObjectPredicate.alwaysTrue()));
|
||||
|
||||
// long: [1, 3, 100, 300]
|
||||
// small enough should be cool
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.segment.virtual;
|
||||
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
|
@ -35,6 +34,7 @@ import org.apache.druid.query.dimension.ExtractionDimensionSpec;
|
|||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.extraction.BucketExtractionFn;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.DruidPredicateMatch;
|
||||
import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -473,7 +473,11 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
|
||||
final ValueMatcher nullMatcher = selector.makeValueMatcher((String) null);
|
||||
final ValueMatcher fiveMatcher = selector.makeValueMatcher("5");
|
||||
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.notNull(), false));
|
||||
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(
|
||||
StringPredicateDruidPredicateFactory.of(
|
||||
value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.TRUE
|
||||
)
|
||||
);
|
||||
|
||||
CURRENT_ROW.set(ROW0);
|
||||
Assert.assertEquals(true, nullMatcher.matches(false));
|
||||
|
@ -516,7 +520,11 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
COLUMN_SELECTOR_FACTORY
|
||||
);
|
||||
|
||||
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.notNull(), false));
|
||||
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(
|
||||
StringPredicateDruidPredicateFactory.of(
|
||||
value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.TRUE
|
||||
)
|
||||
);
|
||||
|
||||
CURRENT_ROW.set(ROW0);
|
||||
Assert.assertEquals(false, nonNullMatcher.matches(false));
|
||||
|
@ -597,7 +605,11 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
|
||||
final ValueMatcher nullMatcher = selector.makeValueMatcher((String) null);
|
||||
final ValueMatcher fiveMatcher = selector.makeValueMatcher("5");
|
||||
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.notNull(), false));
|
||||
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(
|
||||
StringPredicateDruidPredicateFactory.of(
|
||||
value -> value == null ? DruidPredicateMatch.UNKNOWN : DruidPredicateMatch.TRUE
|
||||
)
|
||||
);
|
||||
|
||||
CURRENT_ROW.set(ROW0);
|
||||
Assert.assertEquals(true, nullMatcher.matches(false));
|
||||
|
|
Loading…
Reference in New Issue