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:
Clint Wylie 2023-03-21 23:26:53 -07:00 committed by GitHub
parent b7752a909c
commit f4392a3155
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 688 additions and 335 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1087,7 +1087,7 @@ public class ExpressionsTest extends ExpressionTestBase
DruidExpression.ofColumn(ColumnType.LONG, "y")
)
),
3.0
3L
);
testHelper.testExpression(