mirror of https://github.com/apache/druid.git
support for vectorizing expressions with non-existent inputs, more consistent type handling for non-vectorized expressions (#10499)
* support for vectorizing expressions with non-existent inputs, more consistent type handling for non-vectorized expressions * inspector * changes * more test * clean
This commit is contained in:
parent
a966de5319
commit
d0821de854
|
@ -56,10 +56,10 @@ public interface ApplyFunction
|
|||
* If this method returns true, {@link #asVectorProcessor} is expected to produce a {@link ExprVectorProcessor} which
|
||||
* can evaluate values in batches to use with vectorized query engines.
|
||||
*
|
||||
* @see Expr#canVectorize(Expr.InputBindingTypes)
|
||||
* @see Function#canVectorize(Expr.InputBindingTypes, List)
|
||||
* @see Expr#canVectorize(Expr.InputBindingInspector)
|
||||
* @see Function#canVectorize(Expr.InputBindingInspector, List)
|
||||
*/
|
||||
default boolean canVectorize(Expr.InputBindingTypes inputTypes, Expr lambda, List<Expr> args)
|
||||
default boolean canVectorize(Expr.InputBindingInspector inspector, Expr lambda, List<Expr> args)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
@ -68,10 +68,14 @@ public interface ApplyFunction
|
|||
* Builds a 'vectorized' function expression processor, that can build vectorized processors for its input values
|
||||
* using {@link Expr#buildVectorized}, for use in vectorized query engines.
|
||||
*
|
||||
* @see Expr#buildVectorized(Expr.VectorInputBindingTypes)
|
||||
* @see Function#asVectorProcessor(Expr.VectorInputBindingTypes, List)
|
||||
* @see Expr#buildVectorized(Expr.VectorInputBindingInspector)
|
||||
* @see Function#asVectorProcessor(Expr.VectorInputBindingInspector, List)
|
||||
*/
|
||||
default <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, Expr lambda, List<Expr> args)
|
||||
default <T> ExprVectorProcessor<T> asVectorProcessor(
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr lambda,
|
||||
List<Expr> args
|
||||
)
|
||||
{
|
||||
throw new UOE("%s is not vectorized", name());
|
||||
}
|
||||
|
@ -109,7 +113,7 @@ public interface ApplyFunction
|
|||
* @see Expr#getOutputType
|
||||
*/
|
||||
@Nullable
|
||||
ExprType getOutputType(Expr.InputBindingTypes inputTypes, LambdaExpr expr, List<Expr> args);
|
||||
ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List<Expr> args);
|
||||
|
||||
/**
|
||||
* Base class for "map" functions, which are a class of {@link ApplyFunction} which take a lambda function that is
|
||||
|
@ -126,9 +130,9 @@ public interface ApplyFunction
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(Expr.InputBindingTypes inputTypes, LambdaExpr expr, List<Expr> args)
|
||||
public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List<Expr> args)
|
||||
{
|
||||
return ExprType.asArrayType(expr.getOutputType(new LambdaInputBindingTypes(inputTypes, expr, args)));
|
||||
return ExprType.asArrayType(expr.getOutputType(new LambdaInputBindingInspector(inspector, expr, args)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -332,10 +336,10 @@ public interface ApplyFunction
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(Expr.InputBindingTypes inputTypes, LambdaExpr expr, List<Expr> args)
|
||||
public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List<Expr> args)
|
||||
{
|
||||
// output type is accumulator type, which is last argument
|
||||
return args.get(args.size() - 1).getOutputType(inputTypes);
|
||||
return args.get(args.size() - 1).getOutputType(inspector);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -535,10 +539,10 @@ public interface ApplyFunction
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(Expr.InputBindingTypes inputTypes, LambdaExpr expr, List<Expr> args)
|
||||
public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List<Expr> args)
|
||||
{
|
||||
// output type is input array type
|
||||
return args.get(0).getOutputType(inputTypes);
|
||||
return args.get(0).getOutputType(inspector);
|
||||
}
|
||||
|
||||
private <T> Stream<T> filter(T[] array, LambdaExpr expr, SettableLambdaBinding binding)
|
||||
|
@ -590,7 +594,7 @@ public interface ApplyFunction
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(Expr.InputBindingTypes inputTypes, LambdaExpr expr, List<Expr> args)
|
||||
public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List<Expr> args)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -917,20 +921,20 @@ public interface ApplyFunction
|
|||
}
|
||||
|
||||
/**
|
||||
* Helper that can wrap another {@link Expr.InputBindingTypes} to use to supply the type information of a
|
||||
* Helper that can wrap another {@link Expr.InputBindingInspector} to use to supply the type information of a
|
||||
* {@link LambdaExpr} when evaluating {@link ApplyFunctionExpr#getOutputType}. Lambda identifiers do not exist
|
||||
* in the underlying {@link Expr.InputBindingTypes}, but can be created by mapping the lambda identifiers to the
|
||||
* in the underlying {@link Expr.InputBindingInspector}, but can be created by mapping the lambda identifiers to the
|
||||
* arguments that will be applied to them, to map the type information.
|
||||
*/
|
||||
class LambdaInputBindingTypes implements Expr.InputBindingTypes
|
||||
class LambdaInputBindingInspector implements Expr.InputBindingInspector
|
||||
{
|
||||
private final Object2IntMap<String> lambdaIdentifiers;
|
||||
private final Expr.InputBindingTypes inputTypes;
|
||||
private final Expr.InputBindingInspector inspector;
|
||||
private final List<Expr> args;
|
||||
|
||||
public LambdaInputBindingTypes(Expr.InputBindingTypes inputTypes, LambdaExpr expr, List<Expr> args)
|
||||
public LambdaInputBindingInspector(Expr.InputBindingInspector inspector, LambdaExpr expr, List<Expr> args)
|
||||
{
|
||||
this.inputTypes = inputTypes;
|
||||
this.inspector = inspector;
|
||||
this.args = args;
|
||||
List<String> identifiers = expr.getIdentifiers();
|
||||
this.lambdaIdentifiers = new Object2IntOpenHashMap<>(args.size());
|
||||
|
@ -944,9 +948,9 @@ public interface ApplyFunction
|
|||
public ExprType getType(String name)
|
||||
{
|
||||
if (lambdaIdentifiers.containsKey(name)) {
|
||||
return ExprType.elementType(args.get(lambdaIdentifiers.getInt(name)).getOutputType(inputTypes));
|
||||
return ExprType.elementType(args.get(lambdaIdentifiers.getInt(name)).getOutputType(inspector));
|
||||
}
|
||||
return inputTypes.getType(name);
|
||||
return inspector.getType(name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -62,9 +62,9 @@ class BinLtExpr extends BinaryEvalOpExprBase
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -72,15 +72,15 @@ class BinLtExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorComparisonProcessors.lessThan(inputTypes, left, right);
|
||||
return VectorComparisonProcessors.lessThan(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -118,9 +118,9 @@ class BinLeqExpr extends BinaryEvalOpExprBase
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -128,15 +128,15 @@ class BinLeqExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorComparisonProcessors.lessThanOrEqual(inputTypes, left, right);
|
||||
return VectorComparisonProcessors.lessThanOrEqual(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -174,24 +174,24 @@ class BinGtExpr extends BinaryEvalOpExprBase
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
return implicitCast;
|
||||
}
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorComparisonProcessors.greaterThan(inputTypes, left, right);
|
||||
return VectorComparisonProcessors.greaterThan(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -229,9 +229,9 @@ class BinGeqExpr extends BinaryEvalOpExprBase
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -239,15 +239,15 @@ class BinGeqExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorComparisonProcessors.greaterThanOrEqual(inputTypes, left, right);
|
||||
return VectorComparisonProcessors.greaterThanOrEqual(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -284,9 +284,9 @@ class BinEqExpr extends BinaryEvalOpExprBase
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -294,15 +294,15 @@ class BinEqExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorComparisonProcessors.equal(inputTypes, left, right);
|
||||
return VectorComparisonProcessors.equal(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -339,9 +339,9 @@ class BinNeqExpr extends BinaryEvalOpExprBase
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -349,15 +349,15 @@ class BinNeqExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorComparisonProcessors.notEqual(inputTypes, left, right);
|
||||
return VectorComparisonProcessors.notEqual(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -61,15 +61,15 @@ final class BinPlusExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.plus(inputTypes, left, right);
|
||||
return VectorMathProcessors.plus(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -99,15 +99,15 @@ final class BinMinusExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.minus(inputTypes, left, right);
|
||||
return VectorMathProcessors.minus(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -137,15 +137,15 @@ final class BinMulExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.multiply(inputTypes, left, right);
|
||||
return VectorMathProcessors.multiply(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -175,15 +175,15 @@ final class BinDivExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.divide(inputTypes, left, right);
|
||||
return VectorMathProcessors.divide(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -213,15 +213,15 @@ class BinPowExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.power(inputTypes, left, right);
|
||||
return VectorMathProcessors.power(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -251,14 +251,14 @@ class BinModuloExpr extends BinaryEvalOpExprBase
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
|
||||
return inspector.areNumeric(left, right) && inspector.canVectorize(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.modulo(inputTypes, left, right);
|
||||
return VectorMathProcessors.modulo(inspector, left, right);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -81,15 +81,15 @@ abstract class BinaryOpExprBase implements Expr
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
if (left.isNullLiteral()) {
|
||||
return right.getOutputType(inputTypes);
|
||||
return right.getOutputType(inspector);
|
||||
}
|
||||
if (right.isNullLiteral()) {
|
||||
return left.getOutputType(inputTypes);
|
||||
return left.getOutputType(inspector);
|
||||
}
|
||||
return ExprTypeConversion.operator(left.getOutputType(inputTypes), right.getOutputType(inputTypes));
|
||||
return ExprTypeConversion.operator(left.getOutputType(inspector), right.getOutputType(inspector));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -137,18 +137,18 @@ abstract class BinaryEvalOpExprBase extends BinaryOpExprBase
|
|||
return ExprEval.of(null);
|
||||
}
|
||||
|
||||
if (leftVal.type() == ExprType.STRING && rightVal.type() == ExprType.STRING) {
|
||||
return evalString(leftVal.asString(), rightVal.asString());
|
||||
} else if (leftVal.type() == ExprType.LONG && rightVal.type() == ExprType.LONG) {
|
||||
if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(evalLong(leftVal.asLong(), rightVal.asLong()));
|
||||
} else {
|
||||
if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(evalDouble(leftVal.asDouble(), rightVal.asDouble()));
|
||||
ExprType type = ExprTypeConversion.autoDetect(leftVal, rightVal);
|
||||
switch (type) {
|
||||
case STRING:
|
||||
return evalString(leftVal.asString(), rightVal.asString());
|
||||
case LONG:
|
||||
return ExprEval.of(evalLong(leftVal.asLong(), rightVal.asLong()));
|
||||
case DOUBLE:
|
||||
default:
|
||||
if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(evalDouble(leftVal.asDouble(), rightVal.asDouble()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ abstract class ConstantExpr implements Expr
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return outputType;
|
||||
}
|
||||
|
@ -135,15 +135,15 @@ class LongExpr extends ConstantExpr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorProcessors.constantLong(value, inputTypes.getMaxVectorSize());
|
||||
return VectorProcessors.constantLong(value, inspector.getMaxVectorSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -180,15 +180,15 @@ class NullLongExpr extends NullNumericConstantExpr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorProcessors.constantLong(null, inputTypes.getMaxVectorSize());
|
||||
return VectorProcessors.constantLong(null, inspector.getMaxVectorSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -290,15 +290,15 @@ class DoubleExpr extends ConstantExpr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorProcessors.constantDouble(value, inputTypes.getMaxVectorSize());
|
||||
return VectorProcessors.constantDouble(value, inspector.getMaxVectorSize());
|
||||
}
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
|
@ -334,15 +334,15 @@ class NullDoubleExpr extends NullNumericConstantExpr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorProcessors.constantDouble(null, inputTypes.getMaxVectorSize());
|
||||
return VectorProcessors.constantDouble(null, inspector.getMaxVectorSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -452,15 +452,15 @@ class StringExpr extends ConstantExpr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorProcessors.constantString(value, inputTypes.getMaxVectorSize());
|
||||
return VectorProcessors.constantString(value, inspector.getMaxVectorSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -132,13 +132,14 @@ public interface Expr
|
|||
BindingAnalysis analyzeInputs();
|
||||
|
||||
/**
|
||||
* Given an {@link InputBindingTypes}, compute what the output {@link ExprType} will be for this expression. A return
|
||||
* Given an {@link InputBindingInspector}, compute what the output {@link ExprType} will be for this expression. A return
|
||||
* value of null indicates that the given type information was not enough to resolve the output type, so the
|
||||
* expression must be evaluated using default {@link #eval} handling where types are only known after evaluation,
|
||||
* through {@link ExprEval#type}.
|
||||
* @param inspector
|
||||
*/
|
||||
@Nullable
|
||||
default ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
default ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
@ -147,8 +148,9 @@ public interface Expr
|
|||
* Check if an expression can be 'vectorized', for a given set of inputs. If this method returns true,
|
||||
* {@link #buildVectorized} is expected to produce a {@link ExprVectorProcessor} which can evaluate values in batches
|
||||
* to use with vectorized query engines.
|
||||
* @param inspector
|
||||
*/
|
||||
default boolean canVectorize(InputBindingTypes inputTypes)
|
||||
default boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
@ -156,8 +158,9 @@ public interface Expr
|
|||
/**
|
||||
* Builds a 'vectorized' expression processor, that can operate on batches of input values for use in vectorized
|
||||
* query engines.
|
||||
* @param inspector
|
||||
*/
|
||||
default <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
default <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
throw Exprs.cannotVectorize(this);
|
||||
}
|
||||
|
@ -167,8 +170,12 @@ public interface Expr
|
|||
* inferring the output type of an expression with {@link #getOutputType}. A null value means that either the binding
|
||||
* doesn't exist, or, that the type information is unavailable.
|
||||
*/
|
||||
interface InputBindingTypes
|
||||
interface InputBindingInspector
|
||||
{
|
||||
/**
|
||||
* Get the {@link ExprType} from the backing store for a given identifier (this is likely a column, but could be other
|
||||
* things depending on the backing adapter)
|
||||
*/
|
||||
@Nullable
|
||||
ExprType getType(String name);
|
||||
|
||||
|
@ -179,12 +186,11 @@ public interface Expr
|
|||
*/
|
||||
default boolean areNumeric(List<Expr> args)
|
||||
{
|
||||
boolean numeric = args.size() > 0;
|
||||
boolean numeric = true;
|
||||
for (Expr arg : args) {
|
||||
ExprType argType = arg.getOutputType(this);
|
||||
if (argType == null) {
|
||||
numeric = false;
|
||||
break;
|
||||
continue;
|
||||
}
|
||||
numeric &= argType.isNumeric();
|
||||
}
|
||||
|
@ -202,7 +208,7 @@ public interface Expr
|
|||
}
|
||||
|
||||
/**
|
||||
* Check if every provided {@link Expr} computes {@link Expr#canVectorize(InputBindingTypes)} to a value of true
|
||||
* Check if every provided {@link Expr} computes {@link Expr#canVectorize(InputBindingInspector)} to a value of true
|
||||
*/
|
||||
default boolean canVectorize(List<Expr> args)
|
||||
{
|
||||
|
@ -214,7 +220,7 @@ public interface Expr
|
|||
}
|
||||
|
||||
/**
|
||||
* Check if every provided {@link Expr} computes {@link Expr#canVectorize(InputBindingTypes)} to a value of true
|
||||
* Check if every provided {@link Expr} computes {@link Expr#canVectorize(InputBindingInspector)} to a value of true
|
||||
*/
|
||||
default boolean canVectorize(Expr... args)
|
||||
{
|
||||
|
@ -223,9 +229,9 @@ public interface Expr
|
|||
}
|
||||
|
||||
/**
|
||||
* {@link InputBindingTypes} + vectorizations stuff for {@link #buildVectorized}
|
||||
* {@link InputBindingInspector} + vectorizations stuff for {@link #buildVectorized}
|
||||
*/
|
||||
interface VectorInputBindingTypes extends InputBindingTypes
|
||||
interface VectorInputBindingInspector extends InputBindingInspector
|
||||
{
|
||||
int getMaxVectorSize();
|
||||
}
|
||||
|
@ -247,7 +253,7 @@ public interface Expr
|
|||
* the vectorized column selector interfaces, and includes {@link ExprType} information about all input bindings
|
||||
* which exist
|
||||
*/
|
||||
interface VectorInputBinding extends VectorInputBindingTypes
|
||||
interface VectorInputBinding extends VectorInputBindingInspector
|
||||
{
|
||||
<T> T[] getObjectVector(String name);
|
||||
|
||||
|
|
|
@ -126,7 +126,7 @@ public enum ExprType
|
|||
}
|
||||
}
|
||||
|
||||
public static boolean isNumeric(ExprType type)
|
||||
public static boolean isNumeric(@Nullable ExprType type)
|
||||
{
|
||||
return LONG.equals(type) || DOUBLE.equals(type);
|
||||
}
|
||||
|
|
|
@ -31,7 +31,7 @@ public class ExprTypeConversion
|
|||
* Infer the output type of a list of possible 'conditional' expression outputs (where any of these could be the
|
||||
* output expression if the corresponding case matching expression evaluates to true)
|
||||
*/
|
||||
static ExprType conditional(Expr.InputBindingTypes inputTypes, List<Expr> args)
|
||||
static ExprType conditional(Expr.InputBindingInspector inspector, List<Expr> args)
|
||||
{
|
||||
ExprType type = null;
|
||||
for (Expr arg : args) {
|
||||
|
@ -39,32 +39,52 @@ public class ExprTypeConversion
|
|||
continue;
|
||||
}
|
||||
if (type == null) {
|
||||
type = arg.getOutputType(inputTypes);
|
||||
type = arg.getOutputType(inspector);
|
||||
} else {
|
||||
type = doubleMathFunction(type, arg.getOutputType(inputTypes));
|
||||
type = function(type, arg.getOutputType(inspector));
|
||||
}
|
||||
}
|
||||
return type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given 2 'input' types, which might not be fully trustable, choose the most appropriate combined type for
|
||||
* non-vectorized, per-row type detection. In this mode, null values are {@link ExprType#STRING} typed, despite
|
||||
* potentially coming from an underlying numeric column, or when an underlying column was completely missing and so
|
||||
* all values are null. This method is not well suited for array handling.
|
||||
*/
|
||||
public static ExprType autoDetect(ExprEval eval, ExprEval otherEval)
|
||||
{
|
||||
ExprType type = eval.type();
|
||||
ExprType otherType = otherEval.type();
|
||||
if (type == ExprType.STRING && otherType == ExprType.STRING) {
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
||||
type = eval.value() != null ? type : otherType;
|
||||
otherType = otherEval.value() != null ? otherType : type;
|
||||
return numeric(type, otherType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Given 2 'input' types, choose the most appropriate combined type, if possible
|
||||
*
|
||||
* arrays must be the same type
|
||||
* if both types are {@link ExprType#STRING}, the output type will be preserved as string
|
||||
* if both types are {@link ExprType#LONG}, the output type will be preserved as long
|
||||
*
|
||||
* otherwise, output is {@link ExprType#DOUBLE}
|
||||
*/
|
||||
@Nullable
|
||||
public static ExprType operator(@Nullable ExprType type, @Nullable ExprType other)
|
||||
{
|
||||
if (type == null || other == null) {
|
||||
// cannot auto conversion unknown types
|
||||
return null;
|
||||
if (type == null) {
|
||||
return other;
|
||||
}
|
||||
if (other == null) {
|
||||
return type;
|
||||
}
|
||||
// arrays cannot be auto converted
|
||||
if (ExprType.isArray(type) || ExprType.isArray(other)) {
|
||||
if (!type.equals(other)) {
|
||||
if (type != other) {
|
||||
throw new IAE("Cannot implicitly cast %s to %s", type, other);
|
||||
}
|
||||
return type;
|
||||
|
@ -87,15 +107,17 @@ public class ExprTypeConversion
|
|||
* {@link ExprType#DOUBLE}
|
||||
*/
|
||||
@Nullable
|
||||
public static ExprType doubleMathFunction(@Nullable ExprType type, @Nullable ExprType other)
|
||||
public static ExprType function(@Nullable ExprType type, @Nullable ExprType other)
|
||||
{
|
||||
if (type == null || other == null) {
|
||||
// cannot auto conversion unknown types
|
||||
return null;
|
||||
if (type == null) {
|
||||
type = other;
|
||||
}
|
||||
if (other == null) {
|
||||
other = type;
|
||||
}
|
||||
// arrays cannot be auto converted
|
||||
if (ExprType.isArray(type) || ExprType.isArray(other)) {
|
||||
if (!type.equals(other)) {
|
||||
if (type != other) {
|
||||
throw new IAE("Cannot implicitly cast %s to %s", type, other);
|
||||
}
|
||||
return type;
|
||||
|
@ -108,6 +130,7 @@ public class ExprTypeConversion
|
|||
return numeric(type, other);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Given 2 'input' types, choose the most appropriate combined type, if possible
|
||||
*
|
||||
|
@ -118,37 +141,22 @@ public class ExprTypeConversion
|
|||
@Nullable
|
||||
public static ExprType integerMathFunction(@Nullable ExprType type, @Nullable ExprType other)
|
||||
{
|
||||
if (type == null || other == null) {
|
||||
// cannot auto conversion unknown types
|
||||
return null;
|
||||
}
|
||||
// arrays cannot be auto converted
|
||||
if (ExprType.isArray(type) || ExprType.isArray(other)) {
|
||||
if (!type.equals(other)) {
|
||||
throw new IAE("Cannot implicitly cast %s to %s", type, other);
|
||||
}
|
||||
return type;
|
||||
}
|
||||
// if either argument is a string, type becomes a string
|
||||
if (ExprType.STRING.equals(type) || ExprType.STRING.equals(other)) {
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
||||
final ExprType functionType = ExprTypeConversion.function(type, other);
|
||||
// any number is long
|
||||
return ExprType.LONG;
|
||||
return ExprType.isNumeric(functionType) ? ExprType.LONG : functionType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Default best effort numeric type conversion. If both types are {@link ExprType#LONG}, returns
|
||||
* {@link ExprType#LONG}, else {@link ExprType#DOUBLE}
|
||||
*/
|
||||
public static ExprType numeric(ExprType type, ExprType other)
|
||||
public static ExprType numeric(@Nullable ExprType type, @Nullable ExprType other)
|
||||
{
|
||||
// all numbers win over longs
|
||||
// floats vs doubles would be handled here, but we currently only support doubles...
|
||||
if (ExprType.LONG.equals(type) && ExprType.LONG.equals(other)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
// floats vs doubles would be handled here, but we currently only support doubles...
|
||||
return ExprType.DOUBLE;
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -78,15 +78,15 @@ class LambdaExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return expr.canVectorize(inputTypes);
|
||||
return expr.canVectorize(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return expr.buildVectorized(inputTypes);
|
||||
return expr.buildVectorized(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -119,9 +119,9 @@ class LambdaExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return expr.getOutputType(inputTypes);
|
||||
return expr.getOutputType(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -177,15 +177,15 @@ class FunctionExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return function.canVectorize(inputTypes, args);
|
||||
return function.canVectorize(inspector, args);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprVectorProcessor<?> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public ExprVectorProcessor<?> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return function.asVectorProcessor(inputTypes, args);
|
||||
return function.asVectorProcessor(inspector, args);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -216,9 +216,9 @@ class FunctionExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return function.getOutputType(inputTypes, args);
|
||||
return function.getOutputType(inspector, args);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -298,17 +298,17 @@ class ApplyFunctionExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return function.canVectorize(inputTypes, lambdaExpr, argsExpr) &&
|
||||
lambdaExpr.canVectorize(inputTypes) &&
|
||||
argsExpr.stream().allMatch(expr -> expr.canVectorize(inputTypes));
|
||||
return function.canVectorize(inspector, lambdaExpr, argsExpr) &&
|
||||
lambdaExpr.canVectorize(inspector) &&
|
||||
argsExpr.stream().allMatch(expr -> expr.canVectorize(inspector));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return function.asVectorProcessor(inputTypes, lambdaExpr, argsExpr);
|
||||
return function.asVectorProcessor(inspector, lambdaExpr, argsExpr);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -338,9 +338,9 @@ class ApplyFunctionExpr implements Expr
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return function.getOutputType(inputTypes, lambdaExpr, argsExpr);
|
||||
return function.getOutputType(inspector, lambdaExpr, argsExpr);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -113,9 +113,9 @@ class IdentifierExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.getType(binding);
|
||||
return inspector.getType(binding);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -138,18 +138,26 @@ class IdentifierExpr implements Expr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.getType(binding) != null;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprVectorProcessor<?> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public ExprVectorProcessor<?> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
ExprType inputType = inputTypes.getType(binding);
|
||||
ExprType inputType = inspector.getType(binding);
|
||||
|
||||
if (inputType == null) {
|
||||
throw Exprs.cannotVectorize(this);
|
||||
// nil column, we can be anything, why not be a double
|
||||
return new IdentifierVectorProcessor<double[]>(ExprType.DOUBLE)
|
||||
{
|
||||
@Override
|
||||
public ExprEvalVector<double[]> evalVector(VectorInputBinding bindings)
|
||||
{
|
||||
return new ExprEvalDoubleVector(bindings.getDoubleVector(binding), bindings.getNullVector(binding));
|
||||
}
|
||||
};
|
||||
}
|
||||
switch (inputType) {
|
||||
case LONG:
|
||||
|
|
|
@ -65,9 +65,9 @@ abstract class UnaryExpr implements Expr
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return expr.getOutputType(inputTypes);
|
||||
return expr.getOutputType(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -132,15 +132,15 @@ class UnaryMinusExpr extends UnaryExpr
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return inputTypes.areNumeric(expr) && expr.canVectorize(inputTypes);
|
||||
return inspector.areNumeric(expr) && expr.canVectorize(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
return VectorMathProcessors.negate(inputTypes, expr);
|
||||
return VectorMathProcessors.negate(inspector, expr);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -171,9 +171,9 @@ class UnaryNotExpr extends UnaryExpr
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
ExprType implicitCast = super.getOutputType(inputTypes);
|
||||
ExprType implicitCast = super.getOutputType(inspector);
|
||||
if (ExprType.STRING.equals(implicitCast)) {
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -31,7 +31,7 @@ public abstract class CastToTypeVectorProcessor<TOutput> implements ExprVectorPr
|
|||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> castToType(ExprVectorProcessor<?> delegate, ExprType type)
|
||||
public static <T> ExprVectorProcessor<T> cast(ExprVectorProcessor<?> delegate, ExprType type)
|
||||
{
|
||||
final ExprVectorProcessor<?> caster;
|
||||
if (delegate.getOutputType() == type) {
|
||||
|
|
|
@ -29,7 +29,7 @@ public abstract class DoubleOutDoubleInFunctionVectorProcessor
|
|||
{
|
||||
public DoubleOutDoubleInFunctionVectorProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
|
||||
{
|
||||
super(processor, maxVectorSize, new double[maxVectorSize]);
|
||||
super(CastToTypeVectorProcessor.cast(processor, ExprType.DOUBLE), maxVectorSize, new double[maxVectorSize]);
|
||||
}
|
||||
|
||||
public abstract double apply(double input);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class DoubleOutDoubleLongInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new double[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.LONG),
|
||||
maxVectorSize,
|
||||
new double[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract double apply(double left, long right);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class DoubleOutDoublesInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new double[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE),
|
||||
maxVectorSize,
|
||||
new double[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract double apply(double left, double right);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class DoubleOutLongDoubleInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new double[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.LONG),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE),
|
||||
maxVectorSize,
|
||||
new double[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract double apply(long left, double right);
|
||||
|
|
|
@ -29,7 +29,7 @@ public abstract class DoubleOutLongInFunctionVectorProcessor
|
|||
{
|
||||
public DoubleOutLongInFunctionVectorProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
|
||||
{
|
||||
super(processor, maxVectorSize, new double[maxVectorSize]);
|
||||
super(CastToTypeVectorProcessor.cast(processor, ExprType.LONG), maxVectorSize, new double[maxVectorSize]);
|
||||
}
|
||||
|
||||
public abstract double apply(long input);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class DoubleOutLongsInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new double[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.LONG),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.LONG),
|
||||
maxVectorSize,
|
||||
new double[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract double apply(long left, long right);
|
||||
|
|
|
@ -28,7 +28,7 @@ public abstract class LongOutDoubleInFunctionVectorProcessor extends UnivariateF
|
|||
{
|
||||
public LongOutDoubleInFunctionVectorProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
|
||||
{
|
||||
super(processor, maxVectorSize, new long[maxVectorSize]);
|
||||
super(CastToTypeVectorProcessor.cast(processor, ExprType.DOUBLE), maxVectorSize, new long[maxVectorSize]);
|
||||
}
|
||||
|
||||
public abstract long apply(double input);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class LongOutDoubleLongInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new long[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.LONG),
|
||||
maxVectorSize,
|
||||
new long[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract long apply(double left, long right);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class LongOutDoublesInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new long[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE),
|
||||
maxVectorSize,
|
||||
new long[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract long apply(double left, double right);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class LongOutLongDoubleInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new long[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.LONG),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE),
|
||||
maxVectorSize,
|
||||
new long[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract long apply(long left, double right);
|
||||
|
|
|
@ -28,7 +28,7 @@ public abstract class LongOutLongInFunctionVectorProcessor extends UnivariateFun
|
|||
{
|
||||
public LongOutLongInFunctionVectorProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
|
||||
{
|
||||
super(processor, maxVectorSize, new long[maxVectorSize]);
|
||||
super(CastToTypeVectorProcessor.cast(processor, ExprType.LONG), maxVectorSize, new long[maxVectorSize]);
|
||||
}
|
||||
|
||||
public abstract long apply(long input);
|
||||
|
|
|
@ -33,7 +33,12 @@ public abstract class LongOutLongsInFunctionVectorProcessor
|
|||
int maxVectorSize
|
||||
)
|
||||
{
|
||||
super(left, right, maxVectorSize, new long[maxVectorSize]);
|
||||
super(
|
||||
CastToTypeVectorProcessor.cast(left, ExprType.LONG),
|
||||
CastToTypeVectorProcessor.cast(right, ExprType.LONG),
|
||||
maxVectorSize,
|
||||
new long[maxVectorSize]
|
||||
);
|
||||
}
|
||||
|
||||
public abstract long apply(long left, long right);
|
||||
|
|
|
@ -29,7 +29,7 @@ public abstract class LongOutStringInFunctionVectorProcessor
|
|||
{
|
||||
public LongOutStringInFunctionVectorProcessor(ExprVectorProcessor<String[]> processor, int maxVectorSize)
|
||||
{
|
||||
super(processor, maxVectorSize, new long[maxVectorSize]);
|
||||
super(CastToTypeVectorProcessor.cast(processor, ExprType.STRING), maxVectorSize, new long[maxVectorSize]);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,19 +25,19 @@ import org.apache.druid.math.expr.Expr;
|
|||
public class VectorComparisonProcessors
|
||||
{
|
||||
public static <T> ExprVectorProcessor<T> equal(
|
||||
Expr.VectorInputBindingTypes inputTypes,
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr left,
|
||||
Expr right
|
||||
)
|
||||
{
|
||||
return VectorMathProcessors.makeMathProcessor(
|
||||
inputTypes,
|
||||
inspector,
|
||||
left,
|
||||
right,
|
||||
() -> new LongOutLongsInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -47,9 +47,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -59,9 +59,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -71,9 +71,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoublesInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -86,19 +86,19 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> notEqual(
|
||||
Expr.VectorInputBindingTypes inputTypes,
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr left,
|
||||
Expr right
|
||||
)
|
||||
{
|
||||
return VectorMathProcessors.makeMathProcessor(
|
||||
inputTypes,
|
||||
inspector,
|
||||
left,
|
||||
right,
|
||||
() -> new LongOutLongsInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -108,9 +108,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -120,9 +120,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -132,9 +132,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoublesInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -147,19 +147,19 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> greaterThanOrEqual(
|
||||
Expr.VectorInputBindingTypes inputTypes,
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr left,
|
||||
Expr right
|
||||
)
|
||||
{
|
||||
return VectorMathProcessors.makeMathProcessor(
|
||||
inputTypes,
|
||||
inspector,
|
||||
left,
|
||||
right,
|
||||
() -> new LongOutLongsInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -169,9 +169,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -181,9 +181,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -193,9 +193,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoublesInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -208,19 +208,19 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> greaterThan(
|
||||
Expr.VectorInputBindingTypes inputTypes,
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr left,
|
||||
Expr right
|
||||
)
|
||||
{
|
||||
return VectorMathProcessors.makeMathProcessor(
|
||||
inputTypes,
|
||||
inspector,
|
||||
left,
|
||||
right,
|
||||
() -> new LongOutLongsInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -230,9 +230,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -242,9 +242,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -254,9 +254,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoublesInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -269,19 +269,19 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> lessThanOrEqual(
|
||||
Expr.VectorInputBindingTypes inputTypes,
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr left,
|
||||
Expr right
|
||||
)
|
||||
{
|
||||
return VectorMathProcessors.makeMathProcessor(
|
||||
inputTypes,
|
||||
inspector,
|
||||
left,
|
||||
right,
|
||||
() -> new LongOutLongsInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -291,9 +291,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -303,9 +303,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -315,9 +315,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoublesInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -330,19 +330,19 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> lessThan(
|
||||
Expr.VectorInputBindingTypes inputTypes,
|
||||
Expr.VectorInputBindingInspector inspector,
|
||||
Expr left,
|
||||
Expr right
|
||||
)
|
||||
{
|
||||
return VectorMathProcessors.makeMathProcessor(
|
||||
inputTypes,
|
||||
inspector,
|
||||
left,
|
||||
right,
|
||||
() -> new LongOutLongsInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -352,9 +352,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -364,9 +364,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -376,9 +376,9 @@ public class VectorComparisonProcessors
|
|||
}
|
||||
},
|
||||
() -> new DoubleOutDoublesInFunctionVectorProcessor(
|
||||
left.buildVectorized(inputTypes),
|
||||
right.buildVectorized(inputTypes),
|
||||
inputTypes.getMaxVectorSize()
|
||||
left.buildVectorized(inspector),
|
||||
right.buildVectorized(inspector),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -105,11 +105,11 @@ public class VectorProcessors
|
|||
};
|
||||
}
|
||||
|
||||
public static <T> ExprVectorProcessor<T> parseLong(Expr.VectorInputBindingTypes inputTypes, Expr arg, int radix)
|
||||
public static <T> ExprVectorProcessor<T> parseLong(Expr.VectorInputBindingInspector inspector, Expr arg, int radix)
|
||||
{
|
||||
final ExprVectorProcessor<?> processor = new LongOutStringInFunctionVectorProcessor(
|
||||
CastToTypeVectorProcessor.castToType(arg.buildVectorized(inputTypes), ExprType.STRING),
|
||||
inputTypes.getMaxVectorSize()
|
||||
CastToTypeVectorProcessor.cast(arg.buildVectorized(inspector), ExprType.STRING),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -31,7 +31,7 @@ import java.util.Map;
|
|||
|
||||
public class OutputTypeTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private final Expr.InputBindingTypes inputTypes = inputTypesFromMap(
|
||||
private final Expr.InputBindingInspector inspector = inspectorFromMap(
|
||||
ImmutableMap.<String, ExprType>builder().put("x", ExprType.STRING)
|
||||
.put("x_", ExprType.STRING)
|
||||
.put("y", ExprType.LONG)
|
||||
|
@ -53,352 +53,388 @@ public class OutputTypeTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testConstantsAndIdentifiers()
|
||||
{
|
||||
assertOutputType("'hello'", inputTypes, ExprType.STRING);
|
||||
assertOutputType("23", inputTypes, ExprType.LONG);
|
||||
assertOutputType("3.2", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("['a', 'b']", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("[1,2,3]", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("[1.0]", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("x", inputTypes, ExprType.STRING);
|
||||
assertOutputType("y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("a", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("b", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("c", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("'hello'", inspector, ExprType.STRING);
|
||||
assertOutputType("23", inspector, ExprType.LONG);
|
||||
assertOutputType("3.2", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("['a', 'b']", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("[1,2,3]", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("[1.0]", inspector, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("x", inspector, ExprType.STRING);
|
||||
assertOutputType("y", inspector, ExprType.LONG);
|
||||
assertOutputType("z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("a", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("b", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("c", inspector, ExprType.DOUBLE_ARRAY);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnaryOperators()
|
||||
{
|
||||
assertOutputType("-1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("-1.1", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("-y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("-z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("-1", inspector, ExprType.LONG);
|
||||
assertOutputType("-1.1", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("-y", inspector, ExprType.LONG);
|
||||
assertOutputType("-z", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("!'true'", inputTypes, ExprType.LONG);
|
||||
assertOutputType("!1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("!1.1", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("!x", inputTypes, ExprType.LONG);
|
||||
assertOutputType("!y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("!z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("!'true'", inspector, ExprType.LONG);
|
||||
assertOutputType("!1", inspector, ExprType.LONG);
|
||||
assertOutputType("!1.1", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("!x", inspector, ExprType.LONG);
|
||||
assertOutputType("!y", inspector, ExprType.LONG);
|
||||
assertOutputType("!z", inspector, ExprType.DOUBLE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBinaryMathOperators()
|
||||
{
|
||||
assertOutputType("1+1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("1-1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("1*1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("1/1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("1^1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("1%1", inputTypes, ExprType.LONG);
|
||||
assertOutputType("1+1", inspector, ExprType.LONG);
|
||||
assertOutputType("1-1", inspector, ExprType.LONG);
|
||||
assertOutputType("1*1", inspector, ExprType.LONG);
|
||||
assertOutputType("1/1", inspector, ExprType.LONG);
|
||||
assertOutputType("1^1", inspector, ExprType.LONG);
|
||||
assertOutputType("1%1", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("y+y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y-y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y*y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y/y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y^y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y%y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y+y_", inspector, ExprType.LONG);
|
||||
assertOutputType("y-y_", inspector, ExprType.LONG);
|
||||
assertOutputType("y*y_", inspector, ExprType.LONG);
|
||||
assertOutputType("y/y_", inspector, ExprType.LONG);
|
||||
assertOutputType("y^y_", inspector, ExprType.LONG);
|
||||
assertOutputType("y%y_", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("y+z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y-z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y*z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y/z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y^z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y%z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y+z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y-z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y*z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y/z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y^z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y%z", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("z+z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z-z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z*z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z/z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z^z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z%z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z+z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z-z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z*z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z/z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z^z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z%z_", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("y>y_", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_<y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_<=y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_>=y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_==y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_!=y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_ && y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y_ || y", inputTypes, ExprType.LONG);
|
||||
assertOutputType("y>y_", inspector, ExprType.LONG);
|
||||
assertOutputType("y_<y", inspector, ExprType.LONG);
|
||||
assertOutputType("y_<=y", inspector, ExprType.LONG);
|
||||
assertOutputType("y_>=y", inspector, ExprType.LONG);
|
||||
assertOutputType("y_==y", inspector, ExprType.LONG);
|
||||
assertOutputType("y_!=y", inspector, ExprType.LONG);
|
||||
assertOutputType("y_ && y", inspector, ExprType.LONG);
|
||||
assertOutputType("y_ || y", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("z>y_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z<y", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z<=y", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y>=z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z==y", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z!=y", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z && y", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("y || z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z>y_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z<y", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z<=y", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y>=z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z==y", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z!=y", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z && y", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("y || z", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("z>z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z<z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z<=z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z_>=z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z==z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z!=z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z && z_", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z_ || z", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("z>z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z<z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z<=z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z_>=z", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z==z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z!=z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z && z_", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("z_ || z", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("1*(2 + 3.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("1*(2 + 3.0)", inspector, ExprType.DOUBLE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnivariateMathFunctions()
|
||||
{
|
||||
assertOutputType("pi()", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("abs(x)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("abs(y)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("abs(z)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("cos(y)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("cos(z)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("pi()", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("abs(x)", inspector, ExprType.STRING);
|
||||
assertOutputType("abs(y)", inspector, ExprType.LONG);
|
||||
assertOutputType("abs(z)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("cos(y)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("cos(z)", inspector, ExprType.DOUBLE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBivariateMathFunctions()
|
||||
{
|
||||
assertOutputType("div(y,y_)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("div(y,z_)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("div(z,z_)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("div(y,y_)", inspector, ExprType.LONG);
|
||||
assertOutputType("div(y,z_)", inspector, ExprType.LONG);
|
||||
assertOutputType("div(z,z_)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("max(y,y_)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("max(y,z_)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("max(z,z_)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("max(y,y_)", inspector, ExprType.LONG);
|
||||
assertOutputType("max(y,z_)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("max(z,z_)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("hypot(y,y_)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("hypot(y,z_)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("hypot(z,z_)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("hypot(y,y_)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("hypot(y,z_)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("hypot(z,z_)", inspector, ExprType.DOUBLE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConditionalFunctions()
|
||||
{
|
||||
assertOutputType("if(y, 'foo', 'bar')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("if(y,2,3)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("if(y,2,3.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("if(y, 'foo', 'bar')", inspector, ExprType.STRING);
|
||||
assertOutputType("if(y,2,3)", inspector, ExprType.LONG);
|
||||
assertOutputType("if(y,2,3.0)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType(
|
||||
"case_simple(x,'baz','is baz','foo','is foo','is other')",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.STRING
|
||||
);
|
||||
assertOutputType(
|
||||
"case_simple(y,2,2,3,3,4)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.LONG
|
||||
);
|
||||
assertOutputType(
|
||||
"case_simple(z,2.0,2.0,3.0,3.0,4.0)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.DOUBLE
|
||||
);
|
||||
|
||||
assertOutputType(
|
||||
"case_simple(y,2,2,3,3.0,4)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.DOUBLE
|
||||
);
|
||||
assertOutputType(
|
||||
"case_simple(z,2.0,2.0,3.0,3.0,null)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.DOUBLE
|
||||
);
|
||||
|
||||
assertOutputType(
|
||||
"case_searched(x=='baz','is baz',x=='foo','is foo','is other')",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.STRING
|
||||
);
|
||||
assertOutputType(
|
||||
"case_searched(y==1,1,y==2,2,0)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.LONG
|
||||
);
|
||||
assertOutputType(
|
||||
"case_searched(z==1.0,1.0,z==2.0,2.0,0.0)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.DOUBLE
|
||||
);
|
||||
assertOutputType(
|
||||
"case_searched(y==1,1,y==2,2.0,0)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.DOUBLE
|
||||
);
|
||||
assertOutputType(
|
||||
"case_searched(z==1.0,1,z==2.0,2,null)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.LONG
|
||||
);
|
||||
assertOutputType(
|
||||
"case_searched(z==1.0,1.0,z==2.0,2.0,null)",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.DOUBLE
|
||||
);
|
||||
|
||||
assertOutputType("nvl(x, 'foo')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("nvl(y, 1)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("nvl(y, 1.1)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("nvl(z, 2.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("nvl(y, 2.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("isnull(x)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("isnull(y)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("isnull(z)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("notnull(x)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("notnull(y)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("notnull(z)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("nvl(x, 'foo')", inspector, ExprType.STRING);
|
||||
assertOutputType("nvl(y, 1)", inspector, ExprType.LONG);
|
||||
assertOutputType("nvl(y, 1.1)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("nvl(z, 2.0)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("nvl(y, 2.0)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("isnull(x)", inspector, ExprType.LONG);
|
||||
assertOutputType("isnull(y)", inspector, ExprType.LONG);
|
||||
assertOutputType("isnull(z)", inspector, ExprType.LONG);
|
||||
assertOutputType("notnull(x)", inspector, ExprType.LONG);
|
||||
assertOutputType("notnull(y)", inspector, ExprType.LONG);
|
||||
assertOutputType("notnull(z)", inspector, ExprType.LONG);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringFunctions()
|
||||
{
|
||||
assertOutputType("concat(x, 'foo')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("concat(y, 'foo')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("concat(z, 'foo')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("concat(x, 'foo')", inspector, ExprType.STRING);
|
||||
assertOutputType("concat(y, 'foo')", inspector, ExprType.STRING);
|
||||
assertOutputType("concat(z, 'foo')", inspector, ExprType.STRING);
|
||||
|
||||
assertOutputType("strlen(x)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("format('%s', x)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("format('%s', y)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("format('%s', z)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("strpos(x, x_)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("strpos(x, y)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("strpos(x, z)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("substring(x, 1, 2)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("left(x, 1)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("right(x, 1)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("replace(x, 'foo', '')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("lower(x)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("upper(x)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("reverse(x)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("repeat(x, 4)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("strlen(x)", inspector, ExprType.LONG);
|
||||
assertOutputType("format('%s', x)", inspector, ExprType.STRING);
|
||||
assertOutputType("format('%s', y)", inspector, ExprType.STRING);
|
||||
assertOutputType("format('%s', z)", inspector, ExprType.STRING);
|
||||
assertOutputType("strpos(x, x_)", inspector, ExprType.LONG);
|
||||
assertOutputType("strpos(x, y)", inspector, ExprType.LONG);
|
||||
assertOutputType("strpos(x, z)", inspector, ExprType.LONG);
|
||||
assertOutputType("substring(x, 1, 2)", inspector, ExprType.STRING);
|
||||
assertOutputType("left(x, 1)", inspector, ExprType.STRING);
|
||||
assertOutputType("right(x, 1)", inspector, ExprType.STRING);
|
||||
assertOutputType("replace(x, 'foo', '')", inspector, ExprType.STRING);
|
||||
assertOutputType("lower(x)", inspector, ExprType.STRING);
|
||||
assertOutputType("upper(x)", inspector, ExprType.STRING);
|
||||
assertOutputType("reverse(x)", inspector, ExprType.STRING);
|
||||
assertOutputType("repeat(x, 4)", inspector, ExprType.STRING);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArrayFunctions()
|
||||
{
|
||||
assertOutputType("array(1, 2, 3)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array(1, 2, 3.0)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array(1, 2, 3)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array(1, 2, 3.0)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
|
||||
assertOutputType("array_length(a)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_length(b)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_length(c)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_length(a)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_length(b)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_length(c)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("string_to_array(x, ',')", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("string_to_array(x, ',')", inspector, ExprType.STRING_ARRAY);
|
||||
|
||||
assertOutputType("array_to_string(a, ',')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("array_to_string(b, ',')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("array_to_string(c, ',')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("array_to_string(a, ',')", inspector, ExprType.STRING);
|
||||
assertOutputType("array_to_string(b, ',')", inspector, ExprType.STRING);
|
||||
assertOutputType("array_to_string(c, ',')", inspector, ExprType.STRING);
|
||||
|
||||
assertOutputType("array_offset(a, 1)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("array_offset(b, 1)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_offset(c, 1)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("array_offset(a, 1)", inspector, ExprType.STRING);
|
||||
assertOutputType("array_offset(b, 1)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_offset(c, 1)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("array_ordinal(a, 1)", inputTypes, ExprType.STRING);
|
||||
assertOutputType("array_ordinal(b, 1)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_ordinal(c, 1)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("array_ordinal(a, 1)", inspector, ExprType.STRING);
|
||||
assertOutputType("array_ordinal(b, 1)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_ordinal(c, 1)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("array_offset_of(a, 'a')", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_offset_of(b, 1)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_offset_of(c, 1.0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_offset_of(a, 'a')", inspector, ExprType.LONG);
|
||||
assertOutputType("array_offset_of(b, 1)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_offset_of(c, 1.0)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("array_ordinal_of(a, 'a')", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_ordinal_of(b, 1)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_ordinal_of(c, 1.0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_ordinal_of(a, 'a')", inspector, ExprType.LONG);
|
||||
assertOutputType("array_ordinal_of(b, 1)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_ordinal_of(c, 1.0)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("array_append(x, x_)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_append(a, x_)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_append(y, y_)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_append(b, y_)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_append(z, z_)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_append(c, z_)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_append(x, x_)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_append(a, x_)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_append(y, y_)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_append(b, y_)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_append(z, z_)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_append(c, z_)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
|
||||
assertOutputType("array_concat(x, a)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_concat(a, a)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_concat(y, b)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_concat(b, b)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_concat(z, c)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_concat(c, c)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_concat(x, a)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_concat(a, a)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_concat(y, b)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_concat(b, b)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_concat(z, c)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_concat(c, c)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
|
||||
assertOutputType("array_contains(a, 'a')", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_contains(b, 1)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_contains(c, 2.0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_contains(a, 'a')", inspector, ExprType.LONG);
|
||||
assertOutputType("array_contains(b, 1)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_contains(c, 2.0)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("array_overlap(a, a)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_overlap(b, b)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_overlap(c, c)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("array_overlap(a, a)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_overlap(b, b)", inspector, ExprType.LONG);
|
||||
assertOutputType("array_overlap(c, c)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("array_slice(a, 1, 2)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_slice(b, 1, 2)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_slice(c, 1, 2)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_slice(a, 1, 2)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_slice(b, 1, 2)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_slice(c, 1, 2)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
|
||||
assertOutputType("array_prepend(x, a)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_prepend(x, x_)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_prepend(y, b)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_prepend(y, y_)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_prepend(z, c)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_prepend(z, z_)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_prepend(x, a)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_prepend(x, x_)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("array_prepend(y, b)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_prepend(y, y_)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("array_prepend(z, c)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("array_prepend(z, z_)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReduceFunctions()
|
||||
{
|
||||
assertOutputType("greatest('B', x, 'A')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("greatest(y, 0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("greatest(34.0, z, 5.0, 767.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("greatest('B', x, 'A')", inspector, ExprType.STRING);
|
||||
assertOutputType("greatest(y, 0)", inspector, ExprType.LONG);
|
||||
assertOutputType("greatest(34.0, z, 5.0, 767.0)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("least('B', x, 'A')", inputTypes, ExprType.STRING);
|
||||
assertOutputType("least(y, 0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("least(34.0, z, 5.0, 767.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("least('B', x, 'A')", inspector, ExprType.STRING);
|
||||
assertOutputType("least(y, 0)", inspector, ExprType.LONG);
|
||||
assertOutputType("least(34.0, z, 5.0, 767.0)", inspector, ExprType.DOUBLE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApplyFunctions()
|
||||
{
|
||||
assertOutputType("map((x) -> concat(x, 'foo'), x)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, y)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, z)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("map((x) -> concat(x, 'foo'), a)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, b)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, c)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("map((x) -> concat(x, 'foo'), x)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, y)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, z)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("map((x) -> concat(x, 'foo'), a)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, b)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("map((x) -> x + x, c)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType(
|
||||
"cartesian_map((x, y) -> concat(x, y), ['foo', 'bar', 'baz', 'foobar'], ['bar', 'baz'])",
|
||||
inputTypes,
|
||||
inspector,
|
||||
ExprType.STRING_ARRAY
|
||||
);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, 0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, y)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, 1.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, z)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, 0)", inspector, ExprType.LONG);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, y)", inspector, ExprType.LONG);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, 1.0)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("fold((x, acc) -> x + acc, y, z)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 0)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, y)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 1.0)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, z)", inputTypes, ExprType.DOUBLE);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 0)", inspector, ExprType.LONG);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, y)", inspector, ExprType.LONG);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 1.0)", inspector, ExprType.DOUBLE);
|
||||
assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, z)", inspector, ExprType.DOUBLE);
|
||||
|
||||
assertOutputType("filter((x) -> x == 'foo', a)", inputTypes, ExprType.STRING_ARRAY);
|
||||
assertOutputType("filter((x) -> x > 1, b)", inputTypes, ExprType.LONG_ARRAY);
|
||||
assertOutputType("filter((x) -> x > 1, c)", inputTypes, ExprType.DOUBLE_ARRAY);
|
||||
assertOutputType("filter((x) -> x == 'foo', a)", inspector, ExprType.STRING_ARRAY);
|
||||
assertOutputType("filter((x) -> x > 1, b)", inspector, ExprType.LONG_ARRAY);
|
||||
assertOutputType("filter((x) -> x > 1, c)", inspector, ExprType.DOUBLE_ARRAY);
|
||||
|
||||
assertOutputType("any((x) -> x == 'foo', a)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("any((x) -> x > 1, b)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("any((x) -> x > 1.2, c)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("any((x) -> x == 'foo', a)", inspector, ExprType.LONG);
|
||||
assertOutputType("any((x) -> x > 1, b)", inspector, ExprType.LONG);
|
||||
assertOutputType("any((x) -> x > 1.2, c)", inspector, ExprType.LONG);
|
||||
|
||||
assertOutputType("all((x) -> x == 'foo', a)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("all((x) -> x > 1, b)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("all((x) -> x > 1.2, c)", inputTypes, ExprType.LONG);
|
||||
assertOutputType("all((x) -> x == 'foo', a)", inspector, ExprType.LONG);
|
||||
assertOutputType("all((x) -> x > 1, b)", inspector, ExprType.LONG);
|
||||
assertOutputType("all((x) -> x > 1.2, c)", inspector, ExprType.LONG);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testEvalAutoConversion()
|
||||
{
|
||||
final ExprEval<?> nullStringEval = ExprEval.of(null);
|
||||
final ExprEval<?> stringEval = ExprEval.of("wat");
|
||||
final ExprEval<?> longEval = ExprEval.of(1L);
|
||||
final ExprEval<?> doubleEval = ExprEval.of(1.0);
|
||||
final ExprEval<?> arrayEval = ExprEval.ofLongArray(new Long[]{1L, 2L, 3L});
|
||||
|
||||
// only long stays long
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.autoDetect(longEval, longEval));
|
||||
// only string stays string
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.autoDetect(nullStringEval, nullStringEval));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.autoDetect(stringEval, stringEval));
|
||||
// if only 1 argument is a string, preserve the other type
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.autoDetect(nullStringEval, longEval));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.autoDetect(longEval, nullStringEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, nullStringEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(nullStringEval, doubleEval));
|
||||
// for operators, doubles is the catch all
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(longEval, doubleEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, longEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, doubleEval));
|
||||
// ... even when non-null strings are used with non-double types
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(longEval, stringEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, stringEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(stringEval, doubleEval));
|
||||
// arrays are not a good idea to use with this method
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(arrayEval, nullStringEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(arrayEval, doubleEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(arrayEval, longEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(nullStringEval, arrayEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, arrayEval));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(longEval, arrayEval));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOperatorAutoConversion()
|
||||
{
|
||||
// nulls output nulls
|
||||
Assert.assertNull(ExprTypeConversion.operator(ExprType.LONG, null));
|
||||
Assert.assertNull(ExprTypeConversion.operator(null, ExprType.LONG));
|
||||
Assert.assertNull(ExprTypeConversion.operator(ExprType.DOUBLE, null));
|
||||
Assert.assertNull(ExprTypeConversion.operator(null, ExprType.DOUBLE));
|
||||
Assert.assertNull(ExprTypeConversion.operator(ExprType.STRING, null));
|
||||
Assert.assertNull(ExprTypeConversion.operator(null, ExprType.STRING));
|
||||
// nulls output other
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.operator(ExprType.LONG, null));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.operator(null, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.DOUBLE, null));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(null, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.operator(ExprType.STRING, null));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.operator(null, ExprType.STRING));
|
||||
// only long stays long
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.operator(ExprType.LONG, ExprType.LONG));
|
||||
// only string stays string
|
||||
|
@ -426,47 +462,47 @@ public class OutputTypeTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testFunctionAutoConversion()
|
||||
{
|
||||
// nulls output nulls
|
||||
Assert.assertNull(ExprTypeConversion.doubleMathFunction(ExprType.LONG, null));
|
||||
Assert.assertNull(ExprTypeConversion.doubleMathFunction(null, ExprType.LONG));
|
||||
Assert.assertNull(ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE, null));
|
||||
Assert.assertNull(ExprTypeConversion.doubleMathFunction(null, ExprType.DOUBLE));
|
||||
Assert.assertNull(ExprTypeConversion.doubleMathFunction(ExprType.STRING, null));
|
||||
Assert.assertNull(ExprTypeConversion.doubleMathFunction(null, ExprType.STRING));
|
||||
// nulls output other
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.function(ExprType.LONG, null));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.function(null, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.DOUBLE, null));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(null, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, null));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(null, ExprType.STRING));
|
||||
// only long stays long
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.doubleMathFunction(ExprType.LONG, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.function(ExprType.LONG, ExprType.LONG));
|
||||
// any double makes all doubles
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.doubleMathFunction(ExprType.LONG, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.LONG, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.DOUBLE, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.DOUBLE, ExprType.DOUBLE));
|
||||
// any string makes become string
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.doubleMathFunction(ExprType.LONG, ExprType.STRING));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.doubleMathFunction(ExprType.STRING, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE, ExprType.STRING));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.doubleMathFunction(ExprType.STRING, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.doubleMathFunction(ExprType.STRING, ExprType.STRING));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.LONG, ExprType.STRING));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.DOUBLE, ExprType.STRING));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, ExprType.STRING));
|
||||
// unless it is an array, and those have to be the same
|
||||
Assert.assertEquals(ExprType.LONG_ARRAY, ExprTypeConversion.doubleMathFunction(ExprType.LONG_ARRAY, ExprType.LONG_ARRAY));
|
||||
Assert.assertEquals(ExprType.LONG_ARRAY, ExprTypeConversion.function(ExprType.LONG_ARRAY, ExprType.LONG_ARRAY));
|
||||
Assert.assertEquals(
|
||||
ExprType.DOUBLE_ARRAY,
|
||||
ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE_ARRAY, ExprType.DOUBLE_ARRAY)
|
||||
ExprTypeConversion.function(ExprType.DOUBLE_ARRAY, ExprType.DOUBLE_ARRAY)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ExprType.STRING_ARRAY,
|
||||
ExprTypeConversion.doubleMathFunction(ExprType.STRING_ARRAY, ExprType.STRING_ARRAY)
|
||||
ExprTypeConversion.function(ExprType.STRING_ARRAY, ExprType.STRING_ARRAY)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntegerFunctionAutoConversion()
|
||||
{
|
||||
// nulls output nulls
|
||||
Assert.assertNull(ExprTypeConversion.integerMathFunction(ExprType.LONG, null));
|
||||
Assert.assertNull(ExprTypeConversion.integerMathFunction(null, ExprType.LONG));
|
||||
Assert.assertNull(ExprTypeConversion.integerMathFunction(ExprType.DOUBLE, null));
|
||||
Assert.assertNull(ExprTypeConversion.integerMathFunction(null, ExprType.DOUBLE));
|
||||
Assert.assertNull(ExprTypeConversion.integerMathFunction(ExprType.STRING, null));
|
||||
Assert.assertNull(ExprTypeConversion.integerMathFunction(null, ExprType.STRING));
|
||||
// nulls output other
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.LONG, null));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(null, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.DOUBLE, null));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(null, ExprType.DOUBLE));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.STRING, null));
|
||||
Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(null, ExprType.STRING));
|
||||
// all numbers are longs
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.LONG, ExprType.LONG));
|
||||
Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.LONG, ExprType.DOUBLE));
|
||||
|
@ -494,30 +530,30 @@ public class OutputTypeTest extends InitializedNullHandlingTest
|
|||
public void testAutoConversionArrayMismatchArrays()
|
||||
{
|
||||
expectedException.expect(IAE.class);
|
||||
ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE_ARRAY, ExprType.LONG_ARRAY);
|
||||
ExprTypeConversion.function(ExprType.DOUBLE_ARRAY, ExprType.LONG_ARRAY);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoConversionArrayMismatchArrayScalar()
|
||||
{
|
||||
expectedException.expect(IAE.class);
|
||||
ExprTypeConversion.doubleMathFunction(ExprType.DOUBLE_ARRAY, ExprType.LONG);
|
||||
ExprTypeConversion.function(ExprType.DOUBLE_ARRAY, ExprType.LONG);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoConversionArrayMismatchScalarArray()
|
||||
{
|
||||
expectedException.expect(IAE.class);
|
||||
ExprTypeConversion.doubleMathFunction(ExprType.STRING, ExprType.LONG_ARRAY);
|
||||
ExprTypeConversion.function(ExprType.DOUBLE, ExprType.LONG_ARRAY);
|
||||
}
|
||||
|
||||
private void assertOutputType(String expression, Expr.InputBindingTypes inputTypes, ExprType outputType)
|
||||
private void assertOutputType(String expression, Expr.InputBindingInspector inspector, ExprType outputType)
|
||||
{
|
||||
final Expr expr = Parser.parse(expression, ExprMacroTable.nil(), false);
|
||||
Assert.assertEquals(outputType, expr.getOutputType(inputTypes));
|
||||
Assert.assertEquals(outputType, expr.getOutputType(inspector));
|
||||
}
|
||||
|
||||
Expr.InputBindingTypes inputTypesFromMap(Map<String, ExprType> types)
|
||||
Expr.InputBindingInspector inspectorFromMap(Map<String, ExprType> types)
|
||||
{
|
||||
return types::get;
|
||||
}
|
||||
|
|
|
@ -72,8 +72,9 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testBinaryOperators()
|
||||
{
|
||||
final String[] columns = new String[]{"d1", "d2", "l1", "l2", "1", "1.0"};
|
||||
final String[][] templateInputs = makeTemplateArgs(columns, columns);
|
||||
final String[] columns = new String[]{"d1", "d2", "l1", "l2", "1", "1.0", "nonexistent"};
|
||||
final String[] columns2 = new String[]{"d1", "d2", "l1", "l2", "1", "1.0"};
|
||||
final String[][] templateInputs = makeTemplateArgs(columns, columns2);
|
||||
final String[] templates =
|
||||
Arrays.stream(templateInputs)
|
||||
.map(i -> StringUtils.format("%s %s %s", i[0], "%s", i[1]))
|
||||
|
@ -86,9 +87,9 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testBinaryOperatorTrees()
|
||||
{
|
||||
final String[] columns = new String[]{"d1", "l1", "1", "1.0"};
|
||||
final String[] columns = new String[]{"d1", "l1", "1", "1.0", "nonexistent"};
|
||||
final String[] columns2 = new String[]{"d2", "l2", "2", "2.0"};
|
||||
final String[][] templateInputs = makeTemplateArgs(columns, columns2, columns);
|
||||
final String[][] templateInputs = makeTemplateArgs(columns, columns2, columns2);
|
||||
final String[] templates =
|
||||
Arrays.stream(templateInputs)
|
||||
.map(i -> StringUtils.format("(%s %s %s) %s %s", i[0], "%s", i[1], "%s", i[2]))
|
||||
|
@ -157,7 +158,14 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
"scalb",
|
||||
"pow"
|
||||
};
|
||||
final String[] templates = new String[]{"%s(d1, d2)", "%s(d1, l1)", "%s(l1, d1)", "%s(l1, l2)"};
|
||||
final String[] templates = new String[]{
|
||||
"%s(d1, d2)",
|
||||
"%s(d1, l1)",
|
||||
"%s(l1, d1)",
|
||||
"%s(l1, l2)",
|
||||
"%s(nonexistent, l1)",
|
||||
"%s(nonexistent, d1)"
|
||||
};
|
||||
testFunctions(types, templates, functions);
|
||||
}
|
||||
|
||||
|
@ -218,7 +226,7 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
for (int i = 0; i < VECTOR_SIZE; i++) {
|
||||
ExprEval<?> eval = parsed.eval(bindings.lhs[i]);
|
||||
if (!eval.isNumericNull()) {
|
||||
Assert.assertEquals(outputType, eval.type());
|
||||
Assert.assertEquals(eval.type(), outputType);
|
||||
}
|
||||
Assert.assertEquals(
|
||||
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
|
||||
|
@ -455,7 +463,7 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
@Override
|
||||
public <T> T[] getObjectVector(String name)
|
||||
{
|
||||
return (T[]) objects.get(name);
|
||||
return (T[]) objects.getOrDefault(name, new Object[getCurrentVectorSize()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -467,20 +475,22 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
@Override
|
||||
public long[] getLongVector(String name)
|
||||
{
|
||||
return longs.get(name);
|
||||
return longs.getOrDefault(name, new long[getCurrentVectorSize()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double[] getDoubleVector(String name)
|
||||
{
|
||||
return doubles.get(name);
|
||||
return doubles.getOrDefault(name, new double[getCurrentVectorSize()]);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector(String name)
|
||||
{
|
||||
return nulls.get(name);
|
||||
final boolean[] defaultVector = new boolean[getCurrentVectorSize()];
|
||||
Arrays.fill(defaultVector, NullHandling.sqlCompatible());
|
||||
return nulls.getOrDefault(name, defaultVector);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -127,7 +127,7 @@ public class BloomFilterExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -72,7 +72,7 @@ class ContainsExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -121,7 +121,7 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -97,7 +97,7 @@ public class IPv4AddressParseExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -96,7 +96,7 @@ public class IPv4AddressStringifyExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
|
|
@ -94,7 +94,7 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -98,7 +98,7 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
|
|
@ -101,7 +101,7 @@ public class RegexpExtractExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
|
|
@ -93,7 +93,7 @@ public class RegexpLikeExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -97,7 +97,7 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -165,7 +165,7 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -166,7 +166,7 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
switch (unit) {
|
||||
case CENTURY:
|
||||
|
|
|
@ -118,24 +118,24 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(InputBindingTypes inputTypes)
|
||||
public boolean canVectorize(InputBindingInspector inspector)
|
||||
{
|
||||
return args.get(0).canVectorize(inputTypes);
|
||||
return args.get(0).canVectorize(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
|
||||
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingInspector inspector)
|
||||
{
|
||||
ExprVectorProcessor<?> processor;
|
||||
processor = new LongOutLongInFunctionVectorProcessor(
|
||||
CastToTypeVectorProcessor.castToType(args.get(0).buildVectorized(inputTypes), ExprType.LONG),
|
||||
inputTypes.getMaxVectorSize()
|
||||
CastToTypeVectorProcessor.cast(args.get(0).buildVectorized(inspector), ExprType.LONG),
|
||||
inspector.getMaxVectorSize()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -195,7 +195,7 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -101,7 +101,7 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
|
|
@ -104,7 +104,7 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -106,7 +106,7 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
@ -139,7 +139,7 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.LONG;
|
||||
}
|
||||
|
|
|
@ -172,7 +172,7 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
@ -300,7 +300,7 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExprType getOutputType(InputBindingTypes inputTypes)
|
||||
public ExprType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return ExprType.STRING;
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.druid.segment.column.ColumnCapabilities;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public interface ColumnInspector extends Expr.InputBindingTypes
|
||||
public interface ColumnInspector extends Expr.InputBindingInspector
|
||||
{
|
||||
/**
|
||||
* Returns capabilities of a particular column.
|
||||
|
|
|
@ -37,7 +37,7 @@ public class NilVectorSelector
|
|||
|
||||
static {
|
||||
for (int i = 0; i < DEFAULT_NULLS_VECTOR.length; i++) {
|
||||
DEFAULT_NULLS_VECTOR[i] = true;
|
||||
DEFAULT_NULLS_VECTOR[i] = NullHandling.sqlCompatible();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -53,9 +53,13 @@ public class ExpressionPlan
|
|||
*/
|
||||
NEEDS_APPLIED,
|
||||
/**
|
||||
* expression has inputs whose type was unresolveable, or was incomplete, such as unknown multi-valuedness
|
||||
* expression has inputs whose type was unresolveable
|
||||
*/
|
||||
UNKNOWN_INPUTS,
|
||||
/**
|
||||
* expression has inputs whose type was incomplete, such as unknown multi-valuedness
|
||||
*/
|
||||
INCOMPLETE_INPUTS,
|
||||
/**
|
||||
* expression explicitly using multi-valued inputs as array inputs
|
||||
*/
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment.virtual;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprType;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
|
@ -56,6 +57,7 @@ public class ExpressionPlanner
|
|||
Parser.validateExpr(expression, analysis);
|
||||
|
||||
EnumSet<ExpressionPlan.Trait> traits = EnumSet.noneOf(ExpressionPlan.Trait.class);
|
||||
Set<String> noCapabilities = new HashSet<>();
|
||||
Set<String> maybeMultiValued = new HashSet<>();
|
||||
List<String> needsApplied = ImmutableList.of();
|
||||
ValueType singleInputType = null;
|
||||
|
@ -120,7 +122,7 @@ public class ExpressionPlanner
|
|||
maybeMultiValued.add(column);
|
||||
}
|
||||
} else {
|
||||
maybeMultiValued.add(column);
|
||||
noCapabilities.add(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -135,18 +137,29 @@ public class ExpressionPlanner
|
|||
traits.add(ExpressionPlan.Trait.NON_SCALAR_INPUTS);
|
||||
}
|
||||
|
||||
if (!maybeMultiValued.isEmpty()) {
|
||||
if (!noCapabilities.isEmpty()) {
|
||||
traits.add(ExpressionPlan.Trait.UNKNOWN_INPUTS);
|
||||
}
|
||||
|
||||
if (!maybeMultiValued.isEmpty()) {
|
||||
traits.add(ExpressionPlan.Trait.INCOMPLETE_INPUTS);
|
||||
}
|
||||
|
||||
// if expression needs transformed, lets do it
|
||||
if (!needsApplied.isEmpty()) {
|
||||
traits.add(ExpressionPlan.Trait.NEEDS_APPLIED);
|
||||
}
|
||||
}
|
||||
|
||||
// only set output type
|
||||
if (ExpressionPlan.none(traits, ExpressionPlan.Trait.UNKNOWN_INPUTS, ExpressionPlan.Trait.NEEDS_APPLIED)) {
|
||||
// only set output type if we are pretty confident about input types
|
||||
final boolean shouldComputeOutput = ExpressionPlan.none(
|
||||
traits,
|
||||
ExpressionPlan.Trait.UNKNOWN_INPUTS,
|
||||
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
|
||||
ExpressionPlan.Trait.NEEDS_APPLIED
|
||||
);
|
||||
|
||||
if (shouldComputeOutput) {
|
||||
outputType = expression.getOutputType(inspector);
|
||||
}
|
||||
|
||||
|
@ -163,16 +176,22 @@ public class ExpressionPlanner
|
|||
traits.add(ExpressionPlan.Trait.NON_SCALAR_OUTPUT);
|
||||
}
|
||||
|
||||
// vectorized expressions do not currently support unknown inputs, multi-valued inputs or outputs, implicit mapping
|
||||
// vectorized expressions do not support incomplete, multi-valued inputs or outputs, or implicit mapping
|
||||
// they also do support unknown inputs, but they also do not currently have to deal with them, as missing
|
||||
// capabilites is indicative of a non-existent column instead of an unknown schema. If this ever changes,
|
||||
// this check should also change
|
||||
boolean supportsVector = ExpressionPlan.none(
|
||||
traits,
|
||||
ExpressionPlan.Trait.UNKNOWN_INPUTS,
|
||||
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
|
||||
ExpressionPlan.Trait.NEEDS_APPLIED,
|
||||
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
|
||||
ExpressionPlan.Trait.NON_SCALAR_OUTPUT
|
||||
);
|
||||
|
||||
if (supportsVector && expression.canVectorize(inspector)) {
|
||||
// make sure to compute the output type for a vector expression though, because we might have skipped it earlier
|
||||
// due to unknown inputs, but that's ok here since it just means it doesnt exist
|
||||
outputType = expression.getOutputType(inspector);
|
||||
traits.add(ExpressionPlan.Trait.VECTORIZABLE);
|
||||
}
|
||||
return new ExpressionPlan(
|
||||
|
@ -181,7 +200,7 @@ public class ExpressionPlanner
|
|||
traits,
|
||||
outputType,
|
||||
singleInputType,
|
||||
maybeMultiValued,
|
||||
Sets.union(noCapabilities, maybeMultiValued),
|
||||
needsApplied
|
||||
);
|
||||
}
|
||||
|
|
|
@ -165,7 +165,7 @@ public class ExpressionSelectors
|
|||
|
||||
// if any unknown column input types, fall back to an expression selector that examines input bindings on a
|
||||
// per row basis
|
||||
if (plan.is(ExpressionPlan.Trait.UNKNOWN_INPUTS)) {
|
||||
if (plan.any(ExpressionPlan.Trait.UNKNOWN_INPUTS, ExpressionPlan.Trait.INCOMPLETE_INPUTS)) {
|
||||
return new RowBasedExpressionColumnValueSelector(plan, bindings);
|
||||
}
|
||||
|
||||
|
@ -207,7 +207,8 @@ public class ExpressionSelectors
|
|||
if (plan.any(
|
||||
ExpressionPlan.Trait.NON_SCALAR_OUTPUT,
|
||||
ExpressionPlan.Trait.NEEDS_APPLIED,
|
||||
ExpressionPlan.Trait.UNKNOWN_INPUTS
|
||||
ExpressionPlan.Trait.UNKNOWN_INPUTS,
|
||||
ExpressionPlan.Trait.INCOMPLETE_INPUTS
|
||||
)) {
|
||||
return ExpressionMultiValueDimensionSelector.fromValueSelector(baseSelector, extractionFn);
|
||||
} else {
|
||||
|
|
|
@ -65,16 +65,19 @@ public class ExpressionVectorSelectorsTest
|
|||
{
|
||||
private static List<String> EXPRESSIONS = ImmutableList.of(
|
||||
"long1 * long2",
|
||||
"long1 * nonexistent",
|
||||
"double1 * double3",
|
||||
"float1 + float3",
|
||||
"(long1 - long4) / double3",
|
||||
"long5 * float3 * long1 * long4 * double1",
|
||||
"long5 * double3 * long1 * long4 * double1",
|
||||
"max(double3, double5)",
|
||||
"max(nonexistent, double5)",
|
||||
"min(double4, double1)",
|
||||
"cos(float3)",
|
||||
"sin(long4)",
|
||||
"parse_long(string1)",
|
||||
"parse_long(nonexistent)",
|
||||
"parse_long(string1) * double3",
|
||||
"parse_long(string5) * parse_long(string1)",
|
||||
"parse_long(string5) * parse_long(string1) * double3",
|
||||
|
|
|
@ -216,7 +216,7 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
|
||||
CURRENT_ROW.set(ROW1);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(4.0d, selector.getObject());
|
||||
Assert.assertEquals(4L, selector.getObject());
|
||||
} else {
|
||||
// y is null for row1
|
||||
Assert.assertEquals(null, selector.getObject());
|
||||
|
@ -481,7 +481,7 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(false, nullMatcher.matches());
|
||||
Assert.assertEquals(false, fiveMatcher.matches());
|
||||
Assert.assertEquals(true, nonNullMatcher.matches());
|
||||
Assert.assertEquals("4.0", selector.lookupName(selector.getRow().get(0)));
|
||||
Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0)));
|
||||
} else {
|
||||
// y is null in row1
|
||||
Assert.assertEquals(true, nullMatcher.matches());
|
||||
|
@ -605,7 +605,7 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(false, nullMatcher.matches());
|
||||
Assert.assertEquals(false, fiveMatcher.matches());
|
||||
Assert.assertEquals(true, nonNullMatcher.matches());
|
||||
Assert.assertEquals("4.0", selector.lookupName(selector.getRow().get(0)));
|
||||
Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0)));
|
||||
} else {
|
||||
// y is null in row1
|
||||
Assert.assertEquals(true, nullMatcher.matches());
|
||||
|
|
|
@ -39,7 +39,7 @@ class ReductionOperatorConversionHelper
|
|||
* https://dev.mysql.com/doc/refman/8.0/en/comparison-operators.html#function_least
|
||||
*
|
||||
* @see org.apache.druid.math.expr.Function.ReduceFunction#apply
|
||||
* @see ExprTypeConversion#doubleMathFunction
|
||||
* @see ExprTypeConversion#function
|
||||
*/
|
||||
static final SqlReturnTypeInference TYPE_INFERENCE =
|
||||
opBinding -> {
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.calcite.schema.SchemaPlus;
|
|||
import org.apache.calcite.sql.parser.SqlParseException;
|
||||
import org.apache.calcite.tools.RelConversionException;
|
||||
import org.apache.calcite.tools.ValidationException;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
|
@ -71,18 +70,7 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
|
|||
{
|
||||
private static final Logger log = new Logger(SqlVectorizedExpressionSanityTest.class);
|
||||
|
||||
// cannot vectorize grouping on numeric expressions in group by v2 in sql compatible null handling mode
|
||||
private static final List<String> QUERIES = NullHandling.sqlCompatible() ? ImmutableList.of(
|
||||
"SELECT SUM(long1 * long2) FROM foo",
|
||||
"SELECT SUM((long1 * long2) / double1) FROM foo",
|
||||
"SELECT SUM(float3 + ((long1 * long4)/double1)) FROM foo",
|
||||
"SELECT SUM(long5 - (float3 + ((long1 * long4)/double1))) FROM foo",
|
||||
"SELECT cos(double2) FROM foo",
|
||||
"SELECT SUM(-long4) FROM foo",
|
||||
"SELECT SUM(PARSE_LONG(string1)) FROM foo",
|
||||
"SELECT SUM(PARSE_LONG(string3)) FROM foo",
|
||||
"SELECT string2, SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 2"
|
||||
) : ImmutableList.of(
|
||||
private static final List<String> QUERIES = ImmutableList.of(
|
||||
"SELECT SUM(long1 * long2) FROM foo",
|
||||
"SELECT SUM((long1 * long2) / double1) FROM foo",
|
||||
"SELECT SUM(float3 + ((long1 * long4)/double1)) FROM foo",
|
||||
|
|
Loading…
Reference in New Issue