Rework ExprMacro base classes to simplify implementations. (#15622)

* Rework ExprMacro base classes to simplify implementations.

This patch removes BaseScalarUnivariateMacroFunctionExpr, adds
BaseMacroFunctionExpr at the top of the hierarchy (a suitable base class
for ExprMacros that take either arrays or scalars), and adds an
implementation for "visit" to BaseMacroFunctionExpr.

The effect on implementations is generally cleaner code:

- Exprs no longer need to implement "visit".
- Exprs no longer need to implement "stringify", even if they don't
  use all of their args at runtime, because BaseMacroFunctionExpr has
  access to even unused args.
- Exprs that accept arrays can extend BaseMacroFunctionExpr and
  inherit a bunch of useful methods. The only one they need to
  implement themselves that scalar exprs don't is "supplyAnalyzeInputs".

* Make StringDecodeBase64UTFExpression a static class.

* Remove unused import.

* Formatting, annotation changes.
This commit is contained in:
Gian Merlino 2024-02-12 15:50:45 -08:00 committed by GitHub
parent 0f29ece6a9
commit 0f6a895372
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
34 changed files with 268 additions and 918 deletions

View File

@ -27,7 +27,6 @@ import org.apache.druid.query.aggregation.datasketches.hll.HllSketchHolder;
import javax.annotation.Nullable;
import java.util.List;
import java.util.stream.Collectors;
public class HllPostAggExprMacros
{
@ -40,7 +39,7 @@ public class HllPostAggExprMacros
public Expr apply(List<Expr> args)
{
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
return new HllSketchEstimateExpr(args);
return new HllSketchEstimateExpr(this, args);
}
@Override
@ -55,9 +54,9 @@ public class HllPostAggExprMacros
private Expr estimateExpr;
private Expr isRound;
public HllSketchEstimateExpr(List<Expr> args)
public HllSketchEstimateExpr(HLLSketchEstimateExprMacro macro, List<Expr> args)
{
super(HLL_SKETCH_ESTIMATE, args);
super(macro, args);
this.estimateExpr = args.get(0);
if (args.size() == 2) {
isRound = args.get(1);
@ -88,13 +87,6 @@ public class HllPostAggExprMacros
double estimate = h.getEstimate();
return round ? ExprEval.of(Math.round(estimate)) : ExprEval.of(estimate);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new HllSketchEstimateExpr(newArgs));
}
}
}

View File

@ -19,6 +19,7 @@
package org.apache.druid.query.aggregation.datasketches.theta.sql;
import com.google.common.collect.Iterables;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -39,7 +40,7 @@ public class ThetaPostAggMacros
public Expr apply(List<Expr> args)
{
validationHelperCheckArgumentCount(args, 1);
return new ThetaSketchEstimateExpr(args.get(0));
return new ThetaSketchEstimateExpr(this, args);
}
@Override
@ -49,14 +50,14 @@ public class ThetaPostAggMacros
}
}
public static class ThetaSketchEstimateExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
public static class ThetaSketchEstimateExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private Expr estimateExpr;
public ThetaSketchEstimateExpr(Expr arg)
public ThetaSketchEstimateExpr(ThetaSketchEstimateExprMacro macro, List<Expr> args)
{
super(THETA_SKETCH_ESTIMATE, arg);
this.estimateExpr = arg;
super(macro, args);
this.estimateExpr = Iterables.getOnlyElement(args);
}
@Override
@ -76,12 +77,6 @@ public class ThetaPostAggMacros
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new ThetaSketchEstimateExpr(arg));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)

View File

@ -62,14 +62,14 @@ public class BloomFilterExpressions
throw validationFailed("argument must be a LONG constant");
}
class BloomExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class BloomExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
final int expectedSize;
public BloomExpr(Expr arg)
public BloomExpr(List<Expr> args)
{
super(FN_NAME, arg);
this.expectedSize = arg.eval(InputBindings.nilBindings()).asInt();
super(CreateExprMacro.this, args);
this.expectedSize = args.get(0).eval(InputBindings.nilBindings()).asInt();
}
@Override
@ -81,12 +81,6 @@ public class BloomFilterExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -95,7 +89,7 @@ public class BloomFilterExpressions
}
}
return new BloomExpr(expectedSizeArg);
return new BloomExpr(args);
}
}
@ -118,7 +112,7 @@ public class BloomFilterExpressions
{
private BloomExpr(List<Expr> args)
{
super(FN_NAME, args);
super(AddExprMacro.this, args);
}
@Override
@ -162,13 +156,6 @@ public class BloomFilterExpressions
return ExprEval.ofComplex(BLOOM_FILTER_TYPE, filter);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -196,13 +183,16 @@ public class BloomFilterExpressions
{
validationHelperCheckArgumentCount(args, 2);
class BloomExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
final Expr arg = args.get(0);
final Expr filterExpr = args.get(1);
class BloomExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private final BloomKFilter filter;
private BloomExpr(BloomKFilter filter, Expr arg)
private BloomExpr(BloomKFilter filter, List<Expr> args)
{
super(FN_NAME, arg);
super(TestExprMacro.this, args);
this.filter = filter;
}
@ -248,12 +238,6 @@ public class BloomFilterExpressions
return filter.testBytes(null, 0, 0);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -266,7 +250,7 @@ public class BloomFilterExpressions
{
public DynamicBloomExpr(List<Expr> args)
{
super(FN_NAME, args);
super(TestExprMacro.this, args);
}
@Override
@ -319,12 +303,6 @@ public class BloomFilterExpressions
return filter.testBytes(null, 0, 0);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -333,9 +311,6 @@ public class BloomFilterExpressions
}
}
final Expr arg = args.get(0);
final Expr filterExpr = args.get(1);
if (filterExpr.isLiteral() && filterExpr.getLiteralValue() instanceof String) {
final String serializedFilter = (String) filterExpr.getLiteralValue();
final byte[] decoded = StringUtils.decodeBase64String(serializedFilter);
@ -346,7 +321,7 @@ public class BloomFilterExpressions
catch (IOException ioe) {
throw processingFailed(ioe, "failed to deserialize bloom filter");
}
return new BloomExpr(filter, arg);
return new BloomExpr(filter, args);
} else {
return new DynamicBloomExpr(args);
}

View File

@ -21,7 +21,7 @@ package org.apache.druid.query.expressions;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprMacroTable.ExprMacro;
import org.apache.druid.math.expr.ExpressionType;
@ -52,11 +52,11 @@ public class SleepExprMacro implements ExprMacro
Expr arg = args.get(0);
class SleepExpr extends BaseScalarUnivariateMacroFunctionExpr
class SleepExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
public SleepExpr(Expr arg)
public SleepExpr(List<Expr> args)
{
super(NAME, arg);
super(SleepExprMacro.this, args);
}
@Override
@ -78,12 +78,6 @@ public class SleepExprMacro implements ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
/**
* Explicitly override this method to not vectorize the sleep expression.
* If we ever want to vectorize this expression, {@link #getOutputType} should be considered to return something
@ -101,6 +95,6 @@ public class SleepExprMacro implements ExprMacro
return null;
}
}
return new SleepExpr(arg);
return new SleepExpr(args);
}
}

View File

@ -21,7 +21,7 @@ package org.apache.druid.testing.tools;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprMacroTable.ExprMacro;
import org.apache.druid.math.expr.ExpressionType;
@ -52,11 +52,11 @@ public class SleepExprMacro implements ExprMacro
Expr arg = args.get(0);
class SleepExpr extends BaseScalarUnivariateMacroFunctionExpr
class SleepExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
public SleepExpr(Expr arg)
public SleepExpr(List<Expr> args)
{
super(NAME, arg);
super(SleepExprMacro.this, args);
}
@Override
@ -78,12 +78,6 @@ public class SleepExprMacro implements ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
/**
* Explicitly override this method to not vectorize the sleep expression.
* If we ever want to vectorize this expression, {@link #getOutputType} should be considered to return something
@ -101,6 +95,6 @@ public class SleepExprMacro implements ExprMacro
return null;
}
}
return new SleepExpr(arg);
return new SleepExpr(args);
}
}

View File

@ -19,13 +19,12 @@
package org.apache.druid.math.expr;
import com.google.common.collect.Iterables;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.column.TypeStrategy;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
public class BuiltInExprMacros
{
@ -58,7 +57,7 @@ public class BuiltInExprMacros
public ComplexDecodeBase64Expression(List<Expr> args)
{
super(name(), args);
super(ComplexDecodeBase64ExprMacro.this, args);
validationHelperCheckArgumentCount(args, 2);
final Expr arg0 = args.get(0);
@ -117,13 +116,6 @@ public class BuiltInExprMacros
return ExprEval.ofComplex(complexType, typeStrategy.fromBytes(base64));
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new ComplexDecodeBase64Expression(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -160,7 +152,7 @@ public class BuiltInExprMacros
public Expr apply(List<Expr> args)
{
validationHelperCheckArgumentCount(args, 1);
return new StringDecodeBase64UTFExpression(args.get(0));
return new StringDecodeBase64UTFExpression(this, args);
}
/**
@ -174,11 +166,14 @@ public class BuiltInExprMacros
return NAME;
}
final class StringDecodeBase64UTFExpression extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
static final class StringDecodeBase64UTFExpression extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
public StringDecodeBase64UTFExpression(Expr arg)
private final Expr arg;
public StringDecodeBase64UTFExpression(StringDecodeBase64UTFExprMacro macro, List<Expr> args)
{
super(name(), arg);
super(macro, args);
this.arg = Iterables.getOnlyElement(args);
}
@Override
@ -191,12 +186,6 @@ public class BuiltInExprMacros
return new StringExpr(StringUtils.fromUtf8(StringUtils.decodeBase64String(toDecode.asString()))).eval(bindings);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(Collections.singletonList(arg))));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)

View File

@ -684,7 +684,7 @@ public interface Expr extends Cacheable
* Add set of arguments as {@link BindingAnalysis#arrayVariables} that are *directly* {@link IdentifierExpr},
* else they are ignored.
*/
BindingAnalysis withArrayArguments(Set<Expr> arrayArguments)
public BindingAnalysis withArrayArguments(Set<Expr> arrayArguments)
{
Set<IdentifierExpr> arrayIdentifiers = new HashSet<>();
for (Expr expr : arrayArguments) {
@ -705,7 +705,7 @@ public interface Expr extends Cacheable
/**
* Copy, setting if an expression has array inputs
*/
BindingAnalysis withArrayInputs(boolean hasArrays)
public BindingAnalysis withArrayInputs(boolean hasArrays)
{
return new BindingAnalysis(
freeVariables,
@ -719,7 +719,7 @@ public interface Expr extends Cacheable
/**
* Copy, setting if an expression produces an array output
*/
BindingAnalysis withArrayOutput(boolean isOutputArray)
public BindingAnalysis withArrayOutput(boolean isOutputArray)
{
return new BindingAnalysis(
freeVariables,

View File

@ -24,7 +24,6 @@ import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
@ -32,7 +31,6 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
/**
@ -107,89 +105,26 @@ public class ExprMacroTable
}
/**
* Base class for single argument {@link ExprMacro} function {@link Expr}
* Base class for {@link Expr} from {@link ExprMacro}.
*/
public abstract static class BaseScalarUnivariateMacroFunctionExpr implements ExprMacroFunctionExpr
public abstract static class BaseMacroFunctionExpr implements ExprMacroFunctionExpr
{
protected final String name;
protected final Expr arg;
// Use Supplier to memoize values as ExpressionSelectors#makeExprEvalSelector() can make repeated calls for them
private final Supplier<BindingAnalysis> analyzeInputsSupplier;
public BaseScalarUnivariateMacroFunctionExpr(String name, Expr arg)
{
this.name = name;
this.arg = arg;
analyzeInputsSupplier = Suppliers.memoize(this::supplyAnalyzeInputs);
}
@Override
public List<Expr> getArgs()
{
return Collections.singletonList(arg);
}
@Override
public BindingAnalysis analyzeInputs()
{
return analyzeInputsSupplier.get();
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s)", name, arg.stringify());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BaseScalarUnivariateMacroFunctionExpr that = (BaseScalarUnivariateMacroFunctionExpr) o;
return Objects.equals(name, that.name) &&
Objects.equals(arg, that.arg);
}
@Override
public int hashCode()
{
return Objects.hash(name, arg);
}
private BindingAnalysis supplyAnalyzeInputs()
{
return arg.analyzeInputs().withScalarArguments(ImmutableSet.of(arg));
}
@Override
public String toString()
{
return StringUtils.format("(%s %s)", name, getArgs());
}
}
/**
* Base class for multi-argument {@link ExprMacro} function {@link Expr}
*/
public abstract static class BaseScalarMacroFunctionExpr implements ExprMacroFunctionExpr
{
protected final String name;
protected final ExprMacro macro;
protected final List<Expr> args;
// Use Supplier to memoize values as ExpressionSelectors#makeExprEvalSelector() can make repeated calls for them
private final Supplier<BindingAnalysis> analyzeInputsSupplier;
public BaseScalarMacroFunctionExpr(String name, final List<Expr> args)
/**
* Constructor for subclasses.
*
* @param macro macro that created this expr
* @param macroArgs original args to the macro (not the ones this will be evaled with)
*/
protected BaseMacroFunctionExpr(final ExprMacro macro, final List<Expr> macroArgs)
{
this.name = name;
this.args = args;
this.macro = macro;
this.args = macroArgs;
analyzeInputsSupplier = Suppliers.memoize(this::supplyAnalyzeInputs);
}
@ -204,17 +139,30 @@ public class ExprMacroTable
{
return StringUtils.format(
"%s(%s)",
name,
Expr.ARG_JOINER.join(args.stream().map(Expr::stringify).iterator())
macro.name(),
args.size() == 1
? args.get(0).stringify()
: Expr.ARG_JOINER.join(args.stream().map(Expr::stringify).iterator())
);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(macro.apply(shuttle.visitAll(args)));
}
@Override
public BindingAnalysis analyzeInputs()
{
return analyzeInputsSupplier.get();
}
/**
* Implemented by subclasses to provide the value for {@link #analyzeInputs()}, which uses a memoized supplier.
*/
protected abstract BindingAnalysis supplyAnalyzeInputs();
@Override
public boolean equals(Object o)
{
@ -225,31 +173,38 @@ public class ExprMacroTable
return false;
}
BaseScalarMacroFunctionExpr that = (BaseScalarMacroFunctionExpr) o;
return Objects.equals(name, that.name) &&
return Objects.equals(macro, that.macro) &&
Objects.equals(args, that.args);
}
@Override
public int hashCode()
{
return Objects.hash(name, args);
}
private BindingAnalysis supplyAnalyzeInputs()
{
final Set<Expr> argSet = Sets.newHashSetWithExpectedSize(args.size());
BindingAnalysis accumulator = new BindingAnalysis();
for (Expr arg : args) {
accumulator = accumulator.with(arg);
argSet.add(arg);
}
return accumulator.withScalarArguments(argSet);
return Objects.hash(macro, args);
}
@Override
public String toString()
{
return StringUtils.format("(%s %s)", name, getArgs());
return StringUtils.format("(%s %s)", macro.name(), getArgs());
}
}
/**
* Base class for {@link Expr} from {@link ExprMacro} that accepts all-scalar arguments.
*/
public abstract static class BaseScalarMacroFunctionExpr extends BaseMacroFunctionExpr
{
public BaseScalarMacroFunctionExpr(final ExprMacro macro, final List<Expr> macroArgs)
{
super(macro, macroArgs);
}
@Override
protected BindingAnalysis supplyAnalyzeInputs()
{
return Exprs.analyzeBindings(args)
.withScalarArguments(ImmutableSet.copyOf(args));
}
}

View File

@ -53,6 +53,18 @@ public class Exprs
return new UOE("Unable to vectorize expression: %s", msg);
}
/**
* Return a {@link Expr.BindingAnalysis} that represents an analysis of all provided args.
*/
public static Expr.BindingAnalysis analyzeBindings(final List<Expr> args)
{
Expr.BindingAnalysis accumulator = new Expr.BindingAnalysis();
for (final Expr arg : args) {
accumulator = accumulator.with(arg);
}
return accumulator;
}
/**
* Decomposes any expr into a list of exprs that, if ANDed together, are equivalent to the input expr.
*

View File

@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
import it.unimi.dsi.fastutil.doubles.DoubleArrayList;
import it.unimi.dsi.fastutil.doubles.DoubleList;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -70,11 +69,11 @@ public class ArrayQuantileExprMacro implements ExprMacroTable.ExprMacro
final double rank = ((Number) rankArg.getLiteralValue()).doubleValue();
class ArrayQuantileExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class ArrayQuantileExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private ArrayQuantileExpr(Expr arg)
private ArrayQuantileExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(ArrayQuantileExprMacro.this, args);
}
@Nonnull
@ -92,27 +91,15 @@ public class ArrayQuantileExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.ofDouble(quantileFromSortedArray(doubles, rank));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.DOUBLE;
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), rankArg.stringify());
}
}
return new ArrayQuantileExpr(arg);
return new ArrayQuantileExpr(args);
}
/**

View File

@ -35,7 +35,7 @@ import java.util.List;
* - {@code contains_string("foobar", "car") - 0 }
* - {@code contains_string("foobar", "Bar") - 1 }
* <p>
* See {@link ContainsExprMacro} for the case-sensitive version.
* @see ContainsExprMacro for the case-sensitive version.
*/
public class CaseInsensitiveContainsExprMacro implements ExprMacroTable.ExprMacro
@ -55,6 +55,6 @@ public class CaseInsensitiveContainsExprMacro implements ExprMacroTable.ExprMacr
final Expr arg = args.get(0);
final Expr searchStr = args.get(1);
return new ContainsExpr(FN_NAME, arg, searchStr, false, shuttle -> apply(shuttle.visitAll(args)));
return new ContainsExpr(this, arg, searchStr, false);
}
}

View File

@ -19,6 +19,7 @@
package org.apache.druid.query.expression;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
@ -28,42 +29,38 @@ import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExpressionType;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.function.Function;
/**
* {@link Expr} class returned by {@link ContainsExprMacro} and {@link CaseInsensitiveContainsExprMacro} for
* evaluating the expression.
*/
class ContainsExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class ContainsExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private final Expr arg;
private final Function<String, Boolean> searchFunction;
private final Expr searchStrExpr;
private final Function<Shuttle, Expr> visitFunction;
ContainsExpr(
final String functionName,
final ExprMacroTable.ExprMacro macro,
final Expr arg,
final Expr searchStrExpr,
final boolean caseSensitive,
final Function<Shuttle, Expr> visitFunction
final boolean caseSensitive
)
{
this(functionName, arg, searchStrExpr, createFunction(searchStrExpr, caseSensitive), visitFunction);
this(macro, arg, searchStrExpr, createFunction(getSearchString(searchStrExpr, macro.name()), caseSensitive));
}
private ContainsExpr(
final String functionName,
final ExprMacroTable.ExprMacro macro,
final Expr arg,
final Expr searchStrExpr,
final Function<String, Boolean> searchFunction,
final Function<Shuttle, Expr> visitFunction
final Function<String, Boolean> searchFunction
)
{
super(functionName, arg);
super(macro, ImmutableList.of(arg, searchStrExpr));
this.arg = arg;
this.searchFunction = searchFunction;
this.searchStrExpr = validateSearchExpr(searchStrExpr, functionName);
this.visitFunction = visitFunction;
getSearchString(searchStrExpr, macro.name());
}
@Nonnull
@ -81,39 +78,26 @@ class ContainsExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.LONG;
}
@Override
public Expr visit(Expr.Shuttle shuttle)
{
return visitFunction.apply(shuttle);
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s, %s)", name, arg.stringify(), searchStrExpr.stringify());
}
private Expr validateSearchExpr(Expr searchExpr, String functioName)
private static String getSearchString(Expr searchExpr, String functioName)
{
if (!ExprUtils.isStringLiteral(searchExpr)) {
throw new IAE("Function[%s] substring must be a string literal", functioName);
}
return searchExpr;
return StringUtils.nullToEmptyNonDruidDataString((String) searchExpr.getLiteralValue());
}
private static Function<String, Boolean> createFunction(Expr searchStrExpr, boolean caseSensitive)
private static Function<String, Boolean> createFunction(String searchString, boolean caseSensitive)
{
String searchStr = StringUtils.nullToEmptyNonDruidDataString((String) searchStrExpr.getLiteralValue());
if (caseSensitive) {
return s -> s.contains(searchStr);
return s -> s.contains(searchString);
} else {
return s -> org.apache.commons.lang.StringUtils.containsIgnoreCase(s, searchString);
}
return s -> org.apache.commons.lang.StringUtils.containsIgnoreCase(s, searchStr);
}
}

View File

@ -35,7 +35,7 @@ import java.util.List;
* - {@code contains_string("foobar", "car") - 0 }
* - {@code contains_string("foobar", "Bar") - 0 }
* <p>
* See {@link CaseInsensitiveContainsExprMacro} for the case-insensitive version.
* @see CaseInsensitiveContainsExprMacro for the case-insensitive version.
*/
public class ContainsExprMacro implements ExprMacroTable.ExprMacro
{
@ -54,6 +54,6 @@ public class ContainsExprMacro implements ExprMacroTable.ExprMacro
final Expr arg = args.get(0);
final Expr searchStr = args.get(1);
return new ContainsExpr(FN_NAME, arg, searchStr, true, shuttle -> shuttle.visit(apply(shuttle.visitAll(args))));
return new ContainsExpr(this, arg, searchStr, true);
}
}

View File

@ -141,7 +141,7 @@ public class HyperUniqueExpressions
{
public HllExpr(List<Expr> args)
{
super(NAME, args);
super(HllAddExprMacro.this, args);
}
@Override
@ -177,7 +177,8 @@ public class HyperUniqueExpressions
break;
case DOUBLE:
if (NullHandling.replaceWithDefault() || !input.isNumericNull()) {
collector.add(CardinalityAggregator.HASH_FUNCTION.hashLong(Double.doubleToLongBits(input.asDouble())).asBytes());
collector.add(CardinalityAggregator.HASH_FUNCTION.hashLong(Double.doubleToLongBits(input.asDouble()))
.asBytes());
}
break;
case LONG:
@ -186,7 +187,8 @@ public class HyperUniqueExpressions
}
break;
case COMPLEX:
if (TYPE.equals(input.type()) || hllType.is(ExprType.COMPLEX) && hllCollector.value() instanceof HyperLogLogCollector) {
if (TYPE.equals(input.type())
|| hllType.is(ExprType.COMPLEX) && hllCollector.value() instanceof HyperLogLogCollector) {
collector.fold((HyperLogLogCollector) input.value());
break;
}
@ -200,12 +202,6 @@ public class HyperUniqueExpressions
return ExprEval.ofComplex(TYPE, collector);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -232,11 +228,11 @@ public class HyperUniqueExpressions
{
validationHelperCheckArgumentCount(args, 1);
class HllExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class HllExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
public HllExpr(Expr arg)
public HllExpr(List<Expr> args)
{
super(NAME, arg);
super(HllEstimateExprMacro.this, args);
}
@Override
@ -258,12 +254,6 @@ public class HyperUniqueExpressions
return ExprEval.ofDouble(collector.estimateCardinality());
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -271,7 +261,7 @@ public class HyperUniqueExpressions
return ExpressionType.DOUBLE;
}
}
return new HllExpr(args.get(0));
return new HllExpr(args);
}
}
@ -290,11 +280,11 @@ public class HyperUniqueExpressions
{
validationHelperCheckArgumentCount(args, 1);
class HllExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class HllExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
public HllExpr(Expr arg)
public HllExpr(List<Expr> args)
{
super(NAME, arg);
super(HllRoundEstimateExprMacro.this, args);
}
@Override
@ -312,12 +302,6 @@ public class HyperUniqueExpressions
return ExprEval.ofLong(collector.estimateCardinalityRound());
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -325,7 +309,7 @@ public class HyperUniqueExpressions
return ExpressionType.LONG;
}
}
return new HllExpr(args.get(0));
return new HllExpr(args);
}
}
}

View File

@ -23,7 +23,6 @@ import inet.ipaddr.AddressStringException;
import inet.ipaddr.IPAddress;
import inet.ipaddr.IPAddressString;
import inet.ipaddr.ipv4.IPv4Address;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -77,11 +76,11 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
final IPAddressString blockString = getSubnetInfo(args);
final IPAddress block = blockString.toAddress().toPrefixBlock();
class IPv4AddressMatchExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class IPv4AddressMatchExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private IPv4AddressMatchExpr(Expr arg)
private IPv4AddressMatchExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(IPv4AddressMatchExprMacro.this, args);
}
@Nonnull
@ -115,27 +114,15 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
return address != null && block.contains(address);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.LONG;
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), args.get(ARG_SUBNET).stringify());
}
}
return new IPv4AddressMatchExpr(arg);
return new IPv4AddressMatchExpr(args);
}
catch (AddressStringException e) {

View File

@ -63,11 +63,11 @@ public class IPv4AddressParseExprMacro implements ExprMacroTable.ExprMacro
Expr arg = args.get(0);
class IPv4AddressParseExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class IPv4AddressParseExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private IPv4AddressParseExpr(Expr arg)
private IPv4AddressParseExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(IPv4AddressParseExprMacro.this, args);
}
@Nonnull
@ -85,12 +85,6 @@ public class IPv4AddressParseExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -99,7 +93,7 @@ public class IPv4AddressParseExprMacro implements ExprMacroTable.ExprMacro
}
}
return new IPv4AddressParseExpr(arg);
return new IPv4AddressParseExpr(args);
}
private static ExprEval evalAsString(ExprEval eval)

View File

@ -62,11 +62,11 @@ public class IPv4AddressStringifyExprMacro implements ExprMacroTable.ExprMacro
Expr arg = args.get(0);
class IPv4AddressStringifyExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class IPv4AddressStringifyExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private IPv4AddressStringifyExpr(Expr arg)
private IPv4AddressStringifyExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(IPv4AddressStringifyExprMacro.this, args);
}
@Nonnull
@ -84,12 +84,6 @@ public class IPv4AddressStringifyExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -98,7 +92,7 @@ public class IPv4AddressStringifyExprMacro implements ExprMacroTable.ExprMacro
}
}
return new IPv4AddressStringifyExpr(arg);
return new IPv4AddressStringifyExpr(args);
}
private static ExprEval evalAsString(ExprEval eval)

View File

@ -20,60 +20,58 @@
package org.apache.druid.query.expression;
import inet.ipaddr.IPAddressString;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExpressionType;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.List;
/**
* <pre>
* Implements an expression that checks if an IPv6 address belongs to a subnet.
*
* Expression signatures:
* - long ipv6_match(string address, string subnet)
*
* Valid "address" argument formats are:
* - IPv6 address string (e.g., "2001:4860:4860::8888")
*
* The argument format for the "subnet" argument should be a literal in CIDR notation
* (e.g., "2001:db8::/64 ").
*
* If the "address" argument does not represent an IPv6 address then false is returned.
* </pre>
*
*/
* <pre>
* Implements an expression that checks if an IPv6 address belongs to a subnet.
*
* Expression signatures:
* - long ipv6_match(string address, string subnet)
*
* Valid "address" argument formats are:
* - IPv6 address string (e.g., "2001:4860:4860::8888")
*
* The argument format for the "subnet" argument should be a literal in CIDR notation
* (e.g., "2001:db8::/64 ").
*
* If the "address" argument does not represent an IPv6 address then false is returned.
* </pre>
*/
public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
{
public static final String FN_NAME = "ipv6_match";
private static final int ARG_SUBNET = 1;
@Override
public String name()
{
return FN_NAME;
}
@Override
public Expr apply(final List<Expr> args)
{
validationHelperCheckArgumentCount(args, 2);
try {
final Expr arg = args.get(0);
final IPAddressString blockString = getSubnetInfo(args);
class IPv6AddressMatchExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class IPv6AddressMatchExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private IPv6AddressMatchExpr(Expr arg)
private IPv6AddressMatchExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(IPv6AddressMatchExprMacro.this, args);
}
@Nonnull
@Override
public ExprEval eval(final ObjectBinding bindings)
@ -89,24 +87,12 @@ public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
}
return ExprEval.ofLongBoolean(match);
}
private boolean isStringMatch(String stringValue)
{
IPAddressString addressString = IPv6AddressExprUtils.parseString(stringValue);
return addressString != null && blockString.prefixContains(addressString);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), args.get(ARG_SUBNET).stringify());
}
@Nullable
@Override
@ -115,14 +101,14 @@ public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
return ExpressionType.LONG;
}
}
return new IPv6AddressMatchExpr(arg);
return new IPv6AddressMatchExpr(args);
}
catch (Exception e) {
throw processingFailed(e, "failed to parse address");
}
}
private IPAddressString getSubnetInfo(List<Expr> args)
{
String subnetArgName = "subnet";

View File

@ -20,7 +20,6 @@
package org.apache.druid.query.expression;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -70,11 +69,11 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
escapeChar
);
class LikeExtractExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class LikeExtractExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private LikeExtractExpr(Expr arg)
private LikeExtractExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(LikeExprMacro.this, args);
}
@Nonnull
@ -88,35 +87,13 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.ofLongBoolean(match.matches(false));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.LONG;
}
@Override
public String stringify()
{
if (escapeExpr != null) {
return StringUtils.format(
"%s(%s, %s, %s)",
FN_NAME,
arg.stringify(),
patternExpr.stringify(),
escapeExpr.stringify()
);
}
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), patternExpr.stringify());
}
}
return new LikeExtractExpr(arg);
return new LikeExtractExpr(args);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
import com.google.inject.Inject;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
@ -74,15 +73,15 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
replaceMissingValueWith != null && replaceMissingValueWith.isLiteral()
? Evals.asString(replaceMissingValueWith.getLiteralValue())
: null,
false,
null,
null
);
class LookupExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class LookupExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private LookupExpr(Expr arg)
private LookupExpr(final List<Expr> args)
{
super(FN_NAME, arg);
super(LookupExprMacro.this, args);
}
@Nonnull
@ -92,12 +91,6 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(extractionFn.apply(NullHandling.emptyToNullIfNeeded(arg.eval(bindings).asString())));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -105,21 +98,6 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
return ExpressionType.STRING;
}
@Override
public String stringify()
{
if (replaceMissingValueWith != null) {
return StringUtils.format(
"%s(%s, %s, %s)",
FN_NAME,
arg.stringify(),
lookupExpr.stringify(),
replaceMissingValueWith.stringify()
);
}
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), lookupExpr.stringify());
}
@Override
public void decorateCacheKeyBuilder(CacheKeyBuilder builder)
{
@ -127,7 +105,7 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
}
}
return new LookupExpr(arg);
return new LookupExpr(args);
}
private Expr getReplaceMissingValueWith(final List<Expr> args)

View File

@ -68,7 +68,7 @@ public class NestedDataExpressions
{
public StructExpr(List<Expr> args)
{
super(NAME, args);
super(JsonObjectExprMacro.this, args);
}
@Override
@ -88,13 +88,6 @@ public class NestedDataExpressions
return ExprEval.ofComplex(ExpressionType.NESTED_DATA, theMap);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new StructExpr(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -133,7 +126,7 @@ public class NestedDataExpressions
{
public ToJsonStringExpr(List<Expr> args)
{
super(name(), args);
super(ToJsonStringExprMacro.this, args);
}
@Override
@ -157,13 +150,6 @@ public class NestedDataExpressions
}
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new ToJsonStringExpr(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -202,7 +188,7 @@ public class NestedDataExpressions
{
public ParseJsonExpr(List<Expr> args)
{
super(name(), args);
super(ParseJsonExprMacro.this, args);
}
@Override
@ -230,13 +216,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new ParseJsonExpr(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -275,7 +254,7 @@ public class NestedDataExpressions
{
public ParseJsonExpr(List<Expr> args)
{
super(name(), args);
super(TryParseJsonExprMacro.this, args);
}
@Override
@ -302,13 +281,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new ParseJsonExpr(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -350,7 +322,7 @@ public class NestedDataExpressions
public JsonValueExpr(List<Expr> args)
{
super(name(), args);
super(JsonValueExprMacro.this, args);
this.parts = getJsonPathPartsFromLiteral(JsonValueExprMacro.this, args.get(1));
}
@ -367,17 +339,6 @@ public class NestedDataExpressions
return ExprEval.of(null);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
return shuttle.visit(new JsonValueExpr(newArgs));
} else {
return shuttle.visit(new JsonValueDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -394,7 +355,7 @@ public class NestedDataExpressions
public JsonValueCastExpr(List<Expr> args)
{
super(name(), args);
super(JsonValueExprMacro.this, args);
this.parts = getJsonPathPartsFromLiteral(JsonValueExprMacro.this, args.get(1));
this.castTo = ExpressionType.fromString((String) args.get(2).getLiteralValue());
if (castTo == null) {
@ -418,17 +379,6 @@ public class NestedDataExpressions
return ExprEval.ofType(castTo, null);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
return shuttle.visit(new JsonValueCastExpr(newArgs));
} else {
return shuttle.visit(new JsonValueDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -441,7 +391,7 @@ public class NestedDataExpressions
{
public JsonValueDynamicExpr(List<Expr> args)
{
super(name(), args);
super(JsonValueExprMacro.this, args);
}
@Override
@ -469,21 +419,6 @@ public class NestedDataExpressions
return castTo == null ? ExprEval.of(null) : ExprEval.ofType(castTo, null);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
if (newArgs.size() == 3 && newArgs.get(2).isLiteral()) {
return shuttle.visit(new JsonValueCastExpr(newArgs));
} else {
return shuttle.visit(new JsonValueExpr(newArgs));
}
} else {
return shuttle.visit(new JsonValueDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -520,7 +455,7 @@ public class NestedDataExpressions
public JsonQueryExpr(List<Expr> args)
{
super(name(), args);
super(JsonQueryExprMacro.this, args);
this.parts = getJsonPathPartsFromLiteral(JsonQueryExprMacro.this, args.get(1));
}
@ -534,17 +469,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
return shuttle.visit(new JsonQueryExpr(newArgs));
} else {
return shuttle.visit(new JsonQueryDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -558,7 +482,7 @@ public class NestedDataExpressions
{
public JsonQueryDynamicExpr(List<Expr> args)
{
super(name(), args);
super(JsonQueryExprMacro.this, args);
}
@Override
@ -573,17 +497,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
return shuttle.visit(new JsonQueryExpr(newArgs));
} else {
return shuttle.visit(new JsonQueryDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -620,7 +533,7 @@ public class NestedDataExpressions
public JsonQueryArrayExpr(List<Expr> args)
{
super(name(), args);
super(JsonQueryArrayExprMacro.this, args);
this.parts = getJsonPathPartsFromLiteral(JsonQueryArrayExprMacro.this, args.get(1));
}
@ -641,17 +554,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
return shuttle.visit(new JsonQueryArrayExpr(newArgs));
} else {
return shuttle.visit(new JsonQueryArrayDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -665,7 +567,7 @@ public class NestedDataExpressions
{
public JsonQueryArrayDynamicExpr(List<Expr> args)
{
super(name(), args);
super(JsonQueryArrayExprMacro.this, args);
}
@Override
@ -687,17 +589,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
if (newArgs.get(1).isLiteral()) {
return shuttle.visit(new JsonQueryArrayExpr(newArgs));
} else {
return shuttle.visit(new JsonQueryArrayDynamicExpr(newArgs));
}
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -750,7 +641,7 @@ public class NestedDataExpressions
{
public JsonPathsExpr(List<Expr> args)
{
super(name(), args);
super(JsonPathsExprMacro.this, args);
}
@Override
@ -769,13 +660,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new JsonPathsExpr(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -805,7 +689,7 @@ public class NestedDataExpressions
{
public JsonKeysExpr(List<Expr> args)
{
super(name(), args);
super(JsonKeysExprMacro.this, args);
}
@Override
@ -818,15 +702,6 @@ public class NestedDataExpressions
);
}
@Override
public Expr visit(Shuttle shuttle)
{
List<Expr> newArgs = args.stream().map(x -> x.visit(shuttle)).collect(Collectors.toList());
return shuttle.visit(new JsonKeysExpr(newArgs));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{

View File

@ -66,11 +66,11 @@ public class RegexpExtractExprMacro implements ExprMacroTable.ExprMacro
final int index = indexExpr == null ? 0 : ((Number) indexExpr.getLiteralValue()).intValue();
class RegexpExtractExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class RegexpExtractExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private RegexpExtractExpr(Expr arg)
private RegexpExtractExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(RegexpExtractExprMacro.this, args);
}
@Nonnull
@ -89,34 +89,13 @@ public class RegexpExtractExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.STRING;
}
@Override
public String stringify()
{
if (indexExpr != null) {
return StringUtils.format(
"%s(%s, %s, %s)",
FN_NAME,
arg.stringify(),
patternExpr.stringify(),
indexExpr.stringify()
);
}
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), patternExpr.stringify());
}
}
return new RegexpExtractExpr(arg);
return new RegexpExtractExpr(args);
}
}

View File

@ -59,11 +59,11 @@ public class RegexpLikeExprMacro implements ExprMacroTable.ExprMacro
StringUtils.nullToEmptyNonDruidDataString((String) patternExpr.getLiteralValue())
);
class RegexpLikeExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class RegexpLikeExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private RegexpLikeExpr(Expr arg)
private RegexpLikeExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(RegexpLikeExprMacro.this, args);
}
@Nonnull
@ -81,25 +81,14 @@ public class RegexpLikeExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.LONG;
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), patternExpr.stringify());
}
}
return new RegexpLikeExpr(arg);
return new RegexpLikeExpr(args);
}
}

View File

@ -57,7 +57,7 @@ public class RegexpReplaceExprMacro implements ExprMacroTable.ExprMacro
{
public BaseRegexpReplaceExpr(final List<Expr> args)
{
super(FN_NAME, args);
super(RegexpReplaceExprMacro.this, args);
}
@Nullable
@ -66,12 +66,6 @@ public class RegexpReplaceExprMacro implements ExprMacroTable.ExprMacro
{
return ExpressionType.STRING;
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
}
/**

View File

@ -49,9 +49,9 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
validationHelperCheckArgumentRange(args, 2, 4);
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
return new TimestampCeilExpr(args);
return new TimestampCeilExpr(this, args);
} else {
return new TimestampCeilDynamicExpr(args);
return new TimestampCeilDynamicExpr(this, args);
}
}
@ -60,9 +60,9 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
{
private final Granularity granularity;
TimestampCeilExpr(final List<Expr> args)
TimestampCeilExpr(final TimestampCeilExprMacro macro, final List<Expr> args)
{
super(FN_NAME, args);
super(macro, args);
this.granularity = getGranularity(args, InputBindings.nilBindings());
}
@ -83,12 +83,6 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(granularity.increment(bucketStartTime));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new TimestampCeilExpr(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -132,9 +126,9 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
@VisibleForTesting
static class TimestampCeilDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
TimestampCeilDynamicExpr(final List<Expr> args)
TimestampCeilDynamicExpr(final TimestampCeilExprMacro macro, final List<Expr> args)
{
super(FN_NAME, args);
super(macro, args);
}
@Nonnull
@ -150,12 +144,6 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(granularity.increment(bucketStartTime));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new TimestampCeilDynamicExpr(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)

View File

@ -124,20 +124,6 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
}
}
private static String stringifyExpr(final List<Expr> args)
{
if (args.size() > 2) {
return StringUtils.format(
"%s(%s, %s, %s)",
FN_NAME,
args.get(0).stringify(),
args.get(1).stringify(),
args.get(2).stringify()
);
}
return StringUtils.format("%s(%s, %s)", FN_NAME, args.get(0).stringify(), args.get(1).stringify());
}
private static ISOChronology computeChronology(final List<Expr> args, final Expr.ObjectBinding bindings)
{
String timeZoneVal = (String) args.get(2).eval(bindings).value();
@ -176,7 +162,7 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
private TimestampExtractExpr(final List<Expr> args, final Unit unit, final ISOChronology chronology)
{
super(FN_NAME, args);
super(TimestampExtractExprMacro.this, args);
this.unit = unit;
this.chronology = chronology;
}
@ -194,24 +180,12 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
return getExprEval(dateTime, unit);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return getOutputExpressionType(unit);
}
@Override
public String stringify()
{
return stringifyExpr(args);
}
}
public class TimestampExtractDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
@ -220,7 +194,7 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
private TimestampExtractDynamicExpr(final List<Expr> args, final Unit unit)
{
super(FN_NAME, args);
super(TimestampExtractExprMacro.this, args);
this.unit = unit;
}
@ -238,23 +212,11 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
return getExprEval(dateTime, unit);
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return getOutputExpressionType(unit);
}
@Override
public String stringify()
{
return stringifyExpr(args);
}
}
}

View File

@ -50,9 +50,9 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
validationHelperCheckArgumentRange(args, 2, 4);
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
return new TimestampFloorExpr(args);
return new TimestampFloorExpr(this, args);
} else {
return new TimestampFloorDynamicExpr(args);
return new TimestampFloorDynamicExpr(this, args);
}
}
@ -70,9 +70,9 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
{
private final PeriodGranularity granularity;
TimestampFloorExpr(final List<Expr> args)
TimestampFloorExpr(final TimestampFloorExprMacro macro, final List<Expr> args)
{
super(FN_NAME, args);
super(macro, args);
this.granularity = computeGranularity(args, InputBindings.nilBindings());
}
@ -104,12 +104,6 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(granularity.bucketStart(eval.asLong()));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new TimestampFloorExpr(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
@ -167,9 +161,9 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
public static class TimestampFloorDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
TimestampFloorDynamicExpr(final List<Expr> args)
TimestampFloorDynamicExpr(final TimestampFloorExprMacro macro, final List<Expr> args)
{
super(FN_NAME, args);
super(macro, args);
}
@Nonnull
@ -180,12 +174,6 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(granularity.bucketStart(args.get(0).eval(bindings).asLong()));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new TimestampFloorDynamicExpr(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)

View File

@ -19,7 +19,6 @@
package org.apache.druid.query.expression;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -69,11 +68,11 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
? ISODateTimeFormat.dateTime().withZone(timeZone)
: DateTimeFormat.forPattern(formatString).withZone(timeZone);
class TimestampFormatExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class TimestampFormatExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private TimestampFormatExpr(Expr arg)
private TimestampFormatExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(TimestampFormatExprMacro.this, args);
}
@Nonnull
@ -88,38 +87,14 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(formatter.print(arg.eval(bindings).asLong()));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.STRING;
}
@Override
public String stringify()
{
if (args.size() > 2) {
return StringUtils.format(
"%s(%s, %s, %s)",
FN_NAME,
arg.stringify(),
args.get(1).stringify(),
args.get(2).stringify()
);
}
if (args.size() > 1) {
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), args.get(1).stringify());
}
return super.stringify();
}
}
return new TimestampFormatExpr(arg);
return new TimestampFormatExpr(args);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.query.expression;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -66,11 +65,11 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
? createDefaultParser(timeZone)
: DateTimes.wrapFormatter(DateTimeFormat.forPattern(formatString).withZone(timeZone));
class TimestampParseExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
class TimestampParseExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private TimestampParseExpr(Expr arg)
private TimestampParseExpr(List<Expr> args)
{
super(FN_NAME, arg);
super(TimestampParseExprMacro.this, args);
}
@Nonnull
@ -92,39 +91,15 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(apply(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.LONG;
}
@Override
public String stringify()
{
if (args.size() > 2) {
return StringUtils.format(
"%s(%s, %s, %s)",
FN_NAME,
arg.stringify(),
args.get(1).stringify(),
args.get(2).stringify()
);
}
if (args.size() > 1) {
return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), args.get(1).stringify());
}
return super.stringify();
}
}
return new TimestampParseExpr(arg);
return new TimestampParseExpr(args);
}
/**

View File

@ -52,11 +52,11 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
validationHelperCheckArgumentRange(args, 3, 4);
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
return new TimestampShiftExpr(args);
return new TimestampShiftExpr(this, args);
} else {
// Use dynamic impl if any args are non-literal. Don't bother optimizing for the case where period is
// literal but step isn't.
return new TimestampShiftDynamicExpr(args);
return new TimestampShiftDynamicExpr(this, args);
}
}
@ -87,9 +87,9 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
private final Period period;
private final int step;
TimestampShiftExpr(final List<Expr> args)
TimestampShiftExpr(final TimestampShiftExprMacro macro, final List<Expr> args)
{
super(FN_NAME, args);
super(macro, args);
period = getPeriod(args, InputBindings.nilBindings());
chronology = getTimeZone(args, InputBindings.nilBindings());
step = getStep(args, InputBindings.nilBindings());
@ -106,12 +106,6 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(chronology.add(period, timestamp.asLong(), step));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new TimestampShiftExpr(shuttle.visitAll(args)));
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
@ -147,9 +141,9 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
private static class TimestampShiftDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
TimestampShiftDynamicExpr(final List<Expr> args)
TimestampShiftDynamicExpr(final TimestampShiftExprMacro macro, final List<Expr> args)
{
super(FN_NAME, args);
super(macro, args);
}
@Nonnull
@ -166,12 +160,6 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
return ExprEval.of(chronology.add(period, timestamp.asLong(), step));
}
@Override
public Expr visit(Shuttle shuttle)
{
return shuttle.visit(new TimestampShiftDynamicExpr(shuttle.visitAll(args)));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)

View File

@ -20,8 +20,6 @@
package org.apache.druid.query.expression;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
@ -30,10 +28,7 @@ import org.apache.druid.math.expr.InputBindings;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.function.Function;
public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
{
@ -91,50 +86,44 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
{
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
final Function<Expr.Shuttle, Expr> visitFn = shuttle -> shuttle.visit(apply(shuttle.visitAll(args)));
if (args.size() == 1) {
return new TrimStaticCharsExpr(mode, args.get(0), DEFAULT_CHARS, null, visitFn);
return new TrimStaticCharsExpr(this, args, DEFAULT_CHARS);
} else {
final Expr charsArg = args.get(1);
if (charsArg.isLiteral()) {
final String charsString = charsArg.eval(InputBindings.nilBindings()).asString();
final char[] chars = charsString == null ? EMPTY_CHARS : charsString.toCharArray();
return new TrimStaticCharsExpr(mode, args.get(0), chars, charsArg, visitFn);
return new TrimStaticCharsExpr(this, args, chars);
} else {
return new TrimDynamicCharsExpr(mode, args.get(0), args.get(1), visitFn);
return new TrimDynamicCharsExpr(this, args);
}
}
}
@VisibleForTesting
static class TrimStaticCharsExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
static class TrimStaticCharsExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private final TrimMode mode;
private final char[] chars;
private final Expr charsExpr;
private final Function<Shuttle, Expr> visitFn;
private final Expr stringExpr;
public TrimStaticCharsExpr(
final TrimMode mode,
final Expr stringExpr,
final char[] chars,
final Expr charsExpr,
final Function<Shuttle, Expr> visitFn
final TrimExprMacro macro,
final List<Expr> args,
final char[] chars
)
{
super(mode.getFnName(), stringExpr);
this.mode = mode;
super(macro, args);
this.mode = macro.mode;
this.stringExpr = args.get(0);
this.chars = chars;
this.charsExpr = charsExpr;
this.visitFn = visitFn;
}
@Nonnull
@Override
public ExprEval eval(final ObjectBinding bindings)
{
final ExprEval stringEval = arg.eval(bindings);
final ExprEval stringEval = stringExpr.eval(bindings);
if (chars.length == 0 || stringEval.value() == null) {
return stringEval;
@ -172,76 +161,30 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public Expr visit(Shuttle shuttle)
{
return visitFn.apply(shuttle);
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.STRING;
}
@Override
public String stringify()
{
if (charsExpr != null) {
return StringUtils.format("%s(%s, %s)", mode.getFnName(), arg.stringify(), charsExpr.stringify());
}
return super.stringify();
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
TrimStaticCharsExpr that = (TrimStaticCharsExpr) o;
// Doesn't use "visitFn", but that's OK, because visitFn is determined entirely by "mode".
return mode == that.mode &&
Arrays.equals(chars, that.chars) &&
Objects.equals(charsExpr, that.charsExpr);
}
@Override
public int hashCode()
{
int result = Objects.hash(super.hashCode(), mode, charsExpr);
result = 31 * result + Arrays.hashCode(chars);
return result;
}
}
@VisibleForTesting
static class TrimDynamicCharsExpr implements Expr
static class TrimDynamicCharsExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
{
private final TrimMode mode;
private final Expr stringExpr;
private final Expr charsExpr;
private final Function<Shuttle, Expr> visitFn;
public TrimDynamicCharsExpr(
final TrimMode mode,
final Expr stringExpr,
final Expr charsExpr,
final Function<Shuttle, Expr> visitFn
final TrimExprMacro macro,
final List<Expr> args
)
{
this.mode = mode;
this.stringExpr = stringExpr;
this.charsExpr = charsExpr;
this.visitFn = visitFn;
super(macro, args);
this.mode = macro.mode;
this.stringExpr = args.get(0);
this.charsExpr = args.get(1);
}
@Nonnull
@ -293,55 +236,12 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
}
}
@Override
public String stringify()
{
return StringUtils.format("%s(%s, %s)", mode.getFnName(), stringExpr.stringify(), charsExpr.stringify());
}
@Override
public Expr visit(Shuttle shuttle)
{
return visitFn.apply(shuttle);
}
@Override
public BindingAnalysis analyzeInputs()
{
return stringExpr.analyzeInputs()
.with(charsExpr)
.withScalarArguments(ImmutableSet.of(stringExpr, charsExpr));
}
@Nullable
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return ExpressionType.STRING;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TrimDynamicCharsExpr that = (TrimDynamicCharsExpr) o;
// Doesn't use "visitFn", but that's OK, because visitFn is determined entirely by "mode".
return mode == that.mode &&
Objects.equals(stringExpr, that.stringExpr) &&
Objects.equals(charsExpr, that.charsExpr);
}
@Override
public int hashCode()
{
return Objects.hash(mode, stringExpr, charsExpr);
}
}
private static boolean arrayContains(char[] array, char c)

View File

@ -24,6 +24,7 @@ import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExpressionValidationException;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
import org.junit.Assert;
import org.junit.Test;
@ -41,7 +42,7 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase
private static final Expr IPV6_MAPPED = ExprEval.of("::ffff:192.168.0.1").toExpr();
private static final Expr SUBNET_192_168 = ExprEval.of("192.168.0.0/16").toExpr();
private static final Expr SUBNET_10 = ExprEval.of("10.0.0.0/8").toExpr();
private static final Expr NOT_LITERAL = new NotLiteralExpr(null);
private static final Expr NOT_LITERAL = Parser.parse("\"notliteral\"", ExprMacroTable.nil());
public IPv4AddressMatchExprMacroTest()
{
@ -210,26 +211,4 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase
ExprEval eval = expr.eval(InputBindings.nilBindings());
return eval.asBoolean();
}
/* Helper for tests */
@SuppressWarnings({"ReturnOfNull", "NullableProblems"}) // suppressed since this is a test helper class
private static class NotLiteralExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
{
NotLiteralExpr(Expr arg)
{
super("not", arg);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return null;
}
@Override
public Expr visit(Shuttle shuttle)
{
return null;
}
}
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
import org.joda.time.DateTime;
import org.joda.time.Days;
import org.joda.time.Minutes;
@ -199,7 +200,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
ImmutableList.of(
ExprEval.of(timestamp.getMillis()).toExpr(),
ExprEval.of("P1Y").toExpr(),
new NotLiteralExpr("step"),
Parser.parse("\"step\"", ExprMacroTable.nil()), // "step" is not a literal
ExprEval.of("America/Los_Angeles").toExpr()
));
@ -246,24 +247,4 @@ public class TimestampShiftMacroTest extends MacroTestBase
Assert.assertNull(expr.eval(InputBindings.nilBindings()).value());
}
}
private static class NotLiteralExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
{
NotLiteralExpr(String name)
{
super(name, ExprEval.of(name).toExpr());
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofType(bindings.getType(name), bindings.get(name));
}
@Override
public Expr visit(Shuttle shuttle)
{
return null;
}
}
}

View File

@ -28,7 +28,7 @@ public class TrimExprMacroTest
public void testEqualsContractForTrimStaticCharsExpr()
{
EqualsVerifier.forClass(TrimExprMacro.TrimStaticCharsExpr.class)
.withIgnoredFields("analyzeInputsSupplier", "visitFn")
.withIgnoredFields("analyzeInputsSupplier", "mode", "stringExpr", "chars")
.usingGetClass()
.verify();
}
@ -37,7 +37,7 @@ public class TrimExprMacroTest
public void testEqualsContractForTrimDynamicCharsExpr()
{
EqualsVerifier.forClass(TrimExprMacro.TrimDynamicCharsExpr.class)
.withIgnoredFields("visitFn")
.withIgnoredFields("analyzeInputsSupplier", "mode", "stringExpr", "charsExpr")
.usingGetClass()
.verify();
}