org.apache.druid.math.expr.Function Maven / Gradle / Ivy
Go to download
Show more of this group Show more artifacts with this name
Show all versions of druid-processing Show documentation
Show all versions of druid-processing Show documentation
A module that is everything required to understands Druid Segments
/*
* 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.ImmutableSet;
import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorMathProcessors;
import org.apache.druid.math.expr.vector.VectorProcessors;
import org.apache.druid.math.expr.vector.VectorStringProcessors;
import org.apache.druid.segment.column.TypeSignature;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
import javax.annotation.Nullable;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.function.BinaryOperator;
import java.util.function.DoubleBinaryOperator;
import java.util.function.LongBinaryOperator;
import java.util.stream.Collectors;
/**
* Base interface describing the mechanism used to evaluate a {@link FunctionExpr}. All {@link Function} implementations
* are immutable.
*
* Do NOT remove "unused" members in this class. They are used by generated Antlr
*/
@SuppressWarnings("unused")
public interface Function extends NamedFunction
{
/**
* Possibly convert a {@link Function} into an optimized, possibly not thread-safe {@link Function}.
*/
default Function asSingleThreaded(List args, Expr.InputBindingInspector inspector)
{
return this;
}
/**
* Evaluate the function, given a list of arguments and a set of bindings to provide values for {@link IdentifierExpr}.
*/
ExprEval apply(List args, Expr.ObjectBinding bindings);
/**
* Given a list of arguments to this {@link Function}, get the set of arguments that must evaluate to a scalar value
*/
default Set getScalarInputs(List args)
{
return ImmutableSet.copyOf(args);
}
/**
* Given a list of arguments to this {@link Function}, get the set of arguments that must evaluate to an array
* value
*/
default Set getArrayInputs(List args)
{
return Collections.emptySet();
}
/**
* Returns true if a function expects any array arguments
*/
default boolean hasArrayInputs()
{
return false;
}
/**
* Returns true if function produces an array. All {@link Function} implementations are expected to
* exclusively produce either scalar or array values.
*/
default boolean hasArrayOutput()
{
return false;
}
/**
* Validate function arguments. This method is called whenever a {@link FunctionExpr} is created, and should validate
* everything that is feasible up front. Note that input type information is typically unavailable at the time
* {@link Expr} are parsed, and so this method is incapable of performing complete validation.
*/
void validateArguments(List args);
/**
* Compute the output type of this function for a given set of argument expression inputs.
*
* @see Expr#getOutputType
*/
@Nullable
ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args);
/**
* Check if a function can be 'vectorized', for a given set of {@link Expr} inputs. 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.InputBindingInspector)
* @see ApplyFunction#canVectorize(Expr.InputBindingInspector, Expr, List)
*/
default boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return false;
}
/**
* Builds a 'vectorized' function expression processor, that can build vectorized processors for its input values
* using {@link Expr#asVectorProcessor}, for use in vectorized query engines.
*
* @see Expr#asVectorProcessor(Expr.VectorInputBindingInspector)
* @see ApplyFunction#asVectorProcessor(Expr.VectorInputBindingInspector, Expr, List)
*/
default ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
throw new UOE("Function[%s] is not vectorized", name());
}
/**
* Base class for a single variable input {@link Function} implementation
*/
abstract class UnivariateFunction implements Function
{
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 1);
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
Expr expr = args.get(0);
return eval(expr.eval(bindings));
}
protected abstract ExprEval eval(ExprEval param);
}
/**
* Base class for a 2 variable input {@link Function} implementation
*/
abstract class BivariateFunction implements Function
{
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 2);
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
Expr expr1 = args.get(0);
Expr expr2 = args.get(1);
return eval(expr1.eval(bindings), expr2.eval(bindings));
}
protected abstract ExprEval eval(ExprEval x, ExprEval y);
}
/**
* Base class for a single variable input mathematical {@link Function}, with specialized 'eval' implementations that
* that operate on primitive number types
*/
abstract class UnivariateMathFunction extends UnivariateFunction
{
@Override
protected final ExprEval eval(ExprEval param)
{
if (NullHandling.sqlCompatible() && param.isNumericNull()) {
return ExprEval.of(null);
}
if (param.type().is(ExprType.LONG)) {
return eval(param.asLong());
} else if (param.type().is(ExprType.DOUBLE)) {
return eval(param.asDouble());
}
return ExprEval.of(null);
}
protected ExprEval eval(long param)
{
return eval((double) param);
}
protected ExprEval eval(double param)
{
if (param < Long.MIN_VALUE || param > Long.MAX_VALUE) {
throw validationFailed(
"Possible data truncation, param [%f] is out of LONG value range",
param
);
}
return eval((long) param);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return args.get(0).getOutputType(inspector);
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
// can not vectorize in default mode for 'missing' columns
// it creates inconsistencies as we default the output type to STRING, making the value null
// but the numeric columns expect a non null value
final ExpressionType outputType = args.get(0).getOutputType(inspector);
if (outputType == null && NullHandling.replaceWithDefault()) {
return false;
}
return (outputType == null || outputType.isNumeric()) && inspector.canVectorize(args);
}
}
/**
* Many math functions always output a {@link Double} primitive, regardless of input type.
*/
abstract class DoubleUnivariateMathFunction extends UnivariateMathFunction
{
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.DOUBLE;
}
}
/**
* Base class for a 2 variable input mathematical {@link Function}, with specialized 'eval' implementations that
* operate on primitive number types
*/
abstract class BivariateMathFunction extends BivariateFunction
{
@Override
protected final ExprEval eval(ExprEval x, ExprEval y)
{
// match the logic of BinaryEvalOpExprBase.eval, except there is no string handling so both strings is also null
if (NullHandling.sqlCompatible() && (x.value() == null || y.value() == null)) {
return ExprEval.of(null);
}
ExpressionType type = ExpressionTypeConversion.autoDetect(x, y);
switch (type.getType()) {
case STRING:
return ExprEval.of(null);
case LONG:
return eval(x.asLong(), y.asLong());
case DOUBLE:
default:
return eval(x.asDouble(), y.asDouble());
}
}
protected ExprEval eval(long x, long y)
{
return eval((double) x, (double) y);
}
protected ExprEval eval(double x, double y)
{
return eval((long) x, (long) y);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionTypeConversion.function(
args.get(0).getOutputType(inspector),
args.get(1).getOutputType(inspector)
);
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return inspector.areNumeric(args) && inspector.canVectorize(args);
}
}
/**
* Many math functions always output a {@link Double} primitive, regardless of input type.
*/
abstract class DoubleBivariateMathFunction extends BivariateMathFunction
{
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.DOUBLE;
}
}
abstract class BivariateBitwiseMathFunction extends BivariateFunction
{
@Override
protected final ExprEval eval(ExprEval x, ExprEval y)
{
// this is a copy of the logic of BivariateMathFunction for string handling, which itself is a
// remix of BinaryEvalOpExprBase.eval modified so that string inputs are always null outputs
if (NullHandling.sqlCompatible() && (x.value() == null || y.value() == null)) {
return ExprEval.of(null);
}
ExpressionType type = ExpressionTypeConversion.autoDetect(x, y);
if (type.is(ExprType.STRING)) {
return ExprEval.of(null);
}
return eval(x.asLong(), y.asLong());
}
protected abstract ExprEval eval(long x, long y);
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return inspector.areNumeric(args) && inspector.canVectorize(args);
}
}
/**
* Base class for a 2 variable input {@link Function} whose first argument is a {@link ExprType#STRING} and second
* argument is {@link ExprType#LONG}
*/
abstract class StringLongFunction extends BivariateFunction
{
@Override
protected final ExprEval eval(ExprEval x, ExprEval y)
{
if (!x.type().is(ExprType.STRING) || !y.type().is(ExprType.LONG)) {
throw validationFailed("needs a STRING as first argument and a LONG as second argument");
}
return eval(x.asString(), y.asInt());
}
protected abstract ExprEval eval(@Nullable String x, int y);
}
/**
* {@link Function} that takes 1 array operand and 1 scalar operand
*/
abstract class ArrayScalarFunction implements Function
{
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 2);
}
@Override
public Set getScalarInputs(List args)
{
return ImmutableSet.of(getScalarArgument(args));
}
@Override
public Set getArrayInputs(List args)
{
return ImmutableSet.of(getArrayArgument(args));
}
@Override
public boolean hasArrayInputs()
{
return true;
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval arrayExpr = getArrayArgument(args).eval(bindings);
final ExprEval scalarExpr = getScalarArgument(args).eval(bindings);
if (arrayExpr.asArray() == null) {
return ExprEval.of(null);
}
return doApply(arrayExpr, scalarExpr);
}
Expr getScalarArgument(List args)
{
return args.get(1);
}
Expr getArrayArgument(List args)
{
return args.get(0);
}
abstract ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr);
}
/**
* {@link Function} that takes 2 array operands
*/
abstract class ArraysFunction implements Function
{
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 2);
}
@Override
public Set getScalarInputs(List args)
{
return Collections.emptySet();
}
@Override
public Set getArrayInputs(List args)
{
return ImmutableSet.copyOf(args);
}
@Override
public boolean hasArrayInputs()
{
return true;
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval arrayExpr1 = args.get(0).eval(bindings);
final ExprEval arrayExpr2 = args.get(1).eval(bindings);
if (arrayExpr1.asArray() == null) {
return arrayExpr1;
}
if (arrayExpr2.asArray() == null) {
return arrayExpr2;
}
return doApply(arrayExpr1, arrayExpr2);
}
abstract ExprEval doApply(ExprEval lhsExpr, ExprEval rhsExpr);
}
/**
* Scaffolding for a 2 argument {@link Function} which accepts one array and one scalar input and adds the scalar
* input to the array in some way.
*/
abstract class ArrayAddElementFunction extends ArrayScalarFunction
{
@Override
public boolean hasArrayOutput()
{
return true;
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
ExpressionType arrayType = getArrayArgument(args).getOutputType(inspector);
return Optional.ofNullable(ExpressionType.asArrayType(arrayType)).orElse(arrayType);
}
@Override
ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr)
{
final ExpressionType arrayType = arrayExpr.asArrayType();
if (!scalarExpr.type().equals(arrayExpr.elementType())) {
// try to cast
ExprEval coerced = scalarExpr.castTo(arrayExpr.elementType());
return ExprEval.ofArray(arrayType, add(arrayType.getElementType(), arrayExpr.asArray(), coerced.value()));
}
return ExprEval.ofArray(arrayType, add(arrayType.getElementType(), arrayExpr.asArray(), scalarExpr.value()));
}
abstract Object[] add(TypeSignature elementType, T[] array, @Nullable T val);
}
/**
* Base scaffolding for functions which accept 2 array arguments and combine them in some way
*/
abstract class ArraysMergeFunction extends ArraysFunction
{
@Override
public Set getArrayInputs(List args)
{
return ImmutableSet.copyOf(args);
}
@Override
public boolean hasArrayOutput()
{
return true;
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
ExpressionType arrayType = args.get(0).getOutputType(inspector);
return Optional.ofNullable(ExpressionType.asArrayType(arrayType)).orElse(arrayType);
}
@Override
ExprEval doApply(ExprEval lhsExpr, ExprEval rhsExpr)
{
final Object[] array1 = lhsExpr.asArray();
final Object[] array2 = rhsExpr.asArray();
if (array1 == null) {
return ExprEval.of(null);
}
if (array2 == null) {
return lhsExpr;
}
final ExpressionType arrayType = lhsExpr.asArrayType();
if (!lhsExpr.asArrayType().equals(rhsExpr.asArrayType())) {
// try to cast if they types don't match
ExprEval coerced = rhsExpr.castTo(arrayType);
ExprEval.ofArray(arrayType, merge(arrayType.getElementType(), lhsExpr.asArray(), coerced.asArray()));
}
return ExprEval.ofArray(arrayType, merge(arrayType.getElementType(), lhsExpr.asArray(), rhsExpr.asArray()));
}
abstract Object[] merge(TypeSignature elementType, T[] array1, T[] array2);
}
abstract class ReduceFunction implements Function
{
private final DoubleBinaryOperator doubleReducer;
private final LongBinaryOperator longReducer;
private final BinaryOperator stringReducer;
ReduceFunction(
DoubleBinaryOperator doubleReducer,
LongBinaryOperator longReducer,
BinaryOperator stringReducer
)
{
this.doubleReducer = doubleReducer;
this.longReducer = longReducer;
this.stringReducer = stringReducer;
}
@Override
public void validateArguments(List args)
{
// anything goes
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
ExpressionType outputType = ExpressionType.LONG;
for (Expr expr : args) {
outputType = ExpressionTypeConversion.function(outputType, expr.getOutputType(inspector));
}
return outputType;
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
if (args.isEmpty()) {
return ExprEval.of(null);
}
// evaluate arguments and collect output type
List> evals = new ArrayList<>();
ExpressionType outputType = ExpressionType.LONG;
for (Expr expr : args) {
ExprEval> exprEval = expr.eval(bindings);
ExpressionType exprType = exprEval.type();
if (isValidType(exprType)) {
outputType = ExpressionTypeConversion.function(outputType, exprType);
}
if (exprEval.value() != null) {
evals.add(exprEval);
}
}
if (evals.isEmpty()) {
// The GREATEST/LEAST functions are not in the SQL standard. Emulate the behavior of postgres (return null if
// all expressions are null, otherwise skip null values) since it is used as a base for a wide number of
// databases. This also matches the behavior the long/double greatest/least post aggregators. Some other
// databases (e.g., MySQL) return null if any expression is null.
// https://www.postgresql.org/docs/9.5/functions-conditional.html
// https://dev.mysql.com/doc/refman/8.0/en/comparison-operators.html#function_least
return ExprEval.of(null);
}
switch (outputType.getType()) {
case DOUBLE:
//noinspection OptionalGetWithoutIsPresent (empty list handled earlier)
return ExprEval.of(evals.stream().mapToDouble(ExprEval::asDouble).reduce(doubleReducer).getAsDouble());
case LONG:
//noinspection OptionalGetWithoutIsPresent (empty list handled earlier)
return ExprEval.of(evals.stream().mapToLong(ExprEval::asLong).reduce(longReducer).getAsLong());
default:
//noinspection OptionalGetWithoutIsPresent (empty list handled earlier)
return ExprEval.of(evals.stream().map(ExprEval::asString).reduce(stringReducer).get());
}
}
private boolean isValidType(ExpressionType exprType)
{
switch (exprType.getType()) {
case DOUBLE:
case LONG:
case STRING:
return true;
default:
throw validationFailed("does not accept %s types", exprType);
}
}
}
// ------------------------------ implementations ------------------------------
class ParseLong implements Function
{
@Override
public String name()
{
return "parse_long";
}
@Override
public void validateArguments(List args)
{
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final int radix = args.size() == 1 ? 10 : args.get(1).eval(bindings).asInt();
final String input = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString());
if (input == null) {
return ExprEval.ofLong(null);
}
final long retVal;
try {
if (radix == 16 && (input.startsWith("0x") || input.startsWith("0X"))) {
// Strip leading 0x from hex strings.
retVal = Long.parseLong(input.substring(2), radix);
} else {
retVal = Long.parseLong(input, radix);
}
}
catch (NumberFormatException e) {
return ExprEval.ofLong(null);
}
return ExprEval.of(retVal);
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return (args.size() == 1 || (args.get(1).isLiteral() && args.get(1).getLiteralValue() instanceof Number)) &&
inspector.canVectorize(args);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
if (args.size() == 1 || args.get(1).isLiteral()) {
final int radix = args.size() == 1 ? 10 : ((Number) args.get(1).getLiteralValue()).intValue();
return VectorProcessors.parseLong(inspector, args.get(0), radix);
}
// only single argument and 2 argument where the radix is constant is currently implemented
// the canVectorize check should prevent this from happening, but explode just in case
throw Exprs.cannotVectorize(this);
}
}
class Pi implements Function
{
private static final double PI = Math.PI;
@Override
public String name()
{
return "pi";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
return ExprEval.of(PI);
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 0);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.DOUBLE;
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return true;
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorProcessors.constant(PI, inspector.getMaxVectorSize());
}
}
class Abs extends UnivariateMathFunction
{
@Override
public String name()
{
return "abs";
}
@Override
protected ExprEval eval(long param)
{
return ExprEval.of(Math.abs(param));
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.abs(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.abs(inspector, args.get(0));
}
}
class Acos extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "acos";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.acos(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.acos(inspector, args.get(0));
}
}
class Asin extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "asin";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.asin(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.asin(inspector, args.get(0));
}
}
class Atan extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "atan";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.atan(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.atan(inspector, args.get(0));
}
}
class BitwiseComplement extends UnivariateMathFunction
{
@Override
public String name()
{
return "bitwiseComplement";
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
@Override
protected ExprEval eval(long param)
{
return ExprEval.of(~param);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseComplement(inspector, args.get(0));
}
}
class BitwiseConvertLongBitsToDouble extends UnivariateMathFunction
{
@Override
public String name()
{
return "bitwiseConvertLongBitsToDouble";
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
ExpressionType type = args.get(0).getOutputType(inspector);
if (type == null) {
return null;
}
return ExpressionType.DOUBLE;
}
@Override
protected ExprEval eval(long param)
{
return ExprEval.of(Double.longBitsToDouble(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseConvertLongBitsToDouble(inspector, args.get(0));
}
}
class BitwiseConvertDoubleToLongBits extends UnivariateMathFunction
{
@Override
public String name()
{
return "bitwiseConvertDoubleToLongBits";
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
ExpressionType type = args.get(0).getOutputType(inspector);
if (type == null) {
return null;
}
return ExpressionType.LONG;
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Double.doubleToLongBits(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseConvertDoubleToLongBits(inspector, args.get(0));
}
}
class BitwiseAnd extends BivariateBitwiseMathFunction
{
@Override
public String name()
{
return "bitwiseAnd";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(x & y);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseAnd(inspector, args.get(0), args.get(1));
}
}
class BitwiseOr extends BivariateBitwiseMathFunction
{
@Override
public String name()
{
return "bitwiseOr";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(x | y);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseOr(inspector, args.get(0), args.get(1));
}
}
class BitwiseShiftLeft extends BivariateBitwiseMathFunction
{
@Override
public String name()
{
return "bitwiseShiftLeft";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(x << y);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseShiftLeft(inspector, args.get(0), args.get(1));
}
}
class BitwiseShiftRight extends BivariateBitwiseMathFunction
{
@Override
public String name()
{
return "bitwiseShiftRight";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(x >> y);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseShiftRight(inspector, args.get(0), args.get(1));
}
}
class BitwiseXor extends BivariateBitwiseMathFunction
{
@Override
public String name()
{
return "bitwiseXor";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(x ^ y);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.bitwiseXor(inspector, args.get(0), args.get(1));
}
}
class Cbrt extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "cbrt";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.cbrt(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.cbrt(inspector, args.get(0));
}
}
class Ceil extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "ceil";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.ceil(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.ceil(inspector, args.get(0));
}
}
class Cos extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "cos";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.cos(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.cos(inspector, args.get(0));
}
}
class Cosh extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "cosh";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.cosh(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.cosh(inspector, args.get(0));
}
}
class Cot extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "cot";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.cos(param) / Math.sin(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.cot(inspector, args.get(0));
}
}
class SafeDivide extends BivariateMathFunction
{
public static final String NAME = "safe_divide";
@Override
public String name()
{
return NAME;
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionTypeConversion.function(
args.get(0).getOutputType(inspector),
args.get(1).getOutputType(inspector)
);
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return false;
}
@Override
protected ExprEval eval(final long x, final long y)
{
if (y == 0) {
return ExprEval.ofLong(null);
}
return ExprEval.ofLong(x / y);
}
@Override
protected ExprEval eval(final double x, final double y)
{
if (y == 0 || Double.isNaN(y)) {
if (x != 0) {
return ExprEval.ofDouble(null);
}
return ExprEval.ofDouble(0);
}
return ExprEval.ofDouble(x / y);
}
}
class Div extends BivariateMathFunction
{
@Override
public String name()
{
return "div";
}
@Override
protected ExprEval eval(final long x, final long y)
{
return ExprEval.of(x / y);
}
@Override
protected ExprEval eval(final double x, final double y)
{
return ExprEval.of((long) (x / y));
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionTypeConversion.integerMathFunction(
args.get(0).getOutputType(inspector),
args.get(1).getOutputType(inspector)
);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.longDivide(inspector, args.get(0), args.get(1));
}
}
class Exp extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "exp";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.exp(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.exp(inspector, args.get(0));
}
}
class Expm1 extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "expm1";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.expm1(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.expm1(inspector, args.get(0));
}
}
class Floor extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "floor";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.floor(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.floor(inspector, args.get(0));
}
}
class GetExponent extends UnivariateMathFunction
{
@Override
public String name()
{
return "getExponent";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.getExponent(param));
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.getExponent(inspector, args.get(0));
}
}
class Log extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "log";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.log(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.log(inspector, args.get(0));
}
}
class Log10 extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "log10";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.log10(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.log10(inspector, args.get(0));
}
}
class Log1p extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "log1p";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.log1p(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.log1p(inspector, args.get(0));
}
}
class NextUp extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "nextUp";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.nextUp(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.nextUp(inspector, args.get(0));
}
}
class Rint extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "rint";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.rint(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.rint(inspector, args.get(0));
}
}
class Round implements Function
{
//CHECKSTYLE.OFF: Regexp
private static final BigDecimal MAX_FINITE_VALUE = BigDecimal.valueOf(Double.MAX_VALUE);
private static final BigDecimal MIN_FINITE_VALUE = BigDecimal.valueOf(-1 * Double.MAX_VALUE);
//CHECKSTYLE.ON: Regexp
public static final String NAME = "round";
@Override
public String name()
{
return NAME;
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
ExprEval value1 = args.get(0).eval(bindings);
if (NullHandling.sqlCompatible() && value1.isNumericNull()) {
return ExprEval.of(null);
}
if (!value1.type().anyOf(ExprType.LONG, ExprType.DOUBLE)) {
throw validationFailed(
"first argument should be a LONG or DOUBLE but got %s instead",
value1.type()
);
}
if (args.size() == 1) {
return eval(value1);
} else {
ExprEval value2 = args.get(1).eval(bindings);
if (!value2.type().is(ExprType.LONG)) {
throw validationFailed(
"second argument should be a LONG but got %s instead",
value2.type()
);
}
return eval(value1, value2.asInt());
}
}
@Override
public void validateArguments(List args)
{
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return args.get(0).getOutputType(inspector);
}
private ExprEval eval(ExprEval param)
{
return eval(param, 0);
}
private ExprEval eval(ExprEval param, int scale)
{
if (param.type().is(ExprType.LONG)) {
return ExprEval.of(BigDecimal.valueOf(param.asLong()).setScale(scale, RoundingMode.HALF_UP).longValue());
} else if (param.type().is(ExprType.DOUBLE)) {
BigDecimal decimal = safeGetFromDouble(param.asDouble());
return ExprEval.of(decimal.setScale(scale, RoundingMode.HALF_UP).doubleValue());
} else {
return ExprEval.of(null);
}
}
/**
* Converts non-finite doubles to BigDecimal values instead of throwing a NumberFormatException.
*/
private static BigDecimal safeGetFromDouble(double val)
{
if (Double.isNaN(val)) {
return BigDecimal.ZERO;
} else if (val == Double.POSITIVE_INFINITY) {
return MAX_FINITE_VALUE;
} else if (val == Double.NEGATIVE_INFINITY) {
return MIN_FINITE_VALUE;
}
return BigDecimal.valueOf(val);
}
}
class Signum extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "signum";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.signum(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.signum(inspector, args.get(0));
}
}
class Sin extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "sin";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.sin(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.sin(inspector, args.get(0));
}
}
class Sinh extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "sinh";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.sinh(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.sinh(inspector, args.get(0));
}
}
class Sqrt extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "sqrt";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.sqrt(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.sqrt(inspector, args.get(0));
}
}
class Tan extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "tan";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.tan(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.tan(inspector, args.get(0));
}
}
class Tanh extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "tanh";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.tanh(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.tanh(inspector, args.get(0));
}
}
class ToDegrees extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "toDegrees";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.toDegrees(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.toDegrees(inspector, args.get(0));
}
}
class ToRadians extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "toRadians";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.toRadians(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.toRadians(inspector, args.get(0));
}
}
class Ulp extends DoubleUnivariateMathFunction
{
@Override
public String name()
{
return "ulp";
}
@Override
protected ExprEval eval(double param)
{
return ExprEval.of(Math.ulp(param));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.ulp(inspector, args.get(0));
}
}
class Atan2 extends DoubleBivariateMathFunction
{
@Override
public String name()
{
return "atan2";
}
@Override
protected ExprEval eval(double y, double x)
{
return ExprEval.of(Math.atan2(y, x));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.atan2(inspector, args.get(0), args.get(1));
}
}
class CopySign extends DoubleBivariateMathFunction
{
@Override
public String name()
{
return "copySign";
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.copySign(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.copySign(inspector, args.get(0), args.get(1));
}
}
class Hypot extends DoubleBivariateMathFunction
{
@Override
public String name()
{
return "hypot";
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.hypot(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.hypot(inspector, args.get(0), args.get(1));
}
}
class Remainder extends DoubleBivariateMathFunction
{
@Override
public String name()
{
return "remainder";
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.IEEEremainder(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.remainder(inspector, args.get(0), args.get(1));
}
}
class Max extends BivariateMathFunction
{
@Override
public String name()
{
return "max";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(Math.max(x, y));
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.max(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.max(inspector, args.get(0), args.get(1));
}
}
class Min extends BivariateMathFunction
{
@Override
public String name()
{
return "min";
}
@Override
protected ExprEval eval(long x, long y)
{
return ExprEval.of(Math.min(x, y));
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.min(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.min(inspector, args.get(0), args.get(1));
}
}
class NextAfter extends DoubleBivariateMathFunction
{
@Override
public String name()
{
return "nextAfter";
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.nextAfter(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.nextAfter(inspector, args.get(0), args.get(1));
}
}
class Pow extends DoubleBivariateMathFunction
{
@Override
public String name()
{
return "pow";
}
@Override
protected ExprEval eval(double x, double y)
{
return ExprEval.of(Math.pow(x, y));
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.doublePower(inspector, args.get(0), args.get(1));
}
}
class Scalb extends BivariateFunction
{
@Override
public String name()
{
return "scalb";
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.DOUBLE;
}
@Override
protected ExprEval eval(ExprEval x, ExprEval y)
{
if (NullHandling.sqlCompatible() && (x.value() == null || y.value() == null)) {
return ExprEval.of(null);
}
ExpressionType type = ExpressionTypeConversion.autoDetect(x, y);
switch (type.getType()) {
case STRING:
return ExprEval.of(null);
default:
return ExprEval.of(Math.scalb(x.asDouble(), y.asInt()));
}
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return inspector.areNumeric(args) && inspector.canVectorize(args);
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return VectorMathProcessors.scalb(inspector, args.get(0), args.get(1));
}
}
class CastFunc extends BivariateFunction
{
@Override
public String name()
{
return "cast";
}
@Override
protected ExprEval eval(ExprEval x, ExprEval y)
{
if (NullHandling.sqlCompatible() && x.value() == null) {
return ExprEval.of(null);
}
ExpressionType castTo;
try {
castTo = ExpressionType.fromString(StringUtils.toUpperCase(y.asString()));
}
catch (IllegalArgumentException e) {
throw validationFailed("Invalid type [%s]", y.asString());
}
return x.castTo(castTo);
}
@Override
public Set getScalarInputs(List args)
{
if (args.get(1).isLiteral()) {
ExpressionType castTo = ExpressionType.fromString(
StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())
);
switch (castTo.getType()) {
case ARRAY:
return Collections.emptySet();
default:
return ImmutableSet.of(args.get(0));
}
}
// unknown cast, can't safely assume either way
return Collections.emptySet();
}
@Override
public Set getArrayInputs(List args)
{
if (args.get(1).isLiteral()) {
ExpressionType castTo = ExpressionType.fromString(
StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())
);
switch (castTo.getType()) {
case LONG:
case DOUBLE:
case STRING:
return Collections.emptySet();
default:
return ImmutableSet.of(args.get(0));
}
}
// unknown cast, can't safely assume either way
return Collections.emptySet();
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
// can only know cast output type if cast to argument is constant
if (args.get(1).isLiteral()) {
return ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString()));
}
return null;
}
@Override
public boolean canVectorize(Expr.InputBindingInspector inspector, List args)
{
return args.get(0).canVectorize(inspector) && args.get(1).isLiteral();
}
@Override
public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspector inspector, List args)
{
return CastToTypeVectorProcessor.cast(
args.get(0).asVectorProcessor(inspector),
ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString()))
);
}
}
class GreatestFunc extends ReduceFunction
{
public static final String NAME = "greatest";
public GreatestFunc()
{
super(
Math::max,
Math::max,
BinaryOperator.maxBy(Comparator.naturalOrder())
);
}
@Override
public String name()
{
return NAME;
}
}
class LeastFunc extends ReduceFunction
{
public static final String NAME = "least";
public LeastFunc()
{
super(
Math::min,
Math::min,
BinaryOperator.minBy(Comparator.naturalOrder())
);
}
@Override
public String name()
{
return NAME;
}
}
class ConditionFunc implements Function
{
@Override
public String name()
{
return "if";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
ExprEval x = args.get(0).eval(bindings);
return x.asBoolean() ? args.get(1).eval(bindings) : args.get(2).eval(bindings);
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 3);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionTypeConversion.conditional(inspector, args.subList(1, 3));
}
}
/**
* "Searched CASE" function, similar to {@code CASE WHEN boolean_expr THEN result [ELSE else_result] END} in SQL.
*/
class CaseSearchedFunc implements Function
{
@Override
public String name()
{
return "case_searched";
}
@Override
public ExprEval apply(final List args, final Expr.ObjectBinding bindings)
{
for (int i = 0; i < args.size(); i += 2) {
if (i == args.size() - 1) {
// ELSE else_result.
return args.get(i).eval(bindings);
} else if (args.get(i).eval(bindings).asBoolean()) {
// Matching WHEN boolean_expr THEN result
return args.get(i + 1).eval(bindings);
}
}
return ExprEval.of(null);
}
@Override
public void validateArguments(List args)
{
validationHelperCheckMinArgumentCount(args, 2);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
List results = new ArrayList<>();
for (int i = 1; i < args.size(); i += 2) {
results.add(args.get(i));
}
// add else
results.add(args.get(args.size() - 1));
return ExpressionTypeConversion.conditional(inspector, results);
}
}
/**
* "Simple CASE" function, similar to {@code CASE expr WHEN value THEN result [ELSE else_result] END} in SQL.
*/
class CaseSimpleFunc implements Function
{
@Override
public String name()
{
return "case_simple";
}
@Override
public ExprEval apply(final List args, final Expr.ObjectBinding bindings)
{
for (int i = 1; i < args.size(); i += 2) {
if (i == args.size() - 1) {
// ELSE else_result.
return args.get(i).eval(bindings);
} else if (new BinEqExpr("==", args.get(0), args.get(i)).eval(bindings).asBoolean()) {
// Matching WHEN value THEN result
return args.get(i + 1).eval(bindings);
}
}
return ExprEval.of(null);
}
@Override
public void validateArguments(List args)
{
validationHelperCheckMinArgumentCount(args, 3);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
List results = new ArrayList<>();
for (int i = 2; i < args.size(); i += 2) {
results.add(args.get(i));
}
// add else
results.add(args.get(args.size() - 1));
return ExpressionTypeConversion.conditional(inspector, results);
}
}
/**
* nvl is like coalesce, but accepts exactly two arguments.
*/
class NvlFunc extends CoalesceFunc
{
@Override
public String name()
{
return "nvl";
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 2);
}
}
/**
* SQL function "x IS NOT DISTINCT FROM y". Very similar to "x = y", i.e. {@link BinEqExpr}, except this function
* never returns null, and this function considers NULL as a value, so NULL itself is not-distinct-from NULL. For
* example: `x == null` returns `null` in SQL-compatible null handling mode, but `notdistinctfrom(x, null)` is
* true if `x` is null.
*/
class IsNotDistinctFromFunc implements Function
{
@Override
public String name()
{
return "notdistinctfrom";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval leftVal = args.get(0).eval(bindings);
final ExprEval rightVal = args.get(1).eval(bindings);
if (leftVal.value() == null || rightVal.value() == null) {
return ExprEval.ofLongBoolean(leftVal.value() == null && rightVal.value() == null);
}
// Code copied and adapted from BinaryBooleanOpExprBase and BinEqExpr.
// The code isn't shared due to differences in code structure: BinaryBooleanOpExprBase + BinEqExpr have logic
// interleaved between parent and child class, but we can't use BinaryBooleanOpExprBase as a parent here, because
// (a) this is a function, not an expr; and (b) our logic for handling and returning nulls is different from most
// binary exprs, where null in means null out.
final ExpressionType comparisonType = ExpressionTypeConversion.autoDetect(leftVal, rightVal);
switch (comparisonType.getType()) {
case STRING:
return ExprEval.ofLongBoolean(Objects.equals(leftVal.asString(), rightVal.asString()));
case LONG:
return ExprEval.ofLongBoolean(leftVal.asLong() == rightVal.asLong());
case ARRAY:
final ExpressionType type = Preconditions.checkNotNull(
ExpressionTypeConversion.leastRestrictiveType(leftVal.type(), rightVal.type()),
"Cannot be null because ExprEval type is not nullable"
);
return ExprEval.ofLongBoolean(
type.getNullableStrategy().compare(leftVal.castTo(type).asArray(), rightVal.castTo(type).asArray()) == 0
);
case DOUBLE:
default:
if (leftVal.isNumericNull() || rightVal.isNumericNull()) {
return ExprEval.ofLongBoolean(leftVal.isNumericNull() && rightVal.isNumericNull());
} else {
return ExprEval.ofLongBoolean(leftVal.asDouble() == rightVal.asDouble());
}
}
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 2);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
}
/**
* SQL function "x IS DISTINCT FROM y". Very similar to "x <> y", i.e. {@link BinNeqExpr}, except this function
* never returns null.
*
* Implemented as a subclass of IsNotDistinctFromFunc to keep the code simple, and because we expect "notdistinctfrom"
* to be more common than "isdistinctfrom" in actual usage.
*/
class IsDistinctFromFunc extends IsNotDistinctFromFunc
{
@Override
public String name()
{
return "isdistinctfrom";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
return ExprEval.ofLongBoolean(!super.apply(args, bindings).asBoolean());
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 2);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
}
/**
* SQL function "IS NOT FALSE". Different from "IS TRUE" in that it returns true for NULL as well.
*/
class IsNotFalseFunc implements Function
{
@Override
public String name()
{
return "notfalse";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval arg = args.get(0).eval(bindings);
return ExprEval.ofLongBoolean(arg.value() == null || arg.asBoolean());
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 1);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
}
/**
* SQL function "IS NOT TRUE". Different from "IS FALSE" in that it returns true for NULL as well.
*/
class IsNotTrueFunc implements Function
{
@Override
public String name()
{
return "nottrue";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval arg = args.get(0).eval(bindings);
return ExprEval.ofLongBoolean(arg.value() == null || !arg.asBoolean());
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 1);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
}
/**
* SQL function "IS FALSE".
*/
class IsFalseFunc implements Function
{
@Override
public String name()
{
return "isfalse";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval arg = args.get(0).eval(bindings);
return ExprEval.ofLongBoolean(arg.value() != null && !arg.asBoolean());
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 1);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args)
{
return ExpressionType.LONG;
}
}
/**
* SQL function "IS TRUE".
*/
class IsTrueFunc implements Function
{
@Override
public String name()
{
return "istrue";
}
@Override
public ExprEval apply(List args, Expr.ObjectBinding bindings)
{
final ExprEval arg = args.get(0).eval(bindings);
return ExprEval.ofLongBoolean(arg.asBoolean());
}
@Override
public void validateArguments(List args)
{
validationHelperCheckArgumentCount(args, 1);
}
@Nullable
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List