mirror of https://github.com/apache/druid.git
expression transform improvements and fixes (#13947)
changes: * fixes inconsistent handling of byte[] values between ExprEval.bestEffortOf and ExprEval.ofType, which could cause byte[] values to end up as java toString values instead of base64 encoded strings in ingest time transforms * improved ExpressionTransform binding to re-use ExprEval.bestEffortOf when evaluating a binding instead of throwing it away * improved ExpressionTransform array handling, added RowFunction.evalDimension that returns List<String> to back Row.getDimension and remove the automatic coercing of array types that would typically happen to expression transforms unless using Row.getDimension * added some tests for ExpressionTransform with array inputs * improved ExpressionPostAggregator to use partial type information from decoration * migrate some test uses of InputBindings.forMap to use other methods
This commit is contained in:
parent
b7752a909c
commit
f4392a3155
|
@ -19,11 +19,9 @@
|
|||
|
||||
package org.apache.druid.query.expressions;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -52,15 +50,15 @@ public class BloomFilterExpressionsTest extends InitializedNullHandlingTest
|
|||
BloomFilterExpressions.TestExprMacro testMacro = new BloomFilterExpressions.TestExprMacro();
|
||||
ExprMacroTable macroTable = new ExprMacroTable(ImmutableList.of(createMacro, addMacro, testMacro));
|
||||
|
||||
Expr.ObjectBinding inputBindings = InputBindings.withTypedSuppliers(
|
||||
new ImmutableMap.Builder<String, Pair<ExpressionType, Supplier<Object>>>()
|
||||
.put("bloomy", new Pair<>(BloomFilterExpressions.BLOOM_FILTER_TYPE, () -> new BloomKFilter(100)))
|
||||
.put("string", new Pair<>(ExpressionType.STRING, () -> SOME_STRING))
|
||||
.put("long", new Pair<>(ExpressionType.LONG, () -> SOME_LONG))
|
||||
.put("double", new Pair<>(ExpressionType.DOUBLE, () -> SOME_DOUBLE))
|
||||
.put("string_array", new Pair<>(ExpressionType.STRING_ARRAY, () -> SOME_STRING_ARRAY))
|
||||
.put("long_array", new Pair<>(ExpressionType.LONG_ARRAY, () -> SOME_LONG_ARRAY))
|
||||
.put("double_array", new Pair<>(ExpressionType.DOUBLE_ARRAY, () -> SOME_DOUBLE_ARRAY))
|
||||
Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers(
|
||||
new ImmutableMap.Builder<String, InputBindings.InputSupplier>()
|
||||
.put("bloomy", InputBindings.inputSupplier(BloomFilterExpressions.BLOOM_FILTER_TYPE, () -> new BloomKFilter(100)))
|
||||
.put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> SOME_STRING))
|
||||
.put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> SOME_LONG))
|
||||
.put("double", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> SOME_DOUBLE))
|
||||
.put("string_array", InputBindings.inputSupplier(ExpressionType.STRING_ARRAY, () -> SOME_STRING_ARRAY))
|
||||
.put("long_array", InputBindings.inputSupplier(ExpressionType.LONG_ARRAY, () -> SOME_LONG_ARRAY))
|
||||
.put("double_array", InputBindings.inputSupplier(ExpressionType.DOUBLE_ARRAY, () -> SOME_DOUBLE_ARRAY))
|
||||
.build()
|
||||
);
|
||||
|
||||
|
|
|
@ -515,6 +515,9 @@ public abstract class ExprEval<T>
|
|||
if (value instanceof List) {
|
||||
return bestEffortOf(value);
|
||||
}
|
||||
if (value instanceof byte[]) {
|
||||
return new StringExprEval(StringUtils.encodeBase64String((byte[]) value));
|
||||
}
|
||||
return of(Evals.asString(value));
|
||||
case LONG:
|
||||
if (value instanceof Number) {
|
||||
|
|
|
@ -20,11 +20,14 @@
|
|||
package org.apache.druid.math.expr;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.Objects;
|
||||
|
||||
public class InputBindings
|
||||
{
|
||||
|
@ -45,11 +48,41 @@ public class InputBindings
|
|||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Empty {@link Expr.ObjectBinding} that doesn't complain about attempts to access type or value for any input
|
||||
* identifiers, both of which will be nulls. Typically used for evaluating known constant expressions, or finding
|
||||
* a default or initial value of some expression if all inputs are null.
|
||||
*/
|
||||
public static Expr.ObjectBinding nilBindings()
|
||||
{
|
||||
return NIL_BINDINGS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Empty binding that throw a {@link UOE} if anything attempts to lookup an identifier type or value
|
||||
*/
|
||||
public static Expr.ObjectBinding validateConstant(Expr expr)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
// Sanity check. Bindings should not be used for a constant expression so explode if something tried
|
||||
throw new UOE("Expression " + expr.stringify() + " has non-constant inputs.");
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
// Sanity check. Bindings should not be used for a constant expression so explode if something tried
|
||||
throw new UOE("Expression " + expr.stringify() + " has non-constant inputs.");
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an {@link Expr.InputBindingInspector} backed by a map of binding identifiers to their {@link ExprType}
|
||||
*/
|
||||
|
@ -66,50 +99,40 @@ public class InputBindings
|
|||
};
|
||||
}
|
||||
|
||||
public static Expr.ObjectBinding singleProvider(ExpressionType type, final Function<String, ?> valueFn)
|
||||
/**
|
||||
* Creates a {@link Expr.ObjectBinding} backed by some {@link Row}. {@link ColumnHolder#TIME_COLUMN_NAME} is special
|
||||
* handled to be backed by {@link Row#getTimestampFromEpoch()}, all other values are ethically sourced from
|
||||
* {@link Row#getRaw(String)}.
|
||||
*
|
||||
* Types are detected and values are coereced via {@link ExprEval#bestEffortOf(Object)} because input types are
|
||||
* currently unknown.
|
||||
*/
|
||||
public static Expr.ObjectBinding forRow(Row row)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
return new BestEffortInputBindings()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
ExprEval compute(String name)
|
||||
{
|
||||
return valueFn.apply(name);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
return type;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static Expr.ObjectBinding forFunction(final Function<String, ?> valueFn)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
return valueFn.apply(name);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
return ExprEval.bestEffortOf(valueFn.apply(name)).type();
|
||||
if (ColumnHolder.TIME_COLUMN_NAME.equals(name)) {
|
||||
return ExprEval.ofLong(row.getTimestampFromEpoch());
|
||||
}
|
||||
return ExprEval.bestEffortOf(row.getRaw(name));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create {@link Expr.ObjectBinding} backed by {@link Map} to provide values for identifiers to evaluate {@link Expr}
|
||||
*
|
||||
* Types are detected and values are coereced via {@link ExprEval#bestEffortOf(Object)} because input types are
|
||||
* currently unknown.
|
||||
*
|
||||
* This method is only used for testing and mimics the behavior of {@link #forRow(Row)} except lacks special handling
|
||||
* for columns named {@link ColumnHolder#TIME_COLUMN_NAME}.
|
||||
*/
|
||||
public static Expr.ObjectBinding withMap(final Map<String, ?> bindings)
|
||||
@Deprecated
|
||||
public static Expr.ObjectBinding forMap(final Map<String, ?> bindings)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
|
@ -130,10 +153,51 @@ public class InputBindings
|
|||
}
|
||||
|
||||
/**
|
||||
* Create {@link Expr.ObjectBinding} backed by map of {@link Supplier} to provide values for identifiers to evaluate
|
||||
* {@link Expr}
|
||||
* Create {@link Expr.ObjectBinding} backed by {@link Map} to provide values for identifiers to evaluate {@link Expr}
|
||||
*
|
||||
* Types are detected and values are coereced via {@link ExprEval#bestEffortOf(Object)} because input types are
|
||||
* currently unknown.
|
||||
*/
|
||||
public static Expr.ObjectBinding withTypedSuppliers(final Map<String, Pair<ExpressionType, Supplier<Object>>> bindings)
|
||||
public static Expr.ObjectBinding forMap(final Map<String, ?> bindings, Expr.InputBindingInspector inspector)
|
||||
{
|
||||
final Expr.InputBindingInspector inputBindingInspector = inspector;
|
||||
return new BestEffortInputBindings()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
if (inputBindingInspector.getType(name) != null) {
|
||||
return bindings.get(name);
|
||||
}
|
||||
// we didn't have complete type information on this one, fall through to bestEffortOf
|
||||
return super.get(name);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
final ExpressionType type = inputBindingInspector.getType(name);
|
||||
if (type != null) {
|
||||
return type;
|
||||
}
|
||||
// we didn't have complete type information on this one, fall through to bestEffortOf
|
||||
return super.getType(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
ExprEval compute(String name)
|
||||
{
|
||||
return ExprEval.bestEffortOf(bindings.get(name));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a {@link Expr.ObjectBinding} for a single input value of a known type provided by some {@link Supplier}
|
||||
*/
|
||||
public static Expr.ObjectBinding forInputSupplier(ExpressionType type, Supplier<?> supplier)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
|
@ -141,20 +205,132 @@ public class InputBindings
|
|||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
Pair<ExpressionType, Supplier<Object>> binding = bindings.get(name);
|
||||
return binding == null || binding.rhs == null ? null : binding.rhs.get();
|
||||
return supplier.get();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
Pair<ExpressionType, Supplier<Object>> binding = bindings.get(name);
|
||||
if (binding == null) {
|
||||
return null;
|
||||
}
|
||||
return binding.lhs;
|
||||
return type;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a {@link Expr.ObjectBinding} for a single input value of a known type provided by some {@link Supplier}
|
||||
*/
|
||||
public static Expr.ObjectBinding forInputSupplier(String supplierName, ExpressionType type, Supplier<?> supplier)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
if (Objects.equals(name, supplierName)) {
|
||||
return supplier.get();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
if (Objects.equals(name, supplierName)) {
|
||||
return type;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static <T> InputSupplier<T> inputSupplier(ExpressionType type, Supplier<T> supplier)
|
||||
{
|
||||
return new InputSupplier<>(type, supplier);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create {@link Expr.ObjectBinding} backed by map of {@link Supplier} to provide values for identifiers to evaluate
|
||||
* {@link Expr}
|
||||
*/
|
||||
public static Expr.ObjectBinding forInputSuppliers(final Map<String, InputSupplier> bindings)
|
||||
{
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
InputSupplier<?> binding = bindings.get(name);
|
||||
return binding == null ? null : binding.get();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
InputSupplier<?> binding = bindings.get(name);
|
||||
if (binding == null) {
|
||||
return null;
|
||||
}
|
||||
return binding.getType();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static class InputSupplier<T> implements Supplier<T>
|
||||
{
|
||||
private final ExpressionType type;
|
||||
private final Supplier<T> supplier;
|
||||
|
||||
private InputSupplier(ExpressionType type, Supplier<T> supplier)
|
||||
{
|
||||
this.supplier = supplier;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T get()
|
||||
{
|
||||
return supplier.get();
|
||||
}
|
||||
|
||||
public ExpressionType getType()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Expr.ObjectBinding} backed by a cache populated by {@link ExprEval#bestEffortOf(Object)} for when the input
|
||||
* type information is totally unknown, for a single row worth of values. The values are cached so that asking for a
|
||||
* type and getting the value of some input do not repeat computations.
|
||||
*
|
||||
* This type is not thread-safe, and not suitable for re-use for processing multiple-rows due to the presence of the
|
||||
* result cache.
|
||||
*/
|
||||
public abstract static class BestEffortInputBindings implements Expr.ObjectBinding
|
||||
{
|
||||
private final Map<String, ExprEval> cachedBindings = new HashMap<>();
|
||||
|
||||
abstract ExprEval compute(String name);
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
cachedBindings.computeIfAbsent(name, this::compute);
|
||||
return cachedBindings.get(name).value();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
cachedBindings.computeIfAbsent(name, this::compute);
|
||||
return cachedBindings.get(name).type();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.druid.query.aggregation.PostAggregator;
|
|||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.utils.CollectionUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Comparator;
|
||||
|
@ -70,6 +69,7 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
|
||||
private final ExprMacroTable macroTable;
|
||||
private final Map<String, Function<Object, Object>> finalizers;
|
||||
private final Expr.InputBindingInspector partialTypeInformation;
|
||||
|
||||
private final Supplier<Expr> parsed;
|
||||
private final Supplier<Set<String>> dependentFields;
|
||||
|
@ -91,7 +91,6 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
expression,
|
||||
ordering,
|
||||
macroTable,
|
||||
ImmutableMap.of(),
|
||||
Parser.lazyParse(expression, macroTable)
|
||||
);
|
||||
}
|
||||
|
@ -101,7 +100,6 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
final String expression,
|
||||
@Nullable final String ordering,
|
||||
final ExprMacroTable macroTable,
|
||||
final Map<String, Function<Object, Object>> finalizers,
|
||||
final Supplier<Expr> parsed
|
||||
)
|
||||
{
|
||||
|
@ -110,7 +108,8 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
expression,
|
||||
ordering,
|
||||
macroTable,
|
||||
finalizers,
|
||||
ImmutableMap.of(),
|
||||
InputBindings.nilBindings(),
|
||||
parsed,
|
||||
Suppliers.memoize(() -> parsed.get().analyzeInputs().getRequiredBindings())
|
||||
);
|
||||
|
@ -122,6 +121,7 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
@Nullable final String ordering,
|
||||
final ExprMacroTable macroTable,
|
||||
final Map<String, Function<Object, Object>> finalizers,
|
||||
final Expr.InputBindingInspector partialTypeInformation,
|
||||
final Supplier<Expr> parsed,
|
||||
final Supplier<Set<String>> dependentFields
|
||||
)
|
||||
|
@ -135,6 +135,7 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
this.comparator = ordering == null ? DEFAULT_COMPARATOR : Ordering.valueOf(ordering);
|
||||
this.macroTable = macroTable;
|
||||
this.finalizers = finalizers;
|
||||
this.partialTypeInformation = partialTypeInformation;
|
||||
|
||||
this.parsed = parsed;
|
||||
this.dependentFields = dependentFields;
|
||||
|
@ -171,7 +172,9 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
}
|
||||
);
|
||||
|
||||
return parsed.get().eval(InputBindings.withMap(finalizedValues)).valueOrDefault();
|
||||
// we use partialTypeInformation to avoid unnecessarily coercing aggregator values for which we do have type info
|
||||
// from decoration
|
||||
return parsed.get().eval(InputBindings.forMap(finalizedValues, partialTypeInformation)).valueOrDefault();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -195,12 +198,19 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
@Override
|
||||
public ExpressionPostAggregator decorate(final Map<String, AggregatorFactory> aggregators)
|
||||
{
|
||||
final Map<String, Function<Object, Object>> finalizers = Maps.newHashMapWithExpectedSize(aggregators.size());
|
||||
final Map<String, ExpressionType> types = Maps.newHashMapWithExpectedSize(aggregators.size());
|
||||
for (Map.Entry<String, AggregatorFactory> factory : aggregators.entrySet()) {
|
||||
finalizers.put(factory.getKey(), factory.getValue()::finalizeComputation);
|
||||
types.put(factory.getKey(), ExpressionType.fromColumnType(factory.getValue().getResultType()));
|
||||
}
|
||||
return new ExpressionPostAggregator(
|
||||
name,
|
||||
expression,
|
||||
ordering,
|
||||
macroTable,
|
||||
CollectionUtils.mapValues(aggregators, aggregatorFactory -> aggregatorFactory::finalizeComputation),
|
||||
finalizers,
|
||||
InputBindings.inspectorFromTypeMap(types),
|
||||
parsed,
|
||||
dependentFields
|
||||
);
|
||||
|
@ -242,7 +252,7 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
* Ensures the following order: numeric > NaN > Infinite.
|
||||
*
|
||||
* The name may be referenced via Ordering.valueOf(String) in the constructor {@link
|
||||
* ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, ExprMacroTable, Map, Supplier, Supplier)}.
|
||||
* ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, ExprMacroTable, Map, Expr.InputBindingInspector, Supplier, Supplier)}.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
numericFirst {
|
||||
|
|
|
@ -85,6 +85,11 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer<int[]
|
|||
final int oldDictSize = dimLookup.size();
|
||||
final long oldDictSizeInBytes = useMaxMemoryEstimates ? 0 : dimLookup.sizeInBytes();
|
||||
|
||||
// expressions which operate on multi-value string inputs as arrays might spit out arrays, coerce to list
|
||||
if (dimValues instanceof Object[]) {
|
||||
dimValues = Arrays.asList((Object[]) dimValues);
|
||||
}
|
||||
|
||||
if (dimValues == null) {
|
||||
final int nullId = dimLookup.getId(null);
|
||||
encodedDimensionValues = nullId == DimensionDictionary.ABSENT_VALUE_ID ? new int[]{dimLookup.add(null)} : new int[]{nullId};
|
||||
|
|
|
@ -334,7 +334,10 @@ public class ExpressionFilter implements Filter
|
|||
public Predicate<String> makeStringPredicate()
|
||||
{
|
||||
return value -> expr.get().eval(
|
||||
InputBindings.forFunction(identifierName -> NullHandling.nullToEmptyIfNeeded(value))
|
||||
InputBindings.forInputSupplier(
|
||||
ExpressionType.STRING,
|
||||
() -> NullHandling.nullToEmptyIfNeeded(value)
|
||||
)
|
||||
).asBoolean();
|
||||
}
|
||||
|
||||
|
@ -346,7 +349,7 @@ public class ExpressionFilter implements Filter
|
|||
@Override
|
||||
public boolean applyLong(long input)
|
||||
{
|
||||
return expr.get().eval(InputBindings.forFunction(identifierName -> input)).asBoolean();
|
||||
return expr.get().eval(InputBindings.forInputSupplier(ExpressionType.LONG, () -> input)).asBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -365,7 +368,9 @@ public class ExpressionFilter implements Filter
|
|||
@Override
|
||||
public boolean applyFloat(float input)
|
||||
{
|
||||
return expr.get().eval(InputBindings.forFunction(identifierName -> input)).asBoolean();
|
||||
return expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.DOUBLE, () -> input)
|
||||
).asBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -384,7 +389,9 @@ public class ExpressionFilter implements Filter
|
|||
@Override
|
||||
public boolean applyDouble(double input)
|
||||
{
|
||||
return expr.get().eval(InputBindings.forFunction(identifierName -> input)).asBoolean();
|
||||
return expr.get().eval(
|
||||
InputBindings.forInputSupplier(ExpressionType.DOUBLE, () -> input)
|
||||
).asBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -726,7 +726,7 @@ public abstract class IncrementalIndex extends AbstractIndex implements Iterable
|
|||
}
|
||||
|
||||
if (inputRow instanceof Transformer.TransformedInputRow) {
|
||||
InputRow innerRow = ((Transformer.TransformedInputRow) inputRow).getRow();
|
||||
InputRow innerRow = ((Transformer.TransformedInputRow) inputRow).getBaseRow();
|
||||
if (innerRow instanceof MapBasedInputRow) {
|
||||
return ((MapBasedInputRow) innerRow).getEvent().toString();
|
||||
}
|
||||
|
|
|
@ -25,15 +25,12 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Suppliers;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.data.input.Rows;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.NonnullPair;
|
||||
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.math.expr.InputBindings;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.virtual.ExpressionSelectors;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -103,36 +100,24 @@ public class ExpressionTransform implements Transform
|
|||
public Object eval(final Row row)
|
||||
{
|
||||
try {
|
||||
// this will need adjusted if we want to allow expression transforms to produce true arrays. Currently, calling
|
||||
// this method will coerce any expression output into:
|
||||
// - the expression value if the value is not an array
|
||||
// - the single array element if the value is an array with 1 element
|
||||
// - a list with all of the array elements if the value is an array with more than 1 element
|
||||
// and so is tuned towards multi-value strings
|
||||
return ExpressionSelectors.coerceEvalToObjectOrList(
|
||||
expr.eval(InputBindings.forFunction(name -> getValueFromRow(row, name)))
|
||||
);
|
||||
return expr.eval(InputBindings.forRow(row)).valueOrDefault();
|
||||
}
|
||||
catch (Throwable t) {
|
||||
throw new ISE(t, "Could not transform value for %s reason: %s", name, t.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static Object getValueFromRow(final Row row, final String column)
|
||||
{
|
||||
if (column.equals(ColumnHolder.TIME_COLUMN_NAME)) {
|
||||
return row.getTimestampFromEpoch();
|
||||
} else {
|
||||
Object raw = row.getRaw(column);
|
||||
if (raw instanceof List) {
|
||||
NonnullPair<ExpressionType, Object[]> coerced = ExprEval.coerceListToArray((List) raw, true);
|
||||
if (coerced == null) {
|
||||
return null;
|
||||
}
|
||||
return coerced.rhs;
|
||||
@Override
|
||||
public List<String> evalDimension(Row row)
|
||||
{
|
||||
try {
|
||||
return Rows.objectToStrings(
|
||||
ExpressionSelectors.coerceEvalToObjectOrList(expr.eval(InputBindings.forRow(row)))
|
||||
);
|
||||
}
|
||||
catch (Throwable t) {
|
||||
throw new ISE(t, "Could not transform dimension value for %s reason: %s", name, t.getMessage());
|
||||
}
|
||||
return raw;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,10 +21,14 @@ package org.apache.druid.segment.transform;
|
|||
|
||||
import org.apache.druid.data.input.Row;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Interface for evaluating functions on rows. Used by {@link Transformer}.
|
||||
*/
|
||||
public interface RowFunction
|
||||
{
|
||||
Object eval(Row row);
|
||||
|
||||
List<String> evalDimension(Row row);
|
||||
}
|
||||
|
|
|
@ -204,7 +204,7 @@ public class Transformer
|
|||
{
|
||||
final RowFunction transform = transforms.get(dimension);
|
||||
if (transform != null) {
|
||||
return Rows.objectToStrings(transform.eval(row));
|
||||
return transform.evalDimension(row);
|
||||
} else {
|
||||
return row.getDimension(dimension);
|
||||
}
|
||||
|
@ -232,7 +232,7 @@ public class Transformer
|
|||
}
|
||||
}
|
||||
|
||||
public InputRow getRow()
|
||||
public InputRow getBaseRow()
|
||||
{
|
||||
return row;
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.base.Supplier;
|
|||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.NonnullPair;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
|
@ -322,7 +321,7 @@ public class ExpressionSelectors
|
|||
)
|
||||
{
|
||||
final List<String> columns = plan.getAnalysis().getRequiredBindingsList();
|
||||
final Map<String, Pair<ExpressionType, Supplier<Object>>> suppliers = new HashMap<>();
|
||||
final Map<String, InputBindings.InputSupplier> suppliers = new HashMap<>();
|
||||
for (String columnName : columns) {
|
||||
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName);
|
||||
final boolean multiVal = capabilities != null && capabilities.hasMultipleValues().isTrue();
|
||||
|
@ -377,7 +376,7 @@ public class ExpressionSelectors
|
|||
}
|
||||
|
||||
if (supplier != null) {
|
||||
suppliers.put(columnName, new Pair<>(expressionType, supplier));
|
||||
suppliers.put(columnName, InputBindings.inputSupplier(expressionType, supplier));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -386,29 +385,11 @@ public class ExpressionSelectors
|
|||
} else if (suppliers.size() == 1 && columns.size() == 1) {
|
||||
// If there's only one column (and it has a supplier), we can skip the Map and just use that supplier when
|
||||
// asked for something.
|
||||
final String column = Iterables.getOnlyElement(suppliers.keySet());
|
||||
final Pair<ExpressionType, Supplier<Object>> supplier = Iterables.getOnlyElement(suppliers.values());
|
||||
final InputBindings.InputSupplier<?> supplier = Iterables.getOnlyElement(suppliers.values());
|
||||
|
||||
return new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
// There's only one binding, and it must be the single column, so it can safely be ignored in production.
|
||||
assert column.equals(name);
|
||||
return supplier.rhs.get();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
return supplier.lhs;
|
||||
}
|
||||
};
|
||||
return InputBindings.forInputSupplier(supplier.getType(), supplier);
|
||||
} else {
|
||||
return InputBindings.withTypedSuppliers(suppliers);
|
||||
return InputBindings.forInputSuppliers(suppliers);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -68,7 +68,7 @@ public class SingleStringInputCachingExpressionColumnValueSelector extends BaseE
|
|||
this.expression = Preconditions.checkNotNull(expression, "expression");
|
||||
|
||||
final Supplier<Object> inputSupplier = ExpressionSelectors.supplierFromDimensionSelector(selector, false, false);
|
||||
this.bindings = InputBindings.singleProvider(ExpressionType.STRING, name -> inputSupplier.get());
|
||||
this.bindings = InputBindings.forInputSupplier(ExpressionType.STRING, inputSupplier);
|
||||
|
||||
if (selector.getValueCardinality() == DimensionDictionarySelector.CARDINALITY_UNKNOWN) {
|
||||
throw new ISE("Selector must have a dictionary");
|
||||
|
|
|
@ -48,7 +48,7 @@ public class ApplyFunctionTest extends InitializedNullHandlingTest
|
|||
builder.put("d", new String[] {null});
|
||||
builder.put("e", new String[] {null, "foo", "bar"});
|
||||
builder.put("f", new String[0]);
|
||||
bindings = InputBindings.withMap(builder.build());
|
||||
bindings = InputBindings.forMap(builder.build());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.math.expr;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import junitparams.converters.Nullable;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -76,7 +77,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testDoubleEval()
|
||||
{
|
||||
Expr.ObjectBinding bindings = InputBindings.withMap(ImmutableMap.of("x", 2.0d));
|
||||
Expr.ObjectBinding bindings = InputBindings.forMap(ImmutableMap.of("x", 2.0d));
|
||||
assertEquals(2.0, evalDouble("x", bindings), 0.0001);
|
||||
assertEquals(2.0, evalDouble("\"x\"", bindings), 0.0001);
|
||||
assertEquals(304.0, evalDouble("300 + \"x\" * 2", bindings), 0.0001);
|
||||
|
@ -143,7 +144,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testLongEval()
|
||||
{
|
||||
Expr.ObjectBinding bindings = InputBindings.withMap(ImmutableMap.of("x", 9223372036854775807L));
|
||||
Expr.ObjectBinding bindings = InputBindings.forMap(ImmutableMap.of("x", 9223372036854775807L));
|
||||
|
||||
assertEquals(9223372036854775807L, evalLong("x", bindings));
|
||||
assertEquals(9223372036854775807L, evalLong("\"x\"", bindings));
|
||||
|
@ -411,7 +412,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testBooleanReturn()
|
||||
{
|
||||
Expr.ObjectBinding bindings = InputBindings.withMap(
|
||||
Expr.ObjectBinding bindings = InputBindings.forMap(
|
||||
ImmutableMap.of("x", 100L, "y", 100L, "z", 100D, "w", 100D)
|
||||
);
|
||||
|
||||
|
@ -470,9 +471,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testLogicalOperators()
|
||||
{
|
||||
Expr.ObjectBinding bindings = InputBindings.withMap(
|
||||
ImmutableMap.of()
|
||||
);
|
||||
Expr.ObjectBinding bindings = InputBindings.nilBindings();
|
||||
|
||||
try {
|
||||
ExpressionProcessing.initializeForStrictBooleansTests(true);
|
||||
|
@ -624,7 +623,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
bindingsMap.put("s2", "false");
|
||||
bindingsMap.put("b1", true);
|
||||
bindingsMap.put("b2", false);
|
||||
Expr.ObjectBinding bindings = InputBindings.withMap(bindingsMap);
|
||||
Expr.ObjectBinding bindings = InputBindings.forMap(bindingsMap);
|
||||
|
||||
try {
|
||||
ExpressionProcessing.initializeForStrictBooleansTests(true);
|
||||
|
@ -1052,34 +1051,25 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
public void testBestEffortOf()
|
||||
{
|
||||
// strings
|
||||
ExprEval eval = ExprEval.bestEffortOf("stringy");
|
||||
Assert.assertEquals(ExpressionType.STRING, eval.type());
|
||||
Assert.assertEquals("stringy", eval.value());
|
||||
assertBestEffortOf("stringy", ExpressionType.STRING, "stringy");
|
||||
|
||||
// by default, booleans are handled as strings
|
||||
eval = ExprEval.bestEffortOf(true);
|
||||
Assert.assertEquals(ExpressionType.STRING, eval.type());
|
||||
Assert.assertEquals("true", eval.value());
|
||||
assertBestEffortOf(true, ExpressionType.STRING, "true");
|
||||
|
||||
eval = ExprEval.bestEffortOf(new byte[]{1, 2, 3, 4});
|
||||
Assert.assertEquals(ExpressionType.STRING, eval.type());
|
||||
Assert.assertEquals(StringUtils.encodeBase64String(new byte[]{1, 2, 3, 4}), eval.value());
|
||||
assertBestEffortOf(
|
||||
new byte[]{1, 2, 3, 4},
|
||||
ExpressionType.STRING,
|
||||
StringUtils.encodeBase64String(new byte[]{1, 2, 3, 4})
|
||||
);
|
||||
|
||||
// longs
|
||||
eval = ExprEval.bestEffortOf(1L);
|
||||
Assert.assertEquals(ExpressionType.LONG, eval.type());
|
||||
Assert.assertEquals(1L, eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(1);
|
||||
Assert.assertEquals(ExpressionType.LONG, eval.type());
|
||||
Assert.assertEquals(1L, eval.value());
|
||||
assertBestEffortOf(1L, ExpressionType.LONG, 1L);
|
||||
assertBestEffortOf(1, ExpressionType.LONG, 1L);
|
||||
|
||||
try {
|
||||
// in strict boolean mode, they are longs
|
||||
ExpressionProcessing.initializeForStrictBooleansTests(true);
|
||||
eval = ExprEval.ofType(ExpressionType.LONG, true);
|
||||
Assert.assertEquals(ExpressionType.LONG, eval.type());
|
||||
Assert.assertEquals(1L, eval.value());
|
||||
assertBestEffortOf(true, ExpressionType.LONG, 1L);
|
||||
}
|
||||
finally {
|
||||
// reset
|
||||
|
@ -1087,96 +1077,68 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
// doubles
|
||||
eval = ExprEval.bestEffortOf(1.0);
|
||||
Assert.assertEquals(ExpressionType.DOUBLE, eval.type());
|
||||
Assert.assertEquals(1.0, eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(1.0f);
|
||||
Assert.assertEquals(ExpressionType.DOUBLE, eval.type());
|
||||
Assert.assertEquals(1.0, eval.value());
|
||||
assertBestEffortOf(1.0, ExpressionType.DOUBLE, 1.0);
|
||||
assertBestEffortOf(1.0f, ExpressionType.DOUBLE, 1.0);
|
||||
|
||||
// arrays
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1L, 2L, 3L});
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, 3L}, (Object[]) eval.value());
|
||||
assertBestEffortOf(new Object[] {1L, 2L, 3L}, ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, 3L});
|
||||
assertBestEffortOf(new Object[] {1L, 2L, null, 3L}, ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, null, 3L});
|
||||
assertBestEffortOf(ImmutableList.of(1L, 2L, 3L), ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, 3L});
|
||||
assertBestEffortOf(new long[] {1L, 2L, 3L}, ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, 3L});
|
||||
assertBestEffortOf(new Object[] {1, 2, 3}, ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, 3L});
|
||||
assertBestEffortOf(new Integer[] {1, 2, 3}, ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, 3L});
|
||||
assertBestEffortOf(new int[] {1, 2, 3}, ExpressionType.LONG_ARRAY, new Object[] {1L, 2L, 3L});
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1L, 2L, null, 3L});
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, null, 3L}, (Object[]) eval.value());
|
||||
assertBestEffortOf(new Object[] {1.0, 2.0, 3.0}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0, 3.0});
|
||||
assertBestEffortOf(
|
||||
new Object[] {null, 1.0, 2.0, 3.0},
|
||||
ExpressionType.DOUBLE_ARRAY,
|
||||
new Object[] {null, 1.0, 2.0, 3.0}
|
||||
);
|
||||
assertBestEffortOf(new Double[] {1.0, 2.0, 3.0}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0, 3.0});
|
||||
assertBestEffortOf(new double[] {1.0, 2.0, 3.0}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0, 3.0});
|
||||
assertBestEffortOf(new Object[] {1.0f, 2.0f, 3.0f}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0, 3.0});
|
||||
assertBestEffortOf(new Float[] {1.0f, 2.0f, 3.0f}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0, 3.0});
|
||||
assertBestEffortOf(new float[] {1.0f, 2.0f, 3.0f}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0, 3.0});
|
||||
|
||||
eval = ExprEval.bestEffortOf(ImmutableList.of(1L, 2L, 3L));
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, 3L}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new long[] {1L, 2L, 3L});
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, 3L}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1, 2, 3});
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, 3L}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Integer[] {1, 2, 3});
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, 3L}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new int[] {1, 2, 3});
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1L, 2L, 3L}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1.0, 2.0, 3.0});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Object[] {null, 1.0, 2.0, 3.0});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {null, 1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Double[] {1.0, 2.0, 3.0});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new double[] {1.0, 2.0, 3.0});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1.0f, 2.0f, 3.0f});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Float[] {1.0f, 2.0f, 3.0f});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new float[] {1.0f, 2.0f, 3.0f});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0, 3.0}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new Object[] {"1", "2", "3"});
|
||||
Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {"1", "2", "3"}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(new String[] {"1", "2", "3"});
|
||||
Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {"1", "2", "3"}, (Object[]) eval.value());
|
||||
|
||||
eval = ExprEval.bestEffortOf(ImmutableList.of("1", "2", "3"));
|
||||
Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {"1", "2", "3"}, (Object[]) eval.value());
|
||||
assertBestEffortOf(new Object[] {"1", "2", "3"}, ExpressionType.STRING_ARRAY, new Object[] {"1", "2", "3"});
|
||||
assertBestEffortOf(new String[] {"1", "2", "3"}, ExpressionType.STRING_ARRAY, new Object[] {"1", "2", "3"});
|
||||
assertBestEffortOf(ImmutableList.of("1", "2", "3"), ExpressionType.STRING_ARRAY, new Object[] {"1", "2", "3"});
|
||||
|
||||
// arrays end up as the least restrictive type
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1.0, 2L});
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {1.0, 2.0}, (Object[]) eval.value());
|
||||
assertBestEffortOf(new Object[] {1.0, 2L}, ExpressionType.DOUBLE_ARRAY, new Object[] {1.0, 2.0});
|
||||
|
||||
// arrays end up as the least restrictive type
|
||||
eval = ExprEval.bestEffortOf(new Object[] {1.0, 2L, "3", true, false});
|
||||
Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type());
|
||||
Assert.assertArrayEquals(new Object[] {"1.0", "2", "3", "true", "false"}, (Object[]) eval.value());
|
||||
assertBestEffortOf(
|
||||
new Object[] {1.0, 2L, "3", true, false},
|
||||
ExpressionType.STRING_ARRAY,
|
||||
new Object[] {"1.0", "2", "3", "true", "false"}
|
||||
);
|
||||
|
||||
// json type isn't defined in druid-core, what happens if we have some nested data?
|
||||
eval = ExprEval.bestEffortOf(ImmutableMap.of("x", 1L, "y", 2L));
|
||||
Assert.assertEquals(ExpressionType.UNKNOWN_COMPLEX, eval.type());
|
||||
Assert.assertEquals(ImmutableMap.of("x", 1L, "y", 2L), eval.value());
|
||||
// best effort of doesn't know of nested type, what happens if we have some nested data?
|
||||
assertBestEffortOf(
|
||||
ImmutableMap.of("x", 1L, "y", 2L),
|
||||
ExpressionType.UNKNOWN_COMPLEX,
|
||||
ImmutableMap.of("x", 1L, "y", 2L)
|
||||
);
|
||||
}
|
||||
|
||||
private void assertBestEffortOf(@Nullable Object val, ExpressionType expectedType, @Nullable Object expectedValue)
|
||||
{
|
||||
ExprEval eval = ExprEval.bestEffortOf(val);
|
||||
Assert.assertEquals(expectedType, eval.type());
|
||||
if (eval.type().isArray()) {
|
||||
Assert.assertArrayEquals((Object[]) expectedValue, eval.asArray());
|
||||
} else {
|
||||
Assert.assertEquals(expectedValue, eval.value());
|
||||
}
|
||||
// make sure that ofType matches bestEffortOf
|
||||
eval = ExprEval.ofType(eval.type(), val);
|
||||
Assert.assertEquals(expectedType, eval.type());
|
||||
if (eval.type().isArray()) {
|
||||
Assert.assertArrayEquals((Object[]) expectedValue, eval.asArray());
|
||||
} else {
|
||||
Assert.assertEquals(expectedValue, eval.value());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -82,7 +82,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
.put("someComplex", new TypeStrategiesTest.NullableLongPair(1L, 2L))
|
||||
.put("str1", "v1")
|
||||
.put("str2", "v2");
|
||||
bindings = InputBindings.withMap(builder.build());
|
||||
bindings = InputBindings.forMap(builder.build());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.math.expr;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
|
@ -912,7 +911,7 @@ public class ParserTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(
|
||||
expression,
|
||||
expected,
|
||||
parsed.eval(InputBindings.withMap(ImmutableMap.of())).value()
|
||||
parsed.eval(InputBindings.nilBindings()).value()
|
||||
);
|
||||
|
||||
final Expr parsedNoFlatten = Parser.parse(expression, ExprMacroTable.nil(), false);
|
||||
|
@ -920,7 +919,7 @@ public class ParserTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(
|
||||
expression,
|
||||
expected,
|
||||
parsedRoundTrip.eval(InputBindings.withMap(ImmutableMap.of())).value()
|
||||
parsedRoundTrip.eval(InputBindings.nilBindings()).value()
|
||||
);
|
||||
Assert.assertEquals(parsed.stringify(), parsedRoundTrip.stringify());
|
||||
}
|
||||
|
@ -928,7 +927,7 @@ public class ParserTest extends InitializedNullHandlingTest
|
|||
private void validateConstantExpression(String expression, Object[] expected)
|
||||
{
|
||||
Expr parsed = Parser.parse(expression, ExprMacroTable.nil());
|
||||
Object evaluated = parsed.eval(InputBindings.withMap(ImmutableMap.of())).value();
|
||||
Object evaluated = parsed.eval(InputBindings.nilBindings()).value();
|
||||
Assert.assertArrayEquals(
|
||||
expression,
|
||||
expected,
|
||||
|
@ -941,7 +940,7 @@ public class ParserTest extends InitializedNullHandlingTest
|
|||
Assert.assertArrayEquals(
|
||||
expression,
|
||||
expected,
|
||||
(Object[]) roundTrip.eval(InputBindings.withMap(ImmutableMap.of())).value()
|
||||
(Object[]) roundTrip.eval(InputBindings.nilBindings()).value()
|
||||
);
|
||||
Assert.assertEquals(parsed.stringify(), roundTrip.stringify());
|
||||
}
|
||||
|
|
|
@ -63,7 +63,7 @@ public class ExpressionPostAggregatorTest extends InitializedNullHandlingTest
|
|||
{
|
||||
EqualsVerifier.forClass(ExpressionPostAggregator.class)
|
||||
.usingGetClass()
|
||||
.withIgnoredFields("macroTable", "finalizers", "parsed", "dependentFields", "cacheKey")
|
||||
.withIgnoredFields("macroTable", "finalizers", "parsed", "dependentFields", "cacheKey", "partialTypeInformation")
|
||||
.verify();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
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.math.expr.InputBindings;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -37,20 +37,23 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
public void testErrorZeroArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[icontains_string] requires 2 arguments");
|
||||
eval("icontains_string()", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("icontains_string()", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testErrorThreeArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[icontains_string] requires 2 arguments");
|
||||
eval("icontains_string('a', 'b', 'c')", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("icontains_string('a', 'b', 'c')", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchSearchLowerCase()
|
||||
{
|
||||
final ExprEval<?> result = eval("icontains_string(a, 'OBA')", InputBindings.withMap(ImmutableMap.of("a", "foobar")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, 'OBA')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foobar")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -60,7 +63,10 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testMatchSearchUpperCase()
|
||||
{
|
||||
final ExprEval<?> result = eval("icontains_string(a, 'oba')", InputBindings.withMap(ImmutableMap.of("a", "FOOBAR")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, 'oba')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "FOOBAR")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -70,7 +76,10 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testNoMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("icontains_string(a, 'bar')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, 'bar')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(false).value(),
|
||||
result.value()
|
||||
|
@ -84,7 +93,10 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[icontains_string] substring must be a string literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("icontains_string(a, null)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -94,7 +106,10 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testEmptyStringSearch()
|
||||
{
|
||||
final ExprEval<?> result = eval("icontains_string(a, '')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -108,7 +123,10 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[icontains_string] substring must be a string literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("icontains_string(a, null)", InputBindings.withMap(ImmutableMap.of("a", "")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -118,7 +136,10 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testEmptyStringSearchOnEmptyString()
|
||||
{
|
||||
final ExprEval<?> result = eval("icontains_string(a, '')", InputBindings.withMap(ImmutableMap.of("a", "")));
|
||||
final ExprEval<?> result = eval(
|
||||
"icontains_string(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
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.math.expr.InputBindings;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -37,20 +37,23 @@ public class ContainsExprMacroTest extends MacroTestBase
|
|||
public void testErrorZeroArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[contains_string] requires 2 arguments");
|
||||
eval("contains_string()", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("contains_string()", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testErrorThreeArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[contains_string] requires 2 arguments");
|
||||
eval("contains_string('a', 'b', 'c')", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("contains_string('a', 'b', 'c')", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("contains_string(a, 'oba')", InputBindings.withMap(ImmutableMap.of("a", "foobar")));
|
||||
final ExprEval<?> result = eval(
|
||||
"contains_string(a, 'oba')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foobar")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -60,7 +63,10 @@ public class ContainsExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testNoMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("contains_string(a, 'bar')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"contains_string(a, 'bar')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(false).value(),
|
||||
result.value()
|
||||
|
@ -74,7 +80,10 @@ public class ContainsExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[contains_string] substring must be a string literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("contains_string(a, null)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"contains_string(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -84,7 +93,10 @@ public class ContainsExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testEmptyStringSearch()
|
||||
{
|
||||
final ExprEval<?> result = eval("contains_string(a, '')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"contains_string(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -98,7 +110,10 @@ public class ContainsExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[contains_string] substring must be a string literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("contains_string(a, null)", InputBindings.withMap(ImmutableMap.of("a", "")));
|
||||
final ExprEval<?> result = eval(
|
||||
"contains_string(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -108,7 +123,10 @@ public class ContainsExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testEmptyStringSearchOnEmptyString()
|
||||
{
|
||||
final ExprEval<?> result = eval("contains_string(a, '')", InputBindings.withMap(ImmutableMap.of("a", "")));
|
||||
final ExprEval<?> result = eval(
|
||||
"contains_string(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.math.expr.InputBindings;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
import org.junit.Assert;
|
||||
|
@ -35,22 +36,32 @@ public class ExprMacroTest
|
|||
{
|
||||
private static final String IPV4_STRING = "192.168.0.1";
|
||||
private static final long IPV4_LONG = 3232235521L;
|
||||
private static final Expr.ObjectBinding BINDINGS = InputBindings.withMap(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("t", DateTimes.of("2000-02-03T04:05:06").getMillis())
|
||||
.put("t1", DateTimes.of("2000-02-03T00:00:00").getMillis())
|
||||
.put("tstr", "2000-02-03T04:05:06")
|
||||
.put("tstr_sql", "2000-02-03 04:05:06")
|
||||
.put("x", "foo")
|
||||
.put("y", 2)
|
||||
.put("z", 3.1)
|
||||
.put("CityOfAngels", "America/Los_Angeles")
|
||||
.put("spacey", " hey there ")
|
||||
.put("ipv4_string", IPV4_STRING)
|
||||
.put("ipv4_long", IPV4_LONG)
|
||||
.put("ipv4_network", "192.168.0.0")
|
||||
.put("ipv4_broadcast", "192.168.255.255")
|
||||
.build()
|
||||
private static final Expr.ObjectBinding BINDINGS = InputBindings.forInputSuppliers(
|
||||
ImmutableMap.<String, InputBindings.InputSupplier>builder()
|
||||
.put("t",
|
||||
InputBindings.inputSupplier(
|
||||
ExpressionType.LONG,
|
||||
() -> DateTimes.of("2000-02-03T04:05:06").getMillis()
|
||||
)
|
||||
)
|
||||
.put("t1",
|
||||
InputBindings.inputSupplier(
|
||||
ExpressionType.LONG,
|
||||
() -> DateTimes.of("2000-02-03T00:00:00").getMillis()
|
||||
)
|
||||
)
|
||||
.put("tstr", InputBindings.inputSupplier(ExpressionType.STRING, () -> "2000-02-03T04:05:06"))
|
||||
.put("tstr_sql", InputBindings.inputSupplier(ExpressionType.STRING, () -> "2000-02-03 04:05:06"))
|
||||
.put("x", InputBindings.inputSupplier(ExpressionType.STRING, () -> "foo"))
|
||||
.put("y", InputBindings.inputSupplier(ExpressionType.LONG, () -> 2))
|
||||
.put("z", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> 3.1))
|
||||
.put("CityOfAngels", InputBindings.inputSupplier(ExpressionType.STRING, () -> "America/Los_Angeles"))
|
||||
.put("spacey", InputBindings.inputSupplier(ExpressionType.STRING, () -> " hey there "))
|
||||
.put("ipv4_string", InputBindings.inputSupplier(ExpressionType.STRING, () -> IPV4_STRING))
|
||||
.put("ipv4_long", InputBindings.inputSupplier(ExpressionType.LONG, () -> IPV4_LONG))
|
||||
.put("ipv4_network", InputBindings.inputSupplier(ExpressionType.STRING, () -> "192.168.0.0"))
|
||||
.put("ipv4_broadcast", InputBindings.inputSupplier(ExpressionType.STRING, () -> "192.168.255.255"))
|
||||
.build()
|
||||
);
|
||||
|
||||
@BeforeClass
|
||||
|
|
|
@ -19,13 +19,11 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.hll.HyperLogLogCollector;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -53,15 +51,15 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
|
|||
private static final long SOME_LONG = 1234L;
|
||||
private static final double SOME_DOUBLE = 1.234;
|
||||
|
||||
Expr.ObjectBinding inputBindings = InputBindings.withTypedSuppliers(
|
||||
new ImmutableMap.Builder<String, Pair<ExpressionType, Supplier<Object>>>()
|
||||
.put("hll", new Pair<>(HyperUniqueExpressions.TYPE, HyperLogLogCollector::makeLatestCollector))
|
||||
.put("string", new Pair<>(ExpressionType.STRING, () -> SOME_STRING))
|
||||
.put("long", new Pair<>(ExpressionType.LONG, () -> SOME_LONG))
|
||||
.put("double", new Pair<>(ExpressionType.DOUBLE, () -> SOME_DOUBLE))
|
||||
.put("nullString", new Pair<>(ExpressionType.STRING, () -> null))
|
||||
.put("nullLong", new Pair<>(ExpressionType.LONG, () -> null))
|
||||
.put("nullDouble", new Pair<>(ExpressionType.DOUBLE, () -> null))
|
||||
Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers(
|
||||
new ImmutableMap.Builder<String, InputBindings.InputSupplier>()
|
||||
.put("hll", InputBindings.inputSupplier(HyperUniqueExpressions.TYPE, HyperLogLogCollector::makeLatestCollector))
|
||||
.put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> SOME_STRING))
|
||||
.put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> SOME_LONG))
|
||||
.put("double", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> SOME_DOUBLE))
|
||||
.put("nullString", InputBindings.inputSupplier(ExpressionType.STRING, () -> null))
|
||||
.put("nullLong", InputBindings.inputSupplier(ExpressionType.LONG, () -> null))
|
||||
.put("nullDouble", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> null))
|
||||
.build()
|
||||
);
|
||||
|
||||
|
|
|
@ -21,11 +21,9 @@ package org.apache.druid.query.expression;
|
|||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -66,17 +64,17 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
"y", ImmutableMap.of("a", "hello", "b", "world")
|
||||
);
|
||||
|
||||
Expr.ObjectBinding inputBindings = InputBindings.withTypedSuppliers(
|
||||
new ImmutableMap.Builder<String, Pair<ExpressionType, Supplier<Object>>>()
|
||||
.put("nest", new Pair<>(NestedDataExpressions.TYPE, () -> NEST))
|
||||
.put("nestWrapped", new Pair<>(NestedDataExpressions.TYPE, () -> new StructuredData(NEST)))
|
||||
.put("nester", new Pair<>(NestedDataExpressions.TYPE, () -> NESTER))
|
||||
.put("string", new Pair<>(ExpressionType.STRING, () -> "abcdef"))
|
||||
.put("long", new Pair<>(ExpressionType.LONG, () -> 1234L))
|
||||
.put("double", new Pair<>(ExpressionType.DOUBLE, () -> 1.234))
|
||||
.put("nullString", new Pair<>(ExpressionType.STRING, () -> null))
|
||||
.put("nullLong", new Pair<>(ExpressionType.LONG, () -> null))
|
||||
.put("nullDouble", new Pair<>(ExpressionType.DOUBLE, () -> null))
|
||||
Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers(
|
||||
new ImmutableMap.Builder<String, InputBindings.InputSupplier>()
|
||||
.put("nest", InputBindings.inputSupplier(NestedDataExpressions.TYPE, () -> NEST))
|
||||
.put("nestWrapped", InputBindings.inputSupplier(NestedDataExpressions.TYPE, () -> new StructuredData(NEST)))
|
||||
.put("nester", InputBindings.inputSupplier(NestedDataExpressions.TYPE, () -> NESTER))
|
||||
.put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> "abcdef"))
|
||||
.put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> 1234L))
|
||||
.put("double", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> 1.234))
|
||||
.put("nullString", InputBindings.inputSupplier(ExpressionType.STRING, () -> null))
|
||||
.put("nullLong", InputBindings.inputSupplier(ExpressionType.LONG, () -> null))
|
||||
.put("nullDouble", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> null))
|
||||
.build()
|
||||
);
|
||||
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
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.math.expr.InputBindings;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -37,55 +37,76 @@ public class RegexpExtractExprMacroTest extends MacroTestBase
|
|||
public void testErrorZeroArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[regexp_extract] requires 2 or 3 arguments");
|
||||
eval("regexp_extract()", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("regexp_extract()", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testErrorFourArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[regexp_extract] requires 2 or 3 arguments");
|
||||
eval("regexp_extract('a', 'b', 'c', 'd')", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("regexp_extract('a', 'b', 'c', 'd')", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_extract(a, 'f(.o)')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, 'f(.o)')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals("foo", result.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchGroup0()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_extract(a, 'f(.o)', 0)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, 'f(.o)', 0)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals("foo", result.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchGroup1()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_extract(a, 'f(.o)', 1)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, 'f(.o)', 1)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals("oo", result.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchGroup2()
|
||||
{
|
||||
expectedException.expectMessage("No group 2");
|
||||
final ExprEval<?> result = eval("regexp_extract(a, 'f(.o)', 2)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
Throwable t = Assert.assertThrows(
|
||||
IndexOutOfBoundsException.class,
|
||||
() -> eval(
|
||||
"regexp_extract(a, 'f(.o)', 2)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
)
|
||||
);
|
||||
Assert.assertEquals("No group 2", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_extract(a, 'f(.x)')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, 'f(.x)')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertNull(result.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchInMiddle()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_extract(a, '.o$')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, '.o$')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals("oo", result.value());
|
||||
}
|
||||
|
||||
|
@ -96,14 +117,20 @@ public class RegexpExtractExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("regexp_extract(a, null)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertNull(result.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyStringPattern()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_extract(a, '')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_extract(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(NullHandling.emptyToNullIfNeeded(""), result.value());
|
||||
}
|
||||
|
||||
|
@ -111,14 +138,14 @@ public class RegexpExtractExprMacroTest extends MacroTestBase
|
|||
public void testNumericPattern()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal");
|
||||
eval("regexp_extract(a, 1)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
eval("regexp_extract(a, 1)", InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonLiteralPattern()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal");
|
||||
eval("regexp_extract(a, a)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
eval("regexp_extract(a, a)", InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
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.math.expr.InputBindings;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -37,20 +37,23 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
|
|||
public void testErrorZeroArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[regexp_like] requires 2 arguments");
|
||||
eval("regexp_like()", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("regexp_like()", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testErrorThreeArguments()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "Function[regexp_like] requires 2 arguments");
|
||||
eval("regexp_like('a', 'b', 'c')", InputBindings.withMap(ImmutableMap.of()));
|
||||
eval("regexp_like('a', 'b', 'c')", InputBindings.nilBindings());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_like(a, 'f.o')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_like(a, 'f.o')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -60,7 +63,10 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testNoMatch()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_like(a, 'f.x')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_like(a, 'f.x')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(false).value(),
|
||||
result.value()
|
||||
|
@ -74,7 +80,10 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a STRING literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("regexp_like(a, null)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_like(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -84,7 +93,10 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testEmptyStringPattern()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_like(a, '')", InputBindings.withMap(ImmutableMap.of("a", "foo")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_like(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -98,7 +110,10 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
|
|||
expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a STRING literal");
|
||||
}
|
||||
|
||||
final ExprEval<?> result = eval("regexp_like(a, null)", InputBindings.withMap(ImmutableMap.of("a", "")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_like(a, null)",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
@ -108,7 +123,10 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
|
|||
@Test
|
||||
public void testEmptyStringPatternOnEmptyString()
|
||||
{
|
||||
final ExprEval<?> result = eval("regexp_like(a, '')", InputBindings.withMap(ImmutableMap.of("a", "")));
|
||||
final ExprEval<?> result = eval(
|
||||
"regexp_like(a, '')",
|
||||
InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprEval.ofLongBoolean(true).value(),
|
||||
result.value()
|
||||
|
|
|
@ -321,4 +321,126 @@ public class TransformerTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals("val1", actual.getInputRows().get(0).getRaw("dim"));
|
||||
Assert.assertEquals("val1", actual.getRawValuesList().get(0).get("dim"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithArrayStringInputsExpr()
|
||||
{
|
||||
final Transformer transformer = new Transformer(
|
||||
new TransformSpec(
|
||||
null,
|
||||
ImmutableList.of(new ExpressionTransform("dimlen", "array_length(dim)", TestExprMacroTable.INSTANCE))
|
||||
)
|
||||
);
|
||||
final InputRow row = new MapBasedInputRow(
|
||||
DateTimes.nowUtc(),
|
||||
ImmutableList.of("dim"),
|
||||
ImmutableMap.of("dim", ImmutableList.of("a", "b", "c"))
|
||||
);
|
||||
final InputRow actual = transformer.transform(row);
|
||||
Assert.assertNotNull(actual);
|
||||
Assert.assertEquals(ImmutableList.of("dim"), actual.getDimensions());
|
||||
Assert.assertEquals(3L, actual.getRaw("dimlen"));
|
||||
Assert.assertEquals(ImmutableList.of("3"), actual.getDimension("dimlen"));
|
||||
Assert.assertEquals(row.getTimestamp(), actual.getTimestamp());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithArrayStringInputs()
|
||||
{
|
||||
final Transformer transformer = new Transformer(
|
||||
new TransformSpec(
|
||||
null,
|
||||
ImmutableList.of(new ExpressionTransform("dim", "dim", TestExprMacroTable.INSTANCE))
|
||||
)
|
||||
);
|
||||
final InputRow row = new MapBasedInputRow(
|
||||
DateTimes.nowUtc(),
|
||||
ImmutableList.of("dim"),
|
||||
ImmutableMap.of("dim", ImmutableList.of("a", "b", "c"))
|
||||
);
|
||||
final InputRow actual = transformer.transform(row);
|
||||
Assert.assertNotNull(actual);
|
||||
Assert.assertEquals(ImmutableList.of("dim"), actual.getDimensions());
|
||||
Assert.assertArrayEquals(new Object[]{"a", "b", "c"}, (Object[]) actual.getRaw("dim"));
|
||||
Assert.assertEquals(ImmutableList.of("a", "b", "c"), actual.getDimension("dim"));
|
||||
Assert.assertEquals(row.getTimestamp(), actual.getTimestamp());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithArrayLongInputs()
|
||||
{
|
||||
final Transformer transformer = new Transformer(
|
||||
new TransformSpec(
|
||||
null,
|
||||
ImmutableList.of(new ExpressionTransform("dim", "dim", TestExprMacroTable.INSTANCE))
|
||||
)
|
||||
);
|
||||
final InputRow row = new MapBasedInputRow(
|
||||
DateTimes.nowUtc(),
|
||||
ImmutableList.of("dim"),
|
||||
ImmutableMap.of("dim", Arrays.asList(1, 2, null, 3))
|
||||
);
|
||||
final InputRow actual = transformer.transform(row);
|
||||
Assert.assertNotNull(actual);
|
||||
Assert.assertEquals(ImmutableList.of("dim"), actual.getDimensions());
|
||||
Assert.assertArrayEquals(new Object[]{1L, 2L, null, 3L}, (Object[]) actual.getRaw("dim"));
|
||||
Assert.assertEquals(Arrays.asList("1", "2", "null", "3"), actual.getDimension("dim"));
|
||||
Assert.assertEquals(row.getTimestamp(), actual.getTimestamp());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithArrayFloatInputs()
|
||||
{
|
||||
final Transformer transformer = new Transformer(
|
||||
new TransformSpec(
|
||||
null,
|
||||
ImmutableList.of(new ExpressionTransform("dim", "dim", TestExprMacroTable.INSTANCE))
|
||||
)
|
||||
);
|
||||
final InputRow row = new MapBasedInputRow(
|
||||
DateTimes.nowUtc(),
|
||||
ImmutableList.of("dim"),
|
||||
ImmutableMap.of("dim", Arrays.asList(1.2f, 2.3f, null, 3.4f))
|
||||
);
|
||||
final InputRow actual = transformer.transform(row);
|
||||
Assert.assertNotNull(actual);
|
||||
Assert.assertEquals(ImmutableList.of("dim"), actual.getDimensions());
|
||||
Object[] raw = (Object[]) actual.getRaw("dim");
|
||||
// floats are converted to doubles since expressions have no doubles
|
||||
Assert.assertEquals(1.2, (Double) raw[0], 0.00001);
|
||||
Assert.assertEquals(2.3, (Double) raw[1], 0.00001);
|
||||
Assert.assertNull(raw[2]);
|
||||
Assert.assertEquals(3.4, (Double) raw[3], 0.00001);
|
||||
Assert.assertEquals(
|
||||
Arrays.asList("1.2000000476837158", "2.299999952316284", "null", "3.4000000953674316"),
|
||||
actual.getDimension("dim")
|
||||
);
|
||||
Assert.assertEquals(row.getTimestamp(), actual.getTimestamp());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithArrayDoubleInputs()
|
||||
{
|
||||
final Transformer transformer = new Transformer(
|
||||
new TransformSpec(
|
||||
null,
|
||||
ImmutableList.of(new ExpressionTransform("dim", "dim", TestExprMacroTable.INSTANCE))
|
||||
)
|
||||
);
|
||||
final InputRow row = new MapBasedInputRow(
|
||||
DateTimes.nowUtc(),
|
||||
ImmutableList.of("dim"),
|
||||
ImmutableMap.of("dim", Arrays.asList(1.2, 2.3, null, 3.4))
|
||||
);
|
||||
final InputRow actual = transformer.transform(row);
|
||||
Assert.assertNotNull(actual);
|
||||
Assert.assertEquals(ImmutableList.of("dim"), actual.getDimensions());
|
||||
Object[] raw = (Object[]) actual.getRaw("dim");
|
||||
Assert.assertEquals(1.2, (Double) raw[0], 0.0);
|
||||
Assert.assertEquals(2.3, (Double) raw[1], 0.0);
|
||||
Assert.assertNull(raw[2]);
|
||||
Assert.assertEquals(3.4, (Double) raw[3], 0.0);
|
||||
Assert.assertEquals(Arrays.asList("1.2", "2.3", "null", "3.4"), actual.getDimension("dim"));
|
||||
Assert.assertEquals(row.getTimestamp(), actual.getTimestamp());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.query.expression;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.math.expr.InputBindings;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -32,9 +33,9 @@ import org.junit.rules.ExpectedException;
|
|||
|
||||
public class LookupExprMacroTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final Expr.ObjectBinding BINDINGS = InputBindings.withMap(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("x", "foo")
|
||||
private static final Expr.ObjectBinding BINDINGS = InputBindings.forInputSuppliers(
|
||||
ImmutableMap.<String, InputBindings.InputSupplier>builder()
|
||||
.put("x", InputBindings.inputSupplier(ExpressionType.STRING, () -> "foo"))
|
||||
.build()
|
||||
);
|
||||
|
||||
|
|
|
@ -76,14 +76,7 @@ public class DruidRexExecutor implements RexExecutor
|
|||
plannerContext.getPlannerToolbox().exprMacroTable()
|
||||
);
|
||||
|
||||
final ExprEval exprResult = expr.eval(
|
||||
InputBindings.forFunction(
|
||||
name -> {
|
||||
// Sanity check. Bindings should not be used for a constant expression.
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
)
|
||||
);
|
||||
final ExprEval exprResult = expr.eval(InputBindings.validateConstant(expr));
|
||||
|
||||
final RexNode literal;
|
||||
|
||||
|
|
|
@ -32,8 +32,9 @@ import org.apache.calcite.sql.SqlIntervalQualifier;
|
|||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.data.input.MapBasedRow;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.InputBindings;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
|
@ -67,7 +68,6 @@ import org.joda.time.DateTimeZone;
|
|||
import org.junit.Assert;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -108,6 +108,7 @@ class ExpressionTestHelper
|
|||
|
||||
private final RowSignature rowSignature;
|
||||
private final Map<String, Object> bindings;
|
||||
private final Expr.ObjectBinding expressionBindings;
|
||||
private final RelDataTypeFactory typeFactory;
|
||||
private final RexBuilder rexBuilder;
|
||||
private final RelDataType relDataType;
|
||||
|
@ -116,7 +117,22 @@ class ExpressionTestHelper
|
|||
{
|
||||
this.rowSignature = rowSignature;
|
||||
this.bindings = bindings;
|
||||
this.expressionBindings = new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
return bindings.get(name);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExpressionType getType(String name)
|
||||
{
|
||||
return rowSignature.getType(name);
|
||||
}
|
||||
};
|
||||
this.typeFactory = new JavaTypeFactoryImpl();
|
||||
this.rexBuilder = new RexBuilder(typeFactory);
|
||||
this.relDataType = RowSignatures.toRelDataType(rowSignature, typeFactory);
|
||||
|
@ -319,7 +335,7 @@ class ExpressionTestHelper
|
|||
}
|
||||
|
||||
ExprEval<?> result = Parser.parse(expression.getExpression(), PLANNER_CONTEXT.getExprMacroTable())
|
||||
.eval(InputBindings.withMap(bindings));
|
||||
.eval(expressionBindings);
|
||||
|
||||
Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value());
|
||||
}
|
||||
|
|
|
@ -1087,7 +1087,7 @@ public class ExpressionsTest extends ExpressionTestBase
|
|||
DruidExpression.ofColumn(ColumnType.LONG, "y")
|
||||
)
|
||||
),
|
||||
3.0
|
||||
3L
|
||||
);
|
||||
|
||||
testHelper.testExpression(
|
||||
|
|
Loading…
Reference in New Issue