various fixes and improvements to vectorization fallback (#17098) (#17142)

changes:
* add `ApplyFunction` support to vectorization fallback, allowing many of the remaining expressions to be vectorized
* add `CastToObjectVectorProcessor` so that vector engine can correctly cast any type
* add support for array and complex vector constants
* reduce number of cases which can block vectorization in expression planner to be unknown inputs (such as unknown multi-valuedness)
* fix array constructor expression, apply map expression to make actual evaluated type match the output type inference
* fix bug in array_contains where something like array_contains([null], 'hello') would return true if the array was a numeric array since the non-null string value would cast to a null numeric
* fix isNull/isNotNull to correctly handle any type of input argument
This commit is contained in:
Clint Wylie 2024-09-24 16:40:49 -07:00 committed by GitHub
parent 0ae9988796
commit cf00b4cd24
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
37 changed files with 493 additions and 394 deletions

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet;
import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
@ -135,10 +136,28 @@ public interface ApplyFunction extends NamedFunction
{
final int length = bindings.getLength();
Object[] out = new Object[length];
final boolean computeArrayType = arrayType == null;
ExpressionType arrayElementType = arrayType != null
? (ExpressionType) arrayType.getElementType()
: null;
final ExprEval<?>[] outEval = computeArrayType ? new ExprEval[length] : null;
for (int i = 0; i < length; i++) {
ExprEval evaluated = expr.eval(bindings.withIndex(i));
arrayType = Function.ArrayConstructorFunction.setArrayOutput(arrayType, out, i, evaluated);
final ExprEval<?> eval = expr.eval(bindings.withIndex(i));
if (computeArrayType && outEval[i].value() != null) {
arrayElementType = ExpressionTypeConversion.leastRestrictiveType(arrayElementType, eval.type());
outEval[i] = eval;
} else {
out[i] = eval.castTo(arrayElementType).value();
}
}
if (arrayElementType == null) {
arrayElementType = NullHandling.sqlCompatible() ? ExpressionType.LONG : ExpressionType.STRING;
}
if (computeArrayType) {
arrayType = ExpressionTypeFactory.getInstance().ofArray(arrayElementType);
for (int i = 0; i < length; i++) {
out[i] = outEval[i].castTo(arrayElementType).value();
}
}
return ExprEval.ofArray(arrayType, out);
}
@ -237,7 +256,7 @@ public interface ApplyFunction extends NamedFunction
List<List<Object>> product = CartesianList.create(arrayInputs);
CartesianMapLambdaBinding lambdaBinding = new CartesianMapLambdaBinding(elementType, product, lambdaExpr, bindings);
ExpressionType lambdaType = lambdaExpr.getOutputType(lambdaBinding);
return applyMap(ExpressionType.asArrayType(lambdaType), lambdaExpr, lambdaBinding);
return applyMap(lambdaType == null ? null : ExpressionTypeFactory.getInstance().ofArray(lambdaType), lambdaExpr, lambdaBinding);
}
@Override

View File

@ -127,6 +127,11 @@ abstract class ConstantExpr<T> implements Expr
return new ExprEvalBasedConstantExpr<T>(realEval());
}
@Override
public <E> ExprVectorProcessor<E> asVectorProcessor(VectorInputBindingInspector inspector)
{
return VectorProcessors.constant(value, inspector.getMaxVectorSize(), outputType);
}
/**
* Constant expression based on a concreate ExprEval.
*
@ -415,7 +420,7 @@ class StringExpr extends ConstantExpr<String>
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return VectorProcessors.constant(value, inspector.getMaxVectorSize());
return VectorProcessors.constant(value, inspector.getMaxVectorSize(), ExpressionType.STRING);
}
@Override
@ -459,12 +464,6 @@ class ArrayExpr extends ConstantExpr<Object[]>
return ExprEval.ofArray(outputType, value);
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return false;
}
@Override
public String stringify()
{
@ -547,12 +546,6 @@ class ComplexExpr extends ConstantExpr<Object>
return ExprEval.ofComplex(outputType, value);
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return false;
}
@Override
public String stringify()
{

View File

@ -2026,7 +2026,8 @@ public interface Function extends NamedFunction
{
return CastToTypeVectorProcessor.cast(
args.get(0).asVectorProcessor(inspector),
ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString()))
ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())),
inspector.getMaxVectorSize()
);
}
}
@ -3357,19 +3358,24 @@ public interface Function extends NamedFunction
@Override
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
{
// this is copied from 'BaseMapFunction.applyMap', need to find a better way to consolidate, or construct arrays,
// or.. something...
final int length = args.size();
Object[] out = new Object[length];
ExpressionType arrayType = null;
ExpressionType arrayElementType = null;
final ExprEval[] outEval = new ExprEval[length];
for (int i = 0; i < length; i++) {
ExprEval<?> evaluated = args.get(i).eval(bindings);
arrayType = setArrayOutput(arrayType, out, i, evaluated);
outEval[i] = args.get(i).eval(bindings);
if (outEval[i].value() != null) {
arrayElementType = ExpressionTypeConversion.leastRestrictiveType(arrayElementType, outEval[i].type());
}
}
return ExprEval.ofArray(arrayType, out);
if (arrayElementType == null) {
arrayElementType = NullHandling.sqlCompatible() ? ExpressionType.LONG : ExpressionType.STRING;
}
for (int i = 0; i < length; i++) {
out[i] = outEval[i].castTo(arrayElementType).value();
}
return ExprEval.ofArray(ExpressionTypeFactory.getInstance().ofArray(arrayElementType), out);
}
@Override
@ -3394,28 +3400,6 @@ public interface Function extends NamedFunction
}
return type == null ? null : ExpressionTypeFactory.getInstance().ofArray(type);
}
/**
* Set an array element to the output array, checking for null if the array is numeric. If the type of the evaluated
* array element does not match the array element type, this method will attempt to call {@link ExprEval#castTo}
* to the array element type, else will set the element as is. If the type of the array is unknown, it will be
* detected and defined from the first element. Returns the type of the array, which will be identical to the input
* type, unless the input type was null.
*/
static ExpressionType setArrayOutput(@Nullable ExpressionType arrayType, Object[] out, int i, ExprEval evaluated)
{
if (arrayType == null) {
arrayType = ExpressionTypeFactory.getInstance().ofArray(evaluated.type());
}
if (arrayType.getElementType().isNumeric() && evaluated.isNumericNull()) {
out[i] = null;
} else if (!evaluated.asArrayType().equals(arrayType)) {
out[i] = evaluated.castTo((ExpressionType) arrayType.getElementType()).value();
} else {
out[i] = evaluated.value();
}
return arrayType;
}
}
class ArrayLengthFunction implements Function
@ -3954,6 +3938,9 @@ public interface Function extends NamedFunction
return ExprEval.ofLongBoolean(Arrays.asList(array1).containsAll(Arrays.asList(array2)));
} else {
final Object elem = rhsExpr.castTo((ExpressionType) array1Type.getElementType()).value();
if (elem == null && rhsExpr.value() != null) {
return ExprEval.ofLongBoolean(false);
}
return ExprEval.ofLongBoolean(Arrays.asList(array1).contains(elem));
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.math.expr;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.StringUtils;
@ -30,8 +29,6 @@ import org.apache.druid.segment.column.Types;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
@SuppressWarnings("unused")
final class FunctionalExpr
@ -39,127 +36,6 @@ final class FunctionalExpr
// phony class to enable maven to track the compilation of this class
}
@SuppressWarnings("ClassName")
class LambdaExpr implements Expr
{
private final ImmutableList<IdentifierExpr> args;
private final Expr expr;
LambdaExpr(List<IdentifierExpr> args, Expr expr)
{
this.args = ImmutableList.copyOf(args);
this.expr = expr;
}
@Override
public String toString()
{
return StringUtils.format("(%s -> %s)", args, expr);
}
int identifierCount()
{
return args.size();
}
@Nullable
public String getIdentifier()
{
Preconditions.checkState(args.size() < 2, "LambdaExpr has multiple arguments, use getIdentifiers");
if (args.size() == 1) {
return args.get(0).toString();
}
return null;
}
public List<String> getIdentifiers()
{
return args.stream().map(IdentifierExpr::toString).collect(Collectors.toList());
}
public List<String> stringifyIdentifiers()
{
return args.stream().map(IdentifierExpr::stringify).collect(Collectors.toList());
}
ImmutableList<IdentifierExpr> getIdentifierExprs()
{
return args;
}
public Expr getExpr()
{
return expr;
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return expr.canVectorize(inspector);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return expr.asVectorProcessor(inspector);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return expr.eval(bindings);
}
@Override
public String stringify()
{
return StringUtils.format("(%s) -> %s", ARG_JOINER.join(stringifyIdentifiers()), expr.stringify());
}
@Override
public Expr visit(Shuttle shuttle)
{
List<IdentifierExpr> newArgs =
args.stream().map(arg -> (IdentifierExpr) shuttle.visit(arg)).collect(Collectors.toList());
Expr newBody = expr.visit(shuttle);
return shuttle.visit(new LambdaExpr(newArgs, newBody));
}
@Override
public BindingAnalysis analyzeInputs()
{
final Set<String> lambdaArgs = args.stream().map(IdentifierExpr::toString).collect(Collectors.toSet());
BindingAnalysis bodyDetails = expr.analyzeInputs();
return bodyDetails.removeLambdaArguments(lambdaArgs);
}
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return expr.getOutputType(inspector);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
LambdaExpr that = (LambdaExpr) o;
return Objects.equals(args, that.args) &&
Objects.equals(expr, that.expr);
}
@Override
public int hashCode()
{
return Objects.hash(args, expr);
}
}
/**
* {@link Expr} node for a {@link Function} call. {@link FunctionExpr} has children {@link Expr} in the form of the
* list of arguments that are passed to the {@link Function} along with the {@link Expr.ObjectBinding} when it is
@ -350,15 +226,24 @@ class ApplyFunctionExpr implements Expr
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return function.canVectorize(inspector, lambdaExpr, argsExpr) &&
lambdaExpr.canVectorize(inspector) &&
argsExpr.stream().allMatch(expr -> expr.canVectorize(inspector));
return canVectorizeNative(inspector) || (getOutputType(inspector) != null && inspector.canVectorize(argsExpr));
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return function.asVectorProcessor(inspector, lambdaExpr, argsExpr);
if (canVectorizeNative(inspector)) {
return function.asVectorProcessor(inspector, lambdaExpr, argsExpr);
} else {
return FallbackVectorProcessor.create(function, lambdaExpr, argsExpr, inspector);
}
}
private boolean canVectorizeNative(InputBindingInspector inspector)
{
return function.canVectorize(inspector, lambdaExpr, argsExpr) &&
lambdaExpr.canVectorize(inspector) &&
argsExpr.stream().allMatch(expr -> expr.canVectorize(inspector));
}
@Override

View File

@ -0,0 +1,151 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.math.expr;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
public class LambdaExpr implements Expr
{
private final ImmutableList<IdentifierExpr> args;
private final Expr expr;
LambdaExpr(List<IdentifierExpr> args, Expr expr)
{
this.args = ImmutableList.copyOf(args);
this.expr = expr;
}
@Override
public String toString()
{
return StringUtils.format("(%s -> %s)", args, expr);
}
int identifierCount()
{
return args.size();
}
@Nullable
public String getIdentifier()
{
Preconditions.checkState(args.size() < 2, "LambdaExpr has multiple arguments, use getIdentifiers");
if (args.size() == 1) {
return args.get(0).toString();
}
return null;
}
public List<String> getIdentifiers()
{
return args.stream().map(IdentifierExpr::toString).collect(Collectors.toList());
}
public List<String> stringifyIdentifiers()
{
return args.stream().map(IdentifierExpr::stringify).collect(Collectors.toList());
}
ImmutableList<IdentifierExpr> getIdentifierExprs()
{
return args;
}
public Expr getExpr()
{
return expr;
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return expr.canVectorize(inspector);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return expr.asVectorProcessor(inspector);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return expr.eval(bindings);
}
@Override
public String stringify()
{
return StringUtils.format("(%s) -> %s", ARG_JOINER.join(stringifyIdentifiers()), expr.stringify());
}
@Override
public Expr visit(Shuttle shuttle)
{
List<IdentifierExpr> newArgs =
args.stream().map(arg -> (IdentifierExpr) shuttle.visit(arg)).collect(Collectors.toList());
Expr newBody = expr.visit(shuttle);
return shuttle.visit(new LambdaExpr(newArgs, newBody));
}
@Override
public BindingAnalysis analyzeInputs()
{
final Set<String> lambdaArgs = args.stream().map(IdentifierExpr::toString).collect(Collectors.toSet());
BindingAnalysis bodyDetails = expr.analyzeInputs();
return bodyDetails.removeLambdaArguments(lambdaArgs);
}
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return expr.getOutputType(inspector);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
LambdaExpr that = (LambdaExpr) o;
return Objects.equals(args, that.args) &&
Objects.equals(expr, that.expr);
}
@Override
public int hashCode()
{
return Objects.hash(args, expr);
}
}

View File

@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.math.expr.vector;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
public class CastToObjectVectorProcessor extends CastToTypeVectorProcessor<Object[]>
{
private final ExpressionType outputType;
private final ExpressionType delegateType;
private final Object[] output;
public CastToObjectVectorProcessor(
ExprVectorProcessor<?> delegate,
ExpressionType outputType,
int maxVectorSize
)
{
super(delegate);
this.delegateType = delegate.getOutputType();
this.outputType = outputType;
this.output = new Object[maxVectorSize];
}
@Override
public ExprEvalVector<Object[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<?> delegateOutput = delegate.evalVector(bindings);
final Object[] toCast = delegateOutput.getObjectVector();
for (int i = 0; i < bindings.getCurrentVectorSize(); i++) {
ExprEval<?> cast = ExprEval.ofType(delegateType, toCast[i]).castTo(outputType);
output[i] = cast.value();
}
return new ExprEvalObjectVector(output, outputType);
}
@Override
public ExpressionType getOutputType()
{
return outputType;
}
}

View File

@ -25,9 +25,12 @@ import org.apache.druid.math.expr.ExpressionType;
public final class CastToStringVectorProcessor extends CastToTypeVectorProcessor<Object[]>
{
public CastToStringVectorProcessor(ExprVectorProcessor<?> delegate)
private final Object[] output;
public CastToStringVectorProcessor(ExprVectorProcessor<?> delegate, int maxVectorSize)
{
super(delegate);
this.output = new Object[maxVectorSize];
}
@Override
@ -35,8 +38,7 @@ public final class CastToStringVectorProcessor extends CastToTypeVectorProcessor
{
ExprEvalVector<?> result = delegate.evalVector(bindings);
final Object[] objects = result.getObjectVector();
final Object[] output = new String[objects.length];
for (int i = 0; i < objects.length; i++) {
for (int i = 0; i < bindings.getCurrentVectorSize(); i++) {
output[i] = Evals.asString(objects[i]);
}
return new ExprEvalObjectVector(output, ExpressionType.STRING);

View File

@ -20,7 +20,6 @@
package org.apache.druid.math.expr.vector;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.Exprs;
public abstract class CastToTypeVectorProcessor<TOutput> implements ExprVectorProcessor<TOutput>
{
@ -31,24 +30,29 @@ public abstract class CastToTypeVectorProcessor<TOutput> implements ExprVectorPr
this.delegate = delegate;
}
public static <T> ExprVectorProcessor<T> cast(ExprVectorProcessor<?> delegate, ExpressionType type)
public static <T> ExprVectorProcessor<T> cast(
ExprVectorProcessor<?> castInput,
ExpressionType castToType,
int maxVectorSize
)
{
final ExprVectorProcessor<?> caster;
if (delegate.getOutputType().equals(type)) {
caster = delegate;
final ExpressionType inputType = castInput.getOutputType();
if (inputType.equals(castToType)) {
caster = castInput;
} else {
switch (type.getType()) {
switch (castToType.getType()) {
case STRING:
caster = new CastToStringVectorProcessor(delegate);
caster = new CastToStringVectorProcessor(castInput, maxVectorSize);
break;
case LONG:
caster = new CastToLongVectorProcessor(delegate);
caster = new CastToLongVectorProcessor(castInput);
break;
case DOUBLE:
caster = new CastToDoubleVectorProcessor(delegate);
caster = new CastToDoubleVectorProcessor(castInput);
break;
default:
throw Exprs.cannotVectorize();
caster = new CastToObjectVectorProcessor(castInput, castToType, maxVectorSize);
}
}
return (ExprVectorProcessor<T>) caster;

View File

@ -29,7 +29,7 @@ public abstract class DoubleOutDoubleInFunctionVectorValueProcessor
{
public DoubleOutDoubleInFunctionVectorValueProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE, maxVectorSize), maxVectorSize);
}
public abstract double apply(double input);

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutDoubleLongInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutDoublesInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutLongDoubleInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -29,7 +29,7 @@ public abstract class DoubleOutLongInFunctionVectorValueProcessor
{
public DoubleOutLongInFunctionVectorValueProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG, maxVectorSize), maxVectorSize);
}
public abstract double apply(long input);

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutLongsInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -72,7 +72,7 @@ public final class ExprEvalObjectVector extends ExprEvalVector<Object[]>
}
} else {
// ARRAY, COMPLEX
final ExprEval<?> valueEval = ExprEval.bestEffortOf(values[i]);
final ExprEval<?> valueEval = ExprEval.ofType(type, values[i]).castTo(ExpressionType.DOUBLE);
longs[i] = valueEval.asLong();
doubles[i] = valueEval.asDouble();
numericNulls[i] = valueEval.isNumericNull();

View File

@ -20,12 +20,14 @@
package org.apache.druid.math.expr.vector;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.ApplyFunction;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.Function;
import org.apache.druid.math.expr.LambdaExpr;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -73,6 +75,25 @@ public abstract class FallbackVectorProcessor<T> implements ExprVectorProcessor<
);
}
/**
* Create a processor for a non-vectorizable {@link ApplyFunction}.
*/
public static <T> FallbackVectorProcessor<T> create(
final ApplyFunction function,
final LambdaExpr lambdaExpr,
final List<Expr> args,
final Expr.VectorInputBindingInspector inspector
)
{
final List<Expr> adaptedArgs = makeAdaptedArgs(args, inspector);
return makeFallbackProcessor(
() -> function.apply(lambdaExpr, adaptedArgs, UnusedBinding.INSTANCE),
adaptedArgs,
function.getOutputType(inspector, lambdaExpr, args),
inspector
);
}
/**
* Create a processor for a non-vectorizable {@link ExprMacroTable.ExprMacro}.
*/

View File

@ -29,7 +29,7 @@ public abstract class LongOutDoubleInFunctionVectorValueProcessor
{
public LongOutDoubleInFunctionVectorValueProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE, maxVectorSize), maxVectorSize);
}
public abstract long apply(double input);

View File

@ -34,8 +34,8 @@ public abstract class LongOutDoubleLongInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class LongOutDoublesInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class LongOutLongDoubleInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -29,7 +29,7 @@ public abstract class LongOutLongInFunctionVectorValueProcessor
{
public LongOutLongInFunctionVectorValueProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG, maxVectorSize), maxVectorSize);
}
public abstract long apply(long input);

View File

@ -34,8 +34,8 @@ public abstract class LongOutLongsInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -33,7 +33,7 @@ public abstract class LongOutObjectInFunctionVectorProcessor
ExpressionType inputType
)
{
super(CastToTypeVectorProcessor.cast(processor, inputType), maxVectorSize, new long[maxVectorSize]);
super(CastToTypeVectorProcessor.cast(processor, inputType, maxVectorSize), maxVectorSize, new long[maxVectorSize]);
}
@Override

View File

@ -37,8 +37,8 @@ public abstract class LongOutObjectsInFunctionVectorProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, inputType),
CastToTypeVectorProcessor.cast(right, inputType),
CastToTypeVectorProcessor.cast(left, inputType, maxVectorSize),
CastToTypeVectorProcessor.cast(right, inputType, maxVectorSize),
new long[maxVectorSize]
);
this.outNulls = new boolean[maxVectorSize];

View File

@ -36,8 +36,8 @@ public abstract class ObjectOutObjectsInFunctionVectorProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, expressionType),
CastToTypeVectorProcessor.cast(right, expressionType),
CastToTypeVectorProcessor.cast(left, expressionType, maxVectorSize),
CastToTypeVectorProcessor.cast(right, expressionType, maxVectorSize),
new Object[maxVectorSize]
);
this.expressionType = expressionType;

View File

@ -21,6 +21,7 @@ package org.apache.druid.math.expr.vector;
import com.google.common.base.Preconditions;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
@ -81,15 +82,19 @@ public class VectorProcessors
}
/**
* Creates an {@link ExprVectorProcessor} that creates a {@link ExprEvalVector} for a constant {@link String} value.
* Creates an {@link ExprVectorProcessor} that creates a {@link ExprEvalVector} for a constant any non-numeric value.
* Numeric types should use {@link #constant(Double, int)} or {@link #constant(Long, int)} instead.
*
* @see org.apache.druid.math.expr.ConstantExpr
*/
public static <T> ExprVectorProcessor<T> constant(@Nullable String constant, int maxVectorSize)
public static <T> ExprVectorProcessor<T> constant(@Nullable Object constant, int maxVectorSize, ExpressionType type)
{
final Object[] strings = new Object[maxVectorSize];
Arrays.fill(strings, constant);
final ExprEvalObjectVector eval = new ExprEvalObjectVector(strings, ExpressionType.STRING);
if (type.isNumeric()) {
throw DruidException.defensive("Type[%s] should use the numeric constant creator instead", type);
}
final Object[] objects = new Object[maxVectorSize];
Arrays.fill(objects, constant);
final ExprEvalObjectVector eval = new ExprEvalObjectVector(objects, type);
return new ExprVectorProcessor<T>()
{
@Override
@ -101,7 +106,7 @@ public class VectorProcessors
@Override
public ExpressionType getOutputType()
{
return ExpressionType.STRING;
return type;
}
};
}
@ -194,15 +199,11 @@ public class VectorProcessors
if (inputType == null) {
// nil column, we can be anything, so be a string because it's the most flexible
// (numbers will be populated with default values in default mode and non-null)
return new IdentifierVectorProcessor<Object[]>(ExpressionType.STRING)
{
@Override
public ExprEvalVector<Object[]> evalVector(Expr.VectorInputBinding bindings)
{
return new ExprEvalObjectVector(bindings.getObjectVector(binding), ExpressionType.STRING);
}
};
if (NullHandling.sqlCompatible()) {
return constant((Long) null, inspector.getMaxVectorSize());
} else {
return constant(null, inspector.getMaxVectorSize(), ExpressionType.STRING);
}
}
switch (inputType.getType()) {
case LONG:
@ -292,35 +293,8 @@ public class VectorProcessors
}
final long[] outputValues = new long[inspector.getMaxVectorSize()];
ExprVectorProcessor<?> processor = null;
if (Types.is(type, ExprType.STRING)) {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 1L;
} else {
outputValues[i] = 0L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
} else if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<?> processor;
if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<long[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@ -376,6 +350,33 @@ public class VectorProcessors
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
} else {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 1L;
} else {
outputValues[i] = 0L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
@ -384,9 +385,6 @@ public class VectorProcessors
};
}
if (processor == null) {
throw Exprs.cannotVectorize();
}
return (ExprVectorProcessor<T>) processor;
}
@ -406,35 +404,8 @@ public class VectorProcessors
final long[] outputValues = new long[inspector.getMaxVectorSize()];
ExprVectorProcessor<?> processor = null;
if (Types.is(type, ExprType.STRING)) {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 0L;
} else {
outputValues[i] = 1L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
} else if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<?> processor;
if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<long[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@ -490,6 +461,33 @@ public class VectorProcessors
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
} else {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 0L;
} else {
outputValues[i] = 1L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
@ -498,9 +496,6 @@ public class VectorProcessors
};
}
if (processor == null) {
throw Exprs.cannotVectorize();
}
return (ExprVectorProcessor<T>) processor;
}

View File

@ -74,7 +74,8 @@ public class VectorStringProcessors
for (int i = 0; i < inputs.size(); i++) {
inputProcessors[i] = CastToTypeVectorProcessor.cast(
inputs.get(i).asVectorProcessor(inspector),
ExpressionType.STRING
ExpressionType.STRING,
inspector.getMaxVectorSize()
);
}
final ExprVectorProcessor processor = new ObjectOutMultiObjectInVectorProcessor(

View File

@ -134,7 +134,7 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
{
ExprVectorProcessor<?> processor;
processor = new LongOutLongInFunctionVectorValueProcessor(
CastToTypeVectorProcessor.cast(args.get(0).asVectorProcessor(inspector), ExpressionType.LONG),
CastToTypeVectorProcessor.cast(args.get(0).asVectorProcessor(inspector), ExpressionType.LONG, inspector.getMaxVectorSize()),
inspector.getMaxVectorSize()
)
{

View File

@ -117,7 +117,11 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
{
ExprVectorProcessor<?> processor;
processor = new LongOutLongInFunctionVectorValueProcessor(
CastToTypeVectorProcessor.cast(args.get(0).asVectorProcessor(inspector), ExpressionType.LONG),
CastToTypeVectorProcessor.cast(
args.get(0).asVectorProcessor(inspector),
ExpressionType.LONG,
inspector.getMaxVectorSize()
),
inspector.getMaxVectorSize()
)
{

View File

@ -182,16 +182,12 @@ public class ExpressionPlanner
traits.remove(ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE);
}
// vectorized expressions do not support incomplete, multi-valued inputs or outputs, or implicit mapping
// they also do not 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,
// vectorized expressions do not 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.INCOMPLETE_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.NON_SCALAR_OUTPUT
ExpressionPlan.Trait.INCOMPLETE_INPUTS
);
if (supportsVector && expression.canVectorize(inspector)) {

View File

@ -170,7 +170,8 @@ public class ExpressionVectorSelectors
return new ExpressionVectorObjectSelector(
CastToTypeVectorProcessor.cast(
VectorProcessors.identifier(binding, columnName),
ExpressionType.fromColumnType(castTo)
ExpressionType.fromColumnType(castTo),
binding.getMaxVectorSize()
),
binding
);
@ -190,7 +191,8 @@ public class ExpressionVectorSelectors
return new ExpressionVectorValueSelector(
CastToTypeVectorProcessor.cast(
VectorProcessors.identifier(binding, columnName),
ExpressionType.fromColumnType(castTo)
ExpressionType.fromColumnType(castTo),
binding.getMaxVectorSize()
),
binding
);

View File

@ -309,7 +309,7 @@ public class FunctionTest extends InitializedNullHandlingTest
public void testArrayConstructor()
{
assertArrayExpr("array(1, 2, 3, 4)", new Long[]{1L, 2L, 3L, 4L});
assertArrayExpr("array(1, 2, 3, 'bar')", new Long[]{1L, 2L, 3L, null});
assertArrayExpr("array(1, 2, 3, 'bar')", new String[]{"1", "2", "3", "bar"});
assertArrayExpr("array(1.0)", new Double[]{1.0});
assertArrayExpr("array('foo', 'bar')", new String[]{"foo", "bar"});
assertArrayExpr(
@ -1218,7 +1218,7 @@ public class FunctionTest extends InitializedNullHandlingTest
assertArrayExpr("array_to_mv(a)", new String[]{"foo", "bar", "baz", "foobar"});
assertArrayExpr("array_to_mv(b)", new String[]{"1", "2", "3", "4", "5"});
assertArrayExpr("array_to_mv(c)", new String[]{"3.1", "4.2", "5.3"});
assertArrayExpr("array_to_mv(array(y,z))", new String[]{"2", "3"});
assertArrayExpr("array_to_mv(array(y,z))", new String[]{"2.0", "3.1"});
// array type is determined by the first array type
assertArrayExpr("array_to_mv(array_concat(b,c))", new String[]{"1", "2", "3", "4", "5", "3", "4", "5"});
assertArrayExpr(

View File

@ -19,6 +19,7 @@
package org.apache.druid.math.expr;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.NonnullPair;
@ -26,6 +27,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.vector.ExprEvalVector;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.query.expression.NestedDataExpressions;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert;
import org.junit.Test;
@ -52,6 +54,12 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
private static final int NUM_ITERATIONS = 10;
private static final int VECTOR_SIZE = 512;
private static final ExprMacroTable MACRO_TABLE = new ExprMacroTable(
ImmutableList.of(
new NestedDataExpressions.JsonObjectExprMacro()
)
);
final Map<String, ExpressionType> types = ImmutableMap.<String, ExpressionType>builder()
.put("l1", ExpressionType.LONG)
.put("l2", ExpressionType.LONG)
@ -238,7 +246,7 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
public void testCast()
{
final String[] columns = new String[]{"d1", "l1", "s1"};
final String[] castTo = new String[]{"'STRING'", "'LONG'", "'DOUBLE'"};
final String[] castTo = new String[]{"'STRING'", "'LONG'", "'DOUBLE'", "'ARRAY<STRING>'", "'ARRAY<LONG>'", "'ARRAY<DOUBLE>'"};
final String[][] args = makeTemplateArgs(columns, castTo);
final String[] templates = new String[]{"cast(%s, %s)"};
testFunctions(types, templates, args);
@ -253,6 +261,22 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
testExpression("concat(s1,'-',s2,'-',l1,'-',d1)", types);
}
@Test
public void testArrayFns()
{
testExpression("array(s1, s2)", types);
testExpression("array(l1, l2)", types);
testExpression("array(d1, d2)", types);
testExpression("array(l1, d2)", types);
testExpression("array(s1, l2)", types);
}
@Test
public void testJsonFns()
{
testExpression("json_object('k1', s1, 'k2', l1)", types);
}
@Test
public void testConstants()
{
@ -263,6 +287,7 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
testExpression("Infinity", types);
testExpression("-Infinity", types);
testExpression("'hello'", types);
testExpression("json_object('a', 1, 'b', 'abc', 'c', 3.3, 'd', array(1,2,3))", types);
}
static void testFunctions(Map<String, ExpressionType> types, String[] templates, String[] args)
@ -288,7 +313,7 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
static void testExpression(String expr, Map<String, ExpressionType> types)
{
log.debug("[%s]", expr);
Expr parsed = Parser.parse(expr, ExprMacroTable.nil());
Expr parsed = Parser.parse(expr, MACRO_TABLE);
testExpression(expr, parsed, types, NUM_ITERATIONS);
testSequentialBinding(expr, parsed, types);
@ -315,11 +340,19 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
if (outputType != null && !eval.isNumericNull()) {
Assert.assertEquals(eval.type(), outputType);
}
Assert.assertEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
eval.valueOrDefault(),
vectorVals[i]
);
if (outputType != null && outputType.isArray()) {
Assert.assertArrayEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
(Object[]) eval.valueOrDefault(),
(Object[]) vectorVals[i]
);
} else {
Assert.assertEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
eval.valueOrDefault(),
vectorVals[i]
);
}
}
}
@ -363,11 +396,19 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
if (outputType != null && !eval.isNumericNull()) {
Assert.assertEquals(eval.type(), outputType);
}
Assert.assertEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
eval.valueOrDefault(),
vectorVals[i]
);
if (outputType != null && outputType.isArray()) {
Assert.assertArrayEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
(Object[]) eval.valueOrDefault(),
(Object[]) vectorVals[i]
);
} else {
Assert.assertEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
eval.valueOrDefault(),
vectorVals[i]
);
}
}
}
}

View File

@ -1151,7 +1151,8 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
Assert.assertTrue(
thePlan.is(
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.VECTORIZABLE
)
);
Assert.assertFalse(
@ -1160,8 +1161,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE,
ExpressionPlan.Trait.NON_SCALAR_OUTPUT,
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
ExpressionPlan.Trait.UNKNOWN_INPUTS,
ExpressionPlan.Trait.VECTORIZABLE
ExpressionPlan.Trait.UNKNOWN_INPUTS
)
);
@ -1251,7 +1251,8 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
ExpressionPlan thePlan = plan("array(long1, long2)");
Assert.assertTrue(
thePlan.is(
ExpressionPlan.Trait.NON_SCALAR_OUTPUT
ExpressionPlan.Trait.NON_SCALAR_OUTPUT,
ExpressionPlan.Trait.VECTORIZABLE
)
);
Assert.assertFalse(
@ -1261,8 +1262,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
ExpressionPlan.Trait.UNKNOWN_INPUTS,
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.VECTORIZABLE
ExpressionPlan.Trait.NON_SCALAR_INPUTS
)
);
Assert.assertEquals(ExpressionType.LONG_ARRAY, thePlan.getOutputType());
@ -1387,7 +1387,8 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
{
Assert.assertTrue(
thePlan.is(
ExpressionPlan.Trait.NON_SCALAR_INPUTS
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.VECTORIZABLE
)
);
Assert.assertFalse(
@ -1397,8 +1398,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
ExpressionPlan.Trait.NON_SCALAR_OUTPUT,
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
ExpressionPlan.Trait.UNKNOWN_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.VECTORIZABLE
ExpressionPlan.Trait.NEEDS_APPLIED
)
);
}
@ -1408,7 +1408,8 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
Assert.assertTrue(
thePlan.is(
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.NON_SCALAR_OUTPUT
ExpressionPlan.Trait.NON_SCALAR_OUTPUT,
ExpressionPlan.Trait.VECTORIZABLE
)
);
Assert.assertFalse(
@ -1417,8 +1418,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE,
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
ExpressionPlan.Trait.UNKNOWN_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.VECTORIZABLE
ExpressionPlan.Trait.NEEDS_APPLIED
)
);
}

View File

@ -1525,9 +1525,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayLength()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
testQuery(
"SELECT dim1, ARRAY_LENGTH(dim3), SUM(cnt) FROM druid.numfoo GROUP BY 1, 2 ORDER BY 2 DESC",
ImmutableList.of(
@ -1740,9 +1737,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayPrependAppend()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
ImmutableList<Object[]> results;
if (useDefault) {
results = ImmutableList.of(
@ -1861,9 +1855,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayOffset()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
testQuery(
"SELECT ARRAY_OFFSET(dim3, 1), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -2185,9 +2176,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayOrdinal()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
testQuery(
"SELECT ARRAY_ORDINAL(dim3, 2), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -2228,9 +2216,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayOffsetOf()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
testQuery(
"SELECT ARRAY_OFFSET_OF(dim3, 'b'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -2277,9 +2262,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayOrdinalOf()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
testQuery(
"SELECT ARRAY_ORDINAL_OF(dim3, 'b'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -2327,9 +2309,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayToString()
{
// Cannot vectorize due to array expressions.
cannotVectorize();
ImmutableList<Object[]> results;
if (useDefault) {
results = ImmutableList.of(

View File

@ -469,9 +469,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringLength()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
testQuery(
"SELECT dim1, MV_LENGTH(dim3), SUM(cnt) FROM druid.numfoo GROUP BY 1, 2 ORDER BY 2 DESC",
ImmutableList.of(
@ -632,9 +629,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringPrependAppend()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
ImmutableList<Object[]> results;
if (useDefault) {
results = ImmutableList.of(
@ -818,9 +812,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringOffset()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
testQuery(
"SELECT MV_OFFSET(dim3, 1), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -857,9 +848,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringOrdinal()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
testQuery(
"SELECT MV_ORDINAL(dim3, 2), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -900,9 +888,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringOffsetOf()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
testQuery(
"SELECT MV_OFFSET_OF(dim3, 'b'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -949,9 +934,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringOrdinalOf()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
testQuery(
"SELECT MV_ORDINAL_OF(dim3, 'b'), SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
ImmutableList.of(
@ -999,9 +981,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringToString()
{
// Cannot vectorize due to usage of expressions.
cannotVectorize();
ImmutableList<Object[]> results;
if (useDefault) {
results = ImmutableList.of(

View File

@ -5848,9 +5848,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi'), COUNT(*)\n"
+ "FROM druid.foo\n"
@ -5914,9 +5911,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInOrIsNullExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi') OR dim1 IS NULL, COUNT(*)\n"
+ "FROM druid.foo\n"
@ -5948,9 +5942,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testNotInOrIsNullExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT NOT (dim1 IN ('abc', 'def', 'ghi') OR dim1 IS NULL), COUNT(*)\n"
+ "FROM druid.foo\n"
@ -5982,9 +5973,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testNotInAndIsNotNullExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 NOT IN ('abc', 'def', 'ghi') AND dim1 IS NOT NULL, COUNT(*)\n"
+ "FROM druid.foo\n"
@ -6016,9 +6004,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInOrGreaterThanExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi') OR dim1 > 'zzz', COUNT(*)\n"
+ "FROM druid.foo\n"
@ -6050,9 +6035,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testNotInAndLessThanExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 NOT IN ('abc', 'def', 'ghi') AND dim1 < 'zzz', COUNT(*)\n"
+ "FROM druid.foo\n"
@ -6084,9 +6066,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testNotInOrEqualToOneOfThemExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 NOT IN ('abc', 'def', 'ghi') OR dim1 = 'def', COUNT(*)\n"
+ "FROM druid.foo\n"