mirror of https://github.com/apache/druid.git
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:
parent
0f29ece6a9
commit
0f6a895372
|
@ -27,7 +27,6 @@ import org.apache.druid.query.aggregation.datasketches.hll.HllSketchHolder;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
public class HllPostAggExprMacros
|
public class HllPostAggExprMacros
|
||||||
{
|
{
|
||||||
|
@ -40,7 +39,7 @@ public class HllPostAggExprMacros
|
||||||
public Expr apply(List<Expr> args)
|
public Expr apply(List<Expr> args)
|
||||||
{
|
{
|
||||||
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
|
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
|
||||||
return new HllSketchEstimateExpr(args);
|
return new HllSketchEstimateExpr(this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -55,9 +54,9 @@ public class HllPostAggExprMacros
|
||||||
private Expr estimateExpr;
|
private Expr estimateExpr;
|
||||||
private Expr isRound;
|
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);
|
this.estimateExpr = args.get(0);
|
||||||
if (args.size() == 2) {
|
if (args.size() == 2) {
|
||||||
isRound = args.get(1);
|
isRound = args.get(1);
|
||||||
|
@ -88,13 +87,6 @@ public class HllPostAggExprMacros
|
||||||
double estimate = h.getEstimate();
|
double estimate = h.getEstimate();
|
||||||
return round ? ExprEval.of(Math.round(estimate)) : ExprEval.of(estimate);
|
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));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.query.aggregation.datasketches.theta.sql;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
|
@ -39,7 +40,7 @@ public class ThetaPostAggMacros
|
||||||
public Expr apply(List<Expr> args)
|
public Expr apply(List<Expr> args)
|
||||||
{
|
{
|
||||||
validationHelperCheckArgumentCount(args, 1);
|
validationHelperCheckArgumentCount(args, 1);
|
||||||
return new ThetaSketchEstimateExpr(args.get(0));
|
return new ThetaSketchEstimateExpr(this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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;
|
private Expr estimateExpr;
|
||||||
|
|
||||||
public ThetaSketchEstimateExpr(Expr arg)
|
public ThetaSketchEstimateExpr(ThetaSketchEstimateExprMacro macro, List<Expr> args)
|
||||||
{
|
{
|
||||||
super(THETA_SKETCH_ESTIMATE, arg);
|
super(macro, args);
|
||||||
this.estimateExpr = arg;
|
this.estimateExpr = Iterables.getOnlyElement(args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -76,12 +77,6 @@ public class ThetaPostAggMacros
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(new ThetaSketchEstimateExpr(arg));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
|
|
@ -62,14 +62,14 @@ public class BloomFilterExpressions
|
||||||
throw validationFailed("argument must be a LONG constant");
|
throw validationFailed("argument must be a LONG constant");
|
||||||
}
|
}
|
||||||
|
|
||||||
class BloomExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
|
class BloomExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
||||||
{
|
{
|
||||||
final int expectedSize;
|
final int expectedSize;
|
||||||
|
|
||||||
public BloomExpr(Expr arg)
|
public BloomExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(FN_NAME, arg);
|
super(CreateExprMacro.this, args);
|
||||||
this.expectedSize = arg.eval(InputBindings.nilBindings()).asInt();
|
this.expectedSize = args.get(0).eval(InputBindings.nilBindings()).asInt();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -81,12 +81,6 @@ public class BloomFilterExpressions
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
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)
|
private BloomExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(FN_NAME, args);
|
super(AddExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -162,13 +156,6 @@ public class BloomFilterExpressions
|
||||||
return ExprEval.ofComplex(BLOOM_FILTER_TYPE, filter);
|
return ExprEval.ofComplex(BLOOM_FILTER_TYPE, filter);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -196,13 +183,16 @@ public class BloomFilterExpressions
|
||||||
{
|
{
|
||||||
validationHelperCheckArgumentCount(args, 2);
|
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 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;
|
this.filter = filter;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -248,12 +238,6 @@ public class BloomFilterExpressions
|
||||||
return filter.testBytes(null, 0, 0);
|
return filter.testBytes(null, 0, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -266,7 +250,7 @@ public class BloomFilterExpressions
|
||||||
{
|
{
|
||||||
public DynamicBloomExpr(List<Expr> args)
|
public DynamicBloomExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(FN_NAME, args);
|
super(TestExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -319,12 +303,6 @@ public class BloomFilterExpressions
|
||||||
return filter.testBytes(null, 0, 0);
|
return filter.testBytes(null, 0, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
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) {
|
if (filterExpr.isLiteral() && filterExpr.getLiteralValue() instanceof String) {
|
||||||
final String serializedFilter = (String) filterExpr.getLiteralValue();
|
final String serializedFilter = (String) filterExpr.getLiteralValue();
|
||||||
final byte[] decoded = StringUtils.decodeBase64String(serializedFilter);
|
final byte[] decoded = StringUtils.decodeBase64String(serializedFilter);
|
||||||
|
@ -346,7 +321,7 @@ public class BloomFilterExpressions
|
||||||
catch (IOException ioe) {
|
catch (IOException ioe) {
|
||||||
throw processingFailed(ioe, "failed to deserialize bloom filter");
|
throw processingFailed(ioe, "failed to deserialize bloom filter");
|
||||||
}
|
}
|
||||||
return new BloomExpr(filter, arg);
|
return new BloomExpr(filter, args);
|
||||||
} else {
|
} else {
|
||||||
return new DynamicBloomExpr(args);
|
return new DynamicBloomExpr(args);
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.druid.query.expressions;
|
||||||
|
|
||||||
import org.apache.druid.math.expr.Expr;
|
import org.apache.druid.math.expr.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
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.ExprMacroTable.ExprMacro;
|
||||||
import org.apache.druid.math.expr.ExpressionType;
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
|
|
||||||
|
@ -52,11 +52,11 @@ public class SleepExprMacro implements ExprMacro
|
||||||
|
|
||||||
Expr arg = args.get(0);
|
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
|
@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.
|
* 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
|
* 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 null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return new SleepExpr(arg);
|
return new SleepExpr(args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.druid.testing.tools;
|
||||||
|
|
||||||
import org.apache.druid.math.expr.Expr;
|
import org.apache.druid.math.expr.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
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.ExprMacroTable.ExprMacro;
|
||||||
import org.apache.druid.math.expr.ExpressionType;
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
|
|
||||||
|
@ -52,11 +52,11 @@ public class SleepExprMacro implements ExprMacro
|
||||||
|
|
||||||
Expr arg = args.get(0);
|
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
|
@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.
|
* 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
|
* 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 null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return new SleepExpr(arg);
|
return new SleepExpr(args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,13 +19,12 @@
|
||||||
|
|
||||||
package org.apache.druid.math.expr;
|
package org.apache.druid.math.expr;
|
||||||
|
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.segment.column.TypeStrategy;
|
import org.apache.druid.segment.column.TypeStrategy;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
public class BuiltInExprMacros
|
public class BuiltInExprMacros
|
||||||
{
|
{
|
||||||
|
@ -58,7 +57,7 @@ public class BuiltInExprMacros
|
||||||
|
|
||||||
public ComplexDecodeBase64Expression(List<Expr> args)
|
public ComplexDecodeBase64Expression(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(ComplexDecodeBase64ExprMacro.this, args);
|
||||||
validationHelperCheckArgumentCount(args, 2);
|
validationHelperCheckArgumentCount(args, 2);
|
||||||
final Expr arg0 = args.get(0);
|
final Expr arg0 = args.get(0);
|
||||||
|
|
||||||
|
@ -117,13 +116,6 @@ public class BuiltInExprMacros
|
||||||
return ExprEval.ofComplex(complexType, typeStrategy.fromBytes(base64));
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -160,7 +152,7 @@ public class BuiltInExprMacros
|
||||||
public Expr apply(List<Expr> args)
|
public Expr apply(List<Expr> args)
|
||||||
{
|
{
|
||||||
validationHelperCheckArgumentCount(args, 1);
|
validationHelperCheckArgumentCount(args, 1);
|
||||||
return new StringDecodeBase64UTFExpression(args.get(0));
|
return new StringDecodeBase64UTFExpression(this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -174,11 +166,14 @@ public class BuiltInExprMacros
|
||||||
return NAME;
|
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
|
@Override
|
||||||
|
@ -191,12 +186,6 @@ public class BuiltInExprMacros
|
||||||
return new StringExpr(StringUtils.fromUtf8(StringUtils.decodeBase64String(toDecode.asString()))).eval(bindings);
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
|
|
@ -684,7 +684,7 @@ public interface Expr extends Cacheable
|
||||||
* Add set of arguments as {@link BindingAnalysis#arrayVariables} that are *directly* {@link IdentifierExpr},
|
* Add set of arguments as {@link BindingAnalysis#arrayVariables} that are *directly* {@link IdentifierExpr},
|
||||||
* else they are ignored.
|
* else they are ignored.
|
||||||
*/
|
*/
|
||||||
BindingAnalysis withArrayArguments(Set<Expr> arrayArguments)
|
public BindingAnalysis withArrayArguments(Set<Expr> arrayArguments)
|
||||||
{
|
{
|
||||||
Set<IdentifierExpr> arrayIdentifiers = new HashSet<>();
|
Set<IdentifierExpr> arrayIdentifiers = new HashSet<>();
|
||||||
for (Expr expr : arrayArguments) {
|
for (Expr expr : arrayArguments) {
|
||||||
|
@ -705,7 +705,7 @@ public interface Expr extends Cacheable
|
||||||
/**
|
/**
|
||||||
* Copy, setting if an expression has array inputs
|
* Copy, setting if an expression has array inputs
|
||||||
*/
|
*/
|
||||||
BindingAnalysis withArrayInputs(boolean hasArrays)
|
public BindingAnalysis withArrayInputs(boolean hasArrays)
|
||||||
{
|
{
|
||||||
return new BindingAnalysis(
|
return new BindingAnalysis(
|
||||||
freeVariables,
|
freeVariables,
|
||||||
|
@ -719,7 +719,7 @@ public interface Expr extends Cacheable
|
||||||
/**
|
/**
|
||||||
* Copy, setting if an expression produces an array output
|
* Copy, setting if an expression produces an array output
|
||||||
*/
|
*/
|
||||||
BindingAnalysis withArrayOutput(boolean isOutputArray)
|
public BindingAnalysis withArrayOutput(boolean isOutputArray)
|
||||||
{
|
{
|
||||||
return new BindingAnalysis(
|
return new BindingAnalysis(
|
||||||
freeVariables,
|
freeVariables,
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Suppliers;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -32,7 +31,6 @@ import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Collectors;
|
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 ExprMacro macro;
|
||||||
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 List<Expr> args;
|
protected final List<Expr> args;
|
||||||
|
|
||||||
// Use Supplier to memoize values as ExpressionSelectors#makeExprEvalSelector() can make repeated calls for them
|
// Use Supplier to memoize values as ExpressionSelectors#makeExprEvalSelector() can make repeated calls for them
|
||||||
private final Supplier<BindingAnalysis> analyzeInputsSupplier;
|
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.macro = macro;
|
||||||
this.args = args;
|
this.args = macroArgs;
|
||||||
analyzeInputsSupplier = Suppliers.memoize(this::supplyAnalyzeInputs);
|
analyzeInputsSupplier = Suppliers.memoize(this::supplyAnalyzeInputs);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -204,17 +139,30 @@ public class ExprMacroTable
|
||||||
{
|
{
|
||||||
return StringUtils.format(
|
return StringUtils.format(
|
||||||
"%s(%s)",
|
"%s(%s)",
|
||||||
name,
|
macro.name(),
|
||||||
Expr.ARG_JOINER.join(args.stream().map(Expr::stringify).iterator())
|
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
|
@Override
|
||||||
public BindingAnalysis analyzeInputs()
|
public BindingAnalysis analyzeInputs()
|
||||||
{
|
{
|
||||||
return analyzeInputsSupplier.get();
|
return analyzeInputsSupplier.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Implemented by subclasses to provide the value for {@link #analyzeInputs()}, which uses a memoized supplier.
|
||||||
|
*/
|
||||||
|
protected abstract BindingAnalysis supplyAnalyzeInputs();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o)
|
public boolean equals(Object o)
|
||||||
{
|
{
|
||||||
|
@ -225,31 +173,38 @@ public class ExprMacroTable
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
BaseScalarMacroFunctionExpr that = (BaseScalarMacroFunctionExpr) o;
|
BaseScalarMacroFunctionExpr that = (BaseScalarMacroFunctionExpr) o;
|
||||||
return Objects.equals(name, that.name) &&
|
return Objects.equals(macro, that.macro) &&
|
||||||
Objects.equals(args, that.args);
|
Objects.equals(args, that.args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
return Objects.hash(name, args);
|
return Objects.hash(macro, 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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
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));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -53,6 +53,18 @@ public class Exprs
|
||||||
return new UOE("Unable to vectorize expression: %s", msg);
|
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.
|
* Decomposes any expr into a list of exprs that, if ANDed together, are equivalent to the input expr.
|
||||||
*
|
*
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
|
||||||
|
|
||||||
import it.unimi.dsi.fastutil.doubles.DoubleArrayList;
|
import it.unimi.dsi.fastutil.doubles.DoubleArrayList;
|
||||||
import it.unimi.dsi.fastutil.doubles.DoubleList;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
|
@ -70,11 +69,11 @@ public class ArrayQuantileExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
final double rank = ((Number) rankArg.getLiteralValue()).doubleValue();
|
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
|
@Nonnull
|
||||||
|
@ -92,27 +91,15 @@ public class ArrayQuantileExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.ofDouble(quantileFromSortedArray(doubles, rank));
|
return ExprEval.ofDouble(quantileFromSortedArray(doubles, rank));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.DOUBLE;
|
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -35,7 +35,7 @@ import java.util.List;
|
||||||
* - {@code contains_string("foobar", "car") - 0 }
|
* - {@code contains_string("foobar", "car") - 0 }
|
||||||
* - {@code contains_string("foobar", "Bar") - 1 }
|
* - {@code contains_string("foobar", "Bar") - 1 }
|
||||||
* <p>
|
* <p>
|
||||||
* See {@link ContainsExprMacro} for the case-sensitive version.
|
* @see ContainsExprMacro for the case-sensitive version.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
public class CaseInsensitiveContainsExprMacro implements ExprMacroTable.ExprMacro
|
public class CaseInsensitiveContainsExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
@ -55,6 +55,6 @@ public class CaseInsensitiveContainsExprMacro implements ExprMacroTable.ExprMacr
|
||||||
|
|
||||||
final Expr arg = args.get(0);
|
final Expr arg = args.get(0);
|
||||||
final Expr searchStr = args.get(1);
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.query.expression;
|
package org.apache.druid.query.expression;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import org.apache.druid.common.config.NullHandling;
|
import org.apache.druid.common.config.NullHandling;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
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 org.apache.druid.math.expr.ExpressionType;
|
||||||
|
|
||||||
import javax.annotation.Nonnull;
|
import javax.annotation.Nonnull;
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@link Expr} class returned by {@link ContainsExprMacro} and {@link CaseInsensitiveContainsExprMacro} for
|
* {@link Expr} class returned by {@link ContainsExprMacro} and {@link CaseInsensitiveContainsExprMacro} for
|
||||||
* evaluating the expression.
|
* 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 Function<String, Boolean> searchFunction;
|
||||||
private final Expr searchStrExpr;
|
|
||||||
private final Function<Shuttle, Expr> visitFunction;
|
|
||||||
|
|
||||||
ContainsExpr(
|
ContainsExpr(
|
||||||
final String functionName,
|
final ExprMacroTable.ExprMacro macro,
|
||||||
final Expr arg,
|
final Expr arg,
|
||||||
final Expr searchStrExpr,
|
final Expr searchStrExpr,
|
||||||
final boolean caseSensitive,
|
final boolean caseSensitive
|
||||||
final Function<Shuttle, Expr> visitFunction
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this(functionName, arg, searchStrExpr, createFunction(searchStrExpr, caseSensitive), visitFunction);
|
this(macro, arg, searchStrExpr, createFunction(getSearchString(searchStrExpr, macro.name()), caseSensitive));
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainsExpr(
|
private ContainsExpr(
|
||||||
final String functionName,
|
final ExprMacroTable.ExprMacro macro,
|
||||||
final Expr arg,
|
final Expr arg,
|
||||||
final Expr searchStrExpr,
|
final Expr searchStrExpr,
|
||||||
final Function<String, Boolean> searchFunction,
|
final Function<String, Boolean> searchFunction
|
||||||
final Function<Shuttle, Expr> visitFunction
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(functionName, arg);
|
super(macro, ImmutableList.of(arg, searchStrExpr));
|
||||||
|
this.arg = arg;
|
||||||
this.searchFunction = searchFunction;
|
this.searchFunction = searchFunction;
|
||||||
this.searchStrExpr = validateSearchExpr(searchStrExpr, functionName);
|
getSearchString(searchStrExpr, macro.name());
|
||||||
this.visitFunction = visitFunction;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nonnull
|
@Nonnull
|
||||||
|
@ -81,39 +78,26 @@ class ContainsExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.LONG;
|
return ExpressionType.LONG;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
private static String getSearchString(Expr searchExpr, String functioName)
|
||||||
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)
|
|
||||||
{
|
{
|
||||||
if (!ExprUtils.isStringLiteral(searchExpr)) {
|
if (!ExprUtils.isStringLiteral(searchExpr)) {
|
||||||
throw new IAE("Function[%s] substring must be a string literal", functioName);
|
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) {
|
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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,7 +35,7 @@ import java.util.List;
|
||||||
* - {@code contains_string("foobar", "car") - 0 }
|
* - {@code contains_string("foobar", "car") - 0 }
|
||||||
* - {@code contains_string("foobar", "Bar") - 0 }
|
* - {@code contains_string("foobar", "Bar") - 0 }
|
||||||
* <p>
|
* <p>
|
||||||
* See {@link CaseInsensitiveContainsExprMacro} for the case-insensitive version.
|
* @see CaseInsensitiveContainsExprMacro for the case-insensitive version.
|
||||||
*/
|
*/
|
||||||
public class ContainsExprMacro implements ExprMacroTable.ExprMacro
|
public class ContainsExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
|
@ -54,6 +54,6 @@ public class ContainsExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
final Expr arg = args.get(0);
|
final Expr arg = args.get(0);
|
||||||
final Expr searchStr = args.get(1);
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -141,7 +141,7 @@ public class HyperUniqueExpressions
|
||||||
{
|
{
|
||||||
public HllExpr(List<Expr> args)
|
public HllExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(NAME, args);
|
super(HllAddExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -177,7 +177,8 @@ public class HyperUniqueExpressions
|
||||||
break;
|
break;
|
||||||
case DOUBLE:
|
case DOUBLE:
|
||||||
if (NullHandling.replaceWithDefault() || !input.isNumericNull()) {
|
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;
|
break;
|
||||||
case LONG:
|
case LONG:
|
||||||
|
@ -186,7 +187,8 @@ public class HyperUniqueExpressions
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case COMPLEX:
|
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());
|
collector.fold((HyperLogLogCollector) input.value());
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -200,12 +202,6 @@ public class HyperUniqueExpressions
|
||||||
return ExprEval.ofComplex(TYPE, collector);
|
return ExprEval.ofComplex(TYPE, collector);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -232,11 +228,11 @@ public class HyperUniqueExpressions
|
||||||
{
|
{
|
||||||
validationHelperCheckArgumentCount(args, 1);
|
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
|
@Override
|
||||||
|
@ -258,12 +254,6 @@ public class HyperUniqueExpressions
|
||||||
return ExprEval.ofDouble(collector.estimateCardinality());
|
return ExprEval.ofDouble(collector.estimateCardinality());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -271,7 +261,7 @@ public class HyperUniqueExpressions
|
||||||
return ExpressionType.DOUBLE;
|
return ExpressionType.DOUBLE;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return new HllExpr(args.get(0));
|
return new HllExpr(args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -290,11 +280,11 @@ public class HyperUniqueExpressions
|
||||||
{
|
{
|
||||||
validationHelperCheckArgumentCount(args, 1);
|
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
|
@Override
|
||||||
|
@ -312,12 +302,6 @@ public class HyperUniqueExpressions
|
||||||
return ExprEval.ofLong(collector.estimateCardinalityRound());
|
return ExprEval.ofLong(collector.estimateCardinalityRound());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -325,7 +309,7 @@ public class HyperUniqueExpressions
|
||||||
return ExpressionType.LONG;
|
return ExpressionType.LONG;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return new HllExpr(args.get(0));
|
return new HllExpr(args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import inet.ipaddr.AddressStringException;
|
||||||
import inet.ipaddr.IPAddress;
|
import inet.ipaddr.IPAddress;
|
||||||
import inet.ipaddr.IPAddressString;
|
import inet.ipaddr.IPAddressString;
|
||||||
import inet.ipaddr.ipv4.IPv4Address;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
|
@ -77,11 +76,11 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
||||||
final IPAddressString blockString = getSubnetInfo(args);
|
final IPAddressString blockString = getSubnetInfo(args);
|
||||||
final IPAddress block = blockString.toAddress().toPrefixBlock();
|
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
|
@Nonnull
|
||||||
|
@ -115,27 +114,15 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return address != null && block.contains(address);
|
return address != null && block.contains(address);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.LONG;
|
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) {
|
catch (AddressStringException e) {
|
||||||
|
|
|
@ -63,11 +63,11 @@ public class IPv4AddressParseExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
Expr arg = args.get(0);
|
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
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
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)
|
private static ExprEval evalAsString(ExprEval eval)
|
||||||
|
|
|
@ -62,11 +62,11 @@ public class IPv4AddressStringifyExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
Expr arg = args.get(0);
|
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
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
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)
|
private static ExprEval evalAsString(ExprEval eval)
|
||||||
|
|
|
@ -20,60 +20,58 @@
|
||||||
package org.apache.druid.query.expression;
|
package org.apache.druid.query.expression;
|
||||||
|
|
||||||
import inet.ipaddr.IPAddressString;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.druid.math.expr.ExpressionType;
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
|
|
||||||
import javax.annotation.Nonnull;
|
import javax.annotation.Nonnull;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <pre>
|
* <pre>
|
||||||
* Implements an expression that checks if an IPv6 address belongs to a subnet.
|
* Implements an expression that checks if an IPv6 address belongs to a subnet.
|
||||||
*
|
*
|
||||||
* Expression signatures:
|
* Expression signatures:
|
||||||
* - long ipv6_match(string address, string subnet)
|
* - long ipv6_match(string address, string subnet)
|
||||||
*
|
*
|
||||||
* Valid "address" argument formats are:
|
* Valid "address" argument formats are:
|
||||||
* - IPv6 address string (e.g., "2001:4860:4860::8888")
|
* - IPv6 address string (e.g., "2001:4860:4860::8888")
|
||||||
*
|
*
|
||||||
* The argument format for the "subnet" argument should be a literal in CIDR notation
|
* The argument format for the "subnet" argument should be a literal in CIDR notation
|
||||||
* (e.g., "2001:db8::/64 ").
|
* (e.g., "2001:db8::/64 ").
|
||||||
*
|
*
|
||||||
* If the "address" argument does not represent an IPv6 address then false is returned.
|
* If the "address" argument does not represent an IPv6 address then false is returned.
|
||||||
* </pre>
|
* </pre>
|
||||||
*
|
*/
|
||||||
*/
|
|
||||||
public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
public static final String FN_NAME = "ipv6_match";
|
public static final String FN_NAME = "ipv6_match";
|
||||||
private static final int ARG_SUBNET = 1;
|
private static final int ARG_SUBNET = 1;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String name()
|
public String name()
|
||||||
{
|
{
|
||||||
return FN_NAME;
|
return FN_NAME;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Expr apply(final List<Expr> args)
|
public Expr apply(final List<Expr> args)
|
||||||
{
|
{
|
||||||
validationHelperCheckArgumentCount(args, 2);
|
validationHelperCheckArgumentCount(args, 2);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final Expr arg = args.get(0);
|
final Expr arg = args.get(0);
|
||||||
final IPAddressString blockString = getSubnetInfo(args);
|
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
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(final ObjectBinding bindings)
|
public ExprEval eval(final ObjectBinding bindings)
|
||||||
|
@ -89,24 +87,12 @@ public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
||||||
}
|
}
|
||||||
return ExprEval.ofLongBoolean(match);
|
return ExprEval.ofLongBoolean(match);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isStringMatch(String stringValue)
|
private boolean isStringMatch(String stringValue)
|
||||||
{
|
{
|
||||||
IPAddressString addressString = IPv6AddressExprUtils.parseString(stringValue);
|
IPAddressString addressString = IPv6AddressExprUtils.parseString(stringValue);
|
||||||
return addressString != null && blockString.prefixContains(addressString);
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
|
@ -115,14 +101,14 @@ public class IPv6AddressMatchExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExpressionType.LONG;
|
return ExpressionType.LONG;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return new IPv6AddressMatchExpr(arg);
|
return new IPv6AddressMatchExpr(args);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw processingFailed(e, "failed to parse address");
|
throw processingFailed(e, "failed to parse address");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private IPAddressString getSubnetInfo(List<Expr> args)
|
private IPAddressString getSubnetInfo(List<Expr> args)
|
||||||
{
|
{
|
||||||
String subnetArgName = "subnet";
|
String subnetArgName = "subnet";
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.query.expression;
|
package org.apache.druid.query.expression;
|
||||||
|
|
||||||
import org.apache.druid.common.config.NullHandling;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
|
@ -70,11 +69,11 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
|
||||||
escapeChar
|
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
|
@Nonnull
|
||||||
|
@ -88,35 +87,13 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.ofLongBoolean(match.matches(false));
|
return ExprEval.ofLongBoolean(match.matches(false));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.LONG;
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
|
||||||
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.common.config.NullHandling;
|
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.Evals;
|
||||||
import org.apache.druid.math.expr.Expr;
|
import org.apache.druid.math.expr.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
|
@ -74,15 +73,15 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
|
||||||
replaceMissingValueWith != null && replaceMissingValueWith.isLiteral()
|
replaceMissingValueWith != null && replaceMissingValueWith.isLiteral()
|
||||||
? Evals.asString(replaceMissingValueWith.getLiteralValue())
|
? Evals.asString(replaceMissingValueWith.getLiteralValue())
|
||||||
: null,
|
: null,
|
||||||
false,
|
null,
|
||||||
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
|
@Nonnull
|
||||||
|
@ -92,12 +91,6 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(extractionFn.apply(NullHandling.emptyToNullIfNeeded(arg.eval(bindings).asString())));
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -105,21 +98,6 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExpressionType.STRING;
|
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
|
@Override
|
||||||
public void decorateCacheKeyBuilder(CacheKeyBuilder builder)
|
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)
|
private Expr getReplaceMissingValueWith(final List<Expr> args)
|
||||||
|
|
|
@ -68,7 +68,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public StructExpr(List<Expr> args)
|
public StructExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(NAME, args);
|
super(JsonObjectExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -88,13 +88,6 @@ public class NestedDataExpressions
|
||||||
return ExprEval.ofComplex(ExpressionType.NESTED_DATA, theMap);
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -133,7 +126,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public ToJsonStringExpr(List<Expr> args)
|
public ToJsonStringExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(ToJsonStringExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -202,7 +188,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public ParseJsonExpr(List<Expr> args)
|
public ParseJsonExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(ParseJsonExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -275,7 +254,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public ParseJsonExpr(List<Expr> args)
|
public ParseJsonExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(TryParseJsonExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -350,7 +322,7 @@ public class NestedDataExpressions
|
||||||
|
|
||||||
public JsonValueExpr(List<Expr> args)
|
public JsonValueExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonValueExprMacro.this, args);
|
||||||
this.parts = getJsonPathPartsFromLiteral(JsonValueExprMacro.this, args.get(1));
|
this.parts = getJsonPathPartsFromLiteral(JsonValueExprMacro.this, args.get(1));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -367,17 +339,6 @@ public class NestedDataExpressions
|
||||||
return ExprEval.of(null);
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -394,7 +355,7 @@ public class NestedDataExpressions
|
||||||
|
|
||||||
public JsonValueCastExpr(List<Expr> args)
|
public JsonValueCastExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonValueExprMacro.this, args);
|
||||||
this.parts = getJsonPathPartsFromLiteral(JsonValueExprMacro.this, args.get(1));
|
this.parts = getJsonPathPartsFromLiteral(JsonValueExprMacro.this, args.get(1));
|
||||||
this.castTo = ExpressionType.fromString((String) args.get(2).getLiteralValue());
|
this.castTo = ExpressionType.fromString((String) args.get(2).getLiteralValue());
|
||||||
if (castTo == null) {
|
if (castTo == null) {
|
||||||
|
@ -418,17 +379,6 @@ public class NestedDataExpressions
|
||||||
return ExprEval.ofType(castTo, null);
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -441,7 +391,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public JsonValueDynamicExpr(List<Expr> args)
|
public JsonValueDynamicExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonValueExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -469,21 +419,6 @@ public class NestedDataExpressions
|
||||||
return castTo == null ? ExprEval.of(null) : ExprEval.ofType(castTo, null);
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -520,7 +455,7 @@ public class NestedDataExpressions
|
||||||
|
|
||||||
public JsonQueryExpr(List<Expr> args)
|
public JsonQueryExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonQueryExprMacro.this, args);
|
||||||
this.parts = getJsonPathPartsFromLiteral(JsonQueryExprMacro.this, args.get(1));
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -558,7 +482,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public JsonQueryDynamicExpr(List<Expr> args)
|
public JsonQueryDynamicExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonQueryExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -620,7 +533,7 @@ public class NestedDataExpressions
|
||||||
|
|
||||||
public JsonQueryArrayExpr(List<Expr> args)
|
public JsonQueryArrayExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonQueryArrayExprMacro.this, args);
|
||||||
this.parts = getJsonPathPartsFromLiteral(JsonQueryArrayExprMacro.this, args.get(1));
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -665,7 +567,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public JsonQueryArrayDynamicExpr(List<Expr> args)
|
public JsonQueryArrayDynamicExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonQueryArrayExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -750,7 +641,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public JsonPathsExpr(List<Expr> args)
|
public JsonPathsExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonPathsExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -805,7 +689,7 @@ public class NestedDataExpressions
|
||||||
{
|
{
|
||||||
public JsonKeysExpr(List<Expr> args)
|
public JsonKeysExpr(List<Expr> args)
|
||||||
{
|
{
|
||||||
super(name(), args);
|
super(JsonKeysExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
|
|
|
@ -66,11 +66,11 @@ public class RegexpExtractExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
final int index = indexExpr == null ? 0 : ((Number) indexExpr.getLiteralValue()).intValue();
|
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
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.STRING;
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -59,11 +59,11 @@ public class RegexpLikeExprMacro implements ExprMacroTable.ExprMacro
|
||||||
StringUtils.nullToEmptyNonDruidDataString((String) patternExpr.getLiteralValue())
|
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
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.LONG;
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -57,7 +57,7 @@ public class RegexpReplaceExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
public BaseRegexpReplaceExpr(final List<Expr> args)
|
public BaseRegexpReplaceExpr(final List<Expr> args)
|
||||||
{
|
{
|
||||||
super(FN_NAME, args);
|
super(RegexpReplaceExprMacro.this, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -66,12 +66,6 @@ public class RegexpReplaceExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
return ExpressionType.STRING;
|
return ExpressionType.STRING;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -49,9 +49,9 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
||||||
validationHelperCheckArgumentRange(args, 2, 4);
|
validationHelperCheckArgumentRange(args, 2, 4);
|
||||||
|
|
||||||
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
|
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
|
||||||
return new TimestampCeilExpr(args);
|
return new TimestampCeilExpr(this, args);
|
||||||
} else {
|
} else {
|
||||||
return new TimestampCeilDynamicExpr(args);
|
return new TimestampCeilDynamicExpr(this, args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -60,9 +60,9 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
private final Granularity granularity;
|
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());
|
this.granularity = getGranularity(args, InputBindings.nilBindings());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -83,12 +83,6 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(granularity.increment(bucketStartTime));
|
return ExprEval.of(granularity.increment(bucketStartTime));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(new TimestampCeilExpr(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -132,9 +126,9 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static class TimestampCeilDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
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
|
@Nonnull
|
||||||
|
@ -150,12 +144,6 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(granularity.increment(bucketStartTime));
|
return ExprEval.of(granularity.increment(bucketStartTime));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(new TimestampCeilDynamicExpr(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
|
|
@ -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)
|
private static ISOChronology computeChronology(final List<Expr> args, final Expr.ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
String timeZoneVal = (String) args.get(2).eval(bindings).value();
|
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)
|
private TimestampExtractExpr(final List<Expr> args, final Unit unit, final ISOChronology chronology)
|
||||||
{
|
{
|
||||||
super(FN_NAME, args);
|
super(TimestampExtractExprMacro.this, args);
|
||||||
this.unit = unit;
|
this.unit = unit;
|
||||||
this.chronology = chronology;
|
this.chronology = chronology;
|
||||||
}
|
}
|
||||||
|
@ -194,24 +180,12 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return getExprEval(dateTime, unit);
|
return getExprEval(dateTime, unit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return getOutputExpressionType(unit);
|
return getOutputExpressionType(unit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public String stringify()
|
|
||||||
{
|
|
||||||
return stringifyExpr(args);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public class TimestampExtractDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
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)
|
private TimestampExtractDynamicExpr(final List<Expr> args, final Unit unit)
|
||||||
{
|
{
|
||||||
super(FN_NAME, args);
|
super(TimestampExtractExprMacro.this, args);
|
||||||
this.unit = unit;
|
this.unit = unit;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -238,23 +212,11 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return getExprEval(dateTime, unit);
|
return getExprEval(dateTime, unit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return getOutputExpressionType(unit);
|
return getOutputExpressionType(unit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public String stringify()
|
|
||||||
{
|
|
||||||
return stringifyExpr(args);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -50,9 +50,9 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
||||||
validationHelperCheckArgumentRange(args, 2, 4);
|
validationHelperCheckArgumentRange(args, 2, 4);
|
||||||
|
|
||||||
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
|
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
|
||||||
return new TimestampFloorExpr(args);
|
return new TimestampFloorExpr(this, args);
|
||||||
} else {
|
} else {
|
||||||
return new TimestampFloorDynamicExpr(args);
|
return new TimestampFloorDynamicExpr(this, args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -70,9 +70,9 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
private final PeriodGranularity granularity;
|
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());
|
this.granularity = computeGranularity(args, InputBindings.nilBindings());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -104,12 +104,6 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(granularity.bucketStart(eval.asLong()));
|
return ExprEval.of(granularity.bucketStart(eval.asLong()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(new TimestampFloorExpr(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
@ -167,9 +161,9 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
public static class TimestampFloorDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
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
|
@Nonnull
|
||||||
|
@ -180,12 +174,6 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(granularity.bucketStart(args.get(0).eval(bindings).asLong()));
|
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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.query.expression;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
|
@ -69,11 +68,11 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
|
||||||
? ISODateTimeFormat.dateTime().withZone(timeZone)
|
? ISODateTimeFormat.dateTime().withZone(timeZone)
|
||||||
: DateTimeFormat.forPattern(formatString).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
|
@Nonnull
|
||||||
|
@ -88,38 +87,14 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(formatter.print(arg.eval(bindings).asLong()));
|
return ExprEval.of(formatter.print(arg.eval(bindings).asLong()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.STRING;
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.query.expression;
|
package org.apache.druid.query.expression;
|
||||||
|
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
|
@ -66,11 +65,11 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
|
||||||
? createDefaultParser(timeZone)
|
? createDefaultParser(timeZone)
|
||||||
: DateTimes.wrapFormatter(DateTimeFormat.forPattern(formatString).withZone(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
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.LONG;
|
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -52,11 +52,11 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
||||||
validationHelperCheckArgumentRange(args, 3, 4);
|
validationHelperCheckArgumentRange(args, 3, 4);
|
||||||
|
|
||||||
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
|
if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
|
||||||
return new TimestampShiftExpr(args);
|
return new TimestampShiftExpr(this, args);
|
||||||
} else {
|
} else {
|
||||||
// Use dynamic impl if any args are non-literal. Don't bother optimizing for the case where period is
|
// Use dynamic impl if any args are non-literal. Don't bother optimizing for the case where period is
|
||||||
// literal but step isn't.
|
// 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 Period period;
|
||||||
private final int step;
|
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());
|
period = getPeriod(args, InputBindings.nilBindings());
|
||||||
chronology = getTimeZone(args, InputBindings.nilBindings());
|
chronology = getTimeZone(args, InputBindings.nilBindings());
|
||||||
step = getStep(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));
|
return ExprEval.of(chronology.add(period, timestamp.asLong(), step));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(new TimestampShiftExpr(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean canVectorize(InputBindingInspector inspector)
|
public boolean canVectorize(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
|
@ -147,9 +141,9 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
|
||||||
private static class TimestampShiftDynamicExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
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
|
@Nonnull
|
||||||
|
@ -166,12 +160,6 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
||||||
return ExprEval.of(chronology.add(period, timestamp.asLong(), step));
|
return ExprEval.of(chronology.add(period, timestamp.asLong(), step));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return shuttle.visit(new TimestampShiftDynamicExpr(shuttle.visitAll(args)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
|
|
|
@ -20,8 +20,6 @@
|
||||||
package org.apache.druid.query.expression;
|
package org.apache.druid.query.expression;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
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.Expr;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
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.Nonnull;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.function.Function;
|
|
||||||
|
|
||||||
public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
|
@ -91,50 +86,44 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
||||||
{
|
{
|
||||||
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
|
validationHelperCheckAnyOfArgumentCount(args, 1, 2);
|
||||||
|
|
||||||
final Function<Expr.Shuttle, Expr> visitFn = shuttle -> shuttle.visit(apply(shuttle.visitAll(args)));
|
|
||||||
|
|
||||||
if (args.size() == 1) {
|
if (args.size() == 1) {
|
||||||
return new TrimStaticCharsExpr(mode, args.get(0), DEFAULT_CHARS, null, visitFn);
|
return new TrimStaticCharsExpr(this, args, DEFAULT_CHARS);
|
||||||
} else {
|
} else {
|
||||||
final Expr charsArg = args.get(1);
|
final Expr charsArg = args.get(1);
|
||||||
if (charsArg.isLiteral()) {
|
if (charsArg.isLiteral()) {
|
||||||
final String charsString = charsArg.eval(InputBindings.nilBindings()).asString();
|
final String charsString = charsArg.eval(InputBindings.nilBindings()).asString();
|
||||||
final char[] chars = charsString == null ? EMPTY_CHARS : charsString.toCharArray();
|
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 {
|
} else {
|
||||||
return new TrimDynamicCharsExpr(mode, args.get(0), args.get(1), visitFn);
|
return new TrimDynamicCharsExpr(this, args);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static class TrimStaticCharsExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
|
static class TrimStaticCharsExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
||||||
{
|
{
|
||||||
private final TrimMode mode;
|
private final TrimMode mode;
|
||||||
private final char[] chars;
|
private final char[] chars;
|
||||||
private final Expr charsExpr;
|
private final Expr stringExpr;
|
||||||
private final Function<Shuttle, Expr> visitFn;
|
|
||||||
|
|
||||||
public TrimStaticCharsExpr(
|
public TrimStaticCharsExpr(
|
||||||
final TrimMode mode,
|
final TrimExprMacro macro,
|
||||||
final Expr stringExpr,
|
final List<Expr> args,
|
||||||
final char[] chars,
|
final char[] chars
|
||||||
final Expr charsExpr,
|
|
||||||
final Function<Shuttle, Expr> visitFn
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(mode.getFnName(), stringExpr);
|
super(macro, args);
|
||||||
this.mode = mode;
|
this.mode = macro.mode;
|
||||||
|
this.stringExpr = args.get(0);
|
||||||
this.chars = chars;
|
this.chars = chars;
|
||||||
this.charsExpr = charsExpr;
|
|
||||||
this.visitFn = visitFn;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nonnull
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(final ObjectBinding bindings)
|
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) {
|
if (chars.length == 0 || stringEval.value() == null) {
|
||||||
return stringEval;
|
return stringEval;
|
||||||
|
@ -172,76 +161,30 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Expr visit(Shuttle shuttle)
|
|
||||||
{
|
|
||||||
return visitFn.apply(shuttle);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.STRING;
|
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
|
@VisibleForTesting
|
||||||
static class TrimDynamicCharsExpr implements Expr
|
static class TrimDynamicCharsExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
||||||
{
|
{
|
||||||
private final TrimMode mode;
|
private final TrimMode mode;
|
||||||
private final Expr stringExpr;
|
private final Expr stringExpr;
|
||||||
private final Expr charsExpr;
|
private final Expr charsExpr;
|
||||||
private final Function<Shuttle, Expr> visitFn;
|
|
||||||
|
|
||||||
public TrimDynamicCharsExpr(
|
public TrimDynamicCharsExpr(
|
||||||
final TrimMode mode,
|
final TrimExprMacro macro,
|
||||||
final Expr stringExpr,
|
final List<Expr> args
|
||||||
final Expr charsExpr,
|
|
||||||
final Function<Shuttle, Expr> visitFn
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.mode = mode;
|
super(macro, args);
|
||||||
this.stringExpr = stringExpr;
|
this.mode = macro.mode;
|
||||||
this.charsExpr = charsExpr;
|
this.stringExpr = args.get(0);
|
||||||
this.visitFn = visitFn;
|
this.charsExpr = args.get(1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nonnull
|
@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
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||||
{
|
{
|
||||||
return ExpressionType.STRING;
|
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)
|
private static boolean arrayContains(char[] array, char c)
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.druid.math.expr.ExpressionValidationException;
|
import org.apache.druid.math.expr.ExpressionValidationException;
|
||||||
import org.apache.druid.math.expr.InputBindings;
|
import org.apache.druid.math.expr.InputBindings;
|
||||||
|
import org.apache.druid.math.expr.Parser;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
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 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_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 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()
|
public IPv4AddressMatchExprMacroTest()
|
||||||
{
|
{
|
||||||
|
@ -210,26 +211,4 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase
|
||||||
ExprEval eval = expr.eval(InputBindings.nilBindings());
|
ExprEval eval = expr.eval(InputBindings.nilBindings());
|
||||||
return eval.asBoolean();
|
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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.druid.math.expr.ExpressionType;
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
import org.apache.druid.math.expr.InputBindings;
|
import org.apache.druid.math.expr.InputBindings;
|
||||||
|
import org.apache.druid.math.expr.Parser;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Days;
|
import org.joda.time.Days;
|
||||||
import org.joda.time.Minutes;
|
import org.joda.time.Minutes;
|
||||||
|
@ -199,7 +200,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||||
ExprEval.of("P1Y").toExpr(),
|
ExprEval.of("P1Y").toExpr(),
|
||||||
new NotLiteralExpr("step"),
|
Parser.parse("\"step\"", ExprMacroTable.nil()), // "step" is not a literal
|
||||||
ExprEval.of("America/Los_Angeles").toExpr()
|
ExprEval.of("America/Los_Angeles").toExpr()
|
||||||
));
|
));
|
||||||
|
|
||||||
|
@ -246,24 +247,4 @@ public class TimestampShiftMacroTest extends MacroTestBase
|
||||||
Assert.assertNull(expr.eval(InputBindings.nilBindings()).value());
|
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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,7 +28,7 @@ public class TrimExprMacroTest
|
||||||
public void testEqualsContractForTrimStaticCharsExpr()
|
public void testEqualsContractForTrimStaticCharsExpr()
|
||||||
{
|
{
|
||||||
EqualsVerifier.forClass(TrimExprMacro.TrimStaticCharsExpr.class)
|
EqualsVerifier.forClass(TrimExprMacro.TrimStaticCharsExpr.class)
|
||||||
.withIgnoredFields("analyzeInputsSupplier", "visitFn")
|
.withIgnoredFields("analyzeInputsSupplier", "mode", "stringExpr", "chars")
|
||||||
.usingGetClass()
|
.usingGetClass()
|
||||||
.verify();
|
.verify();
|
||||||
}
|
}
|
||||||
|
@ -37,7 +37,7 @@ public class TrimExprMacroTest
|
||||||
public void testEqualsContractForTrimDynamicCharsExpr()
|
public void testEqualsContractForTrimDynamicCharsExpr()
|
||||||
{
|
{
|
||||||
EqualsVerifier.forClass(TrimExprMacro.TrimDynamicCharsExpr.class)
|
EqualsVerifier.forClass(TrimExprMacro.TrimDynamicCharsExpr.class)
|
||||||
.withIgnoredFields("visitFn")
|
.withIgnoredFields("analyzeInputsSupplier", "mode", "stringExpr", "charsExpr")
|
||||||
.usingGetClass()
|
.usingGetClass()
|
||||||
.verify();
|
.verify();
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue