mirror of https://github.com/apache/druid.git
Expressions: Add ExprMacros. (#4365)
* Expressions: Add ExprMacros, which have the same syntax as functions, but can convert themselves to any kind of Expr at parse-time. ExprMacroTable is an extension point for adding new ExprMacros. Anything that might need to parse expressions needs an ExprMacroTable, which can be injected through Guice. * Address code review comments.
This commit is contained in:
parent
9ae04b7375
commit
1f2afccdf8
|
@ -31,6 +31,7 @@ import io.druid.js.JavaScriptConfig;
|
||||||
import io.druid.query.aggregation.BufferAggregator;
|
import io.druid.query.aggregation.BufferAggregator;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.Cursor;
|
import io.druid.segment.Cursor;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
@ -112,7 +113,8 @@ public class ExpressionBenchmark
|
||||||
this.expressionAggregatorFactory = new DoubleSumAggregatorFactory(
|
this.expressionAggregatorFactory = new DoubleSumAggregatorFactory(
|
||||||
"name",
|
"name",
|
||||||
null,
|
null,
|
||||||
"if(x>0,1.0+x,y+1)"
|
"if(x>0,1.0+x,y+1)",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -148,6 +148,7 @@ public class SqlBenchmark
|
||||||
Calcites.createRootSchema(druidSchema),
|
Calcites.createRootSchema(druidSchema),
|
||||||
walker,
|
walker,
|
||||||
CalciteTests.createOperatorTable(),
|
CalciteTests.createOperatorTable(),
|
||||||
|
CalciteTests.createExprMacroTable(),
|
||||||
plannerConfig,
|
plannerConfig,
|
||||||
new ServerConfig()
|
new ServerConfig()
|
||||||
);
|
);
|
||||||
|
|
|
@ -20,14 +20,39 @@
|
||||||
package io.druid.math.expr;
|
package io.druid.math.expr;
|
||||||
|
|
||||||
import com.google.common.math.LongMath;
|
import com.google.common.math.LongMath;
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
import io.druid.java.util.common.IAE;
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.java.util.common.ISE;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface Expr
|
public interface Expr
|
||||||
{
|
{
|
||||||
|
default boolean isLiteral()
|
||||||
|
{
|
||||||
|
// Overridden by things that are literals.
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the value of expr if expr is a literal, or throws an exception otherwise.
|
||||||
|
*
|
||||||
|
* @return expr's literal value
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if expr is not a literal
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
default Object getLiteralValue()
|
||||||
|
{
|
||||||
|
// Overridden by things that are literals.
|
||||||
|
throw new ISE("Not a literal");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
ExprEval eval(ObjectBinding bindings);
|
ExprEval eval(ObjectBinding bindings);
|
||||||
|
|
||||||
interface ObjectBinding
|
interface ObjectBinding
|
||||||
|
@ -45,6 +70,12 @@ public interface Expr
|
||||||
|
|
||||||
abstract class ConstantExpr implements Expr
|
abstract class ConstantExpr implements Expr
|
||||||
{
|
{
|
||||||
|
@Override
|
||||||
|
public boolean isLiteral()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void visit(Visitor visitor)
|
public void visit(Visitor visitor)
|
||||||
{
|
{
|
||||||
|
@ -61,12 +92,20 @@ class LongExpr extends ConstantExpr
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Object getLiteralValue()
|
||||||
|
{
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
return String.valueOf(value);
|
return String.valueOf(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -83,12 +122,20 @@ class StringExpr extends ConstantExpr
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Object getLiteralValue()
|
||||||
|
{
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -105,12 +152,20 @@ class DoubleExpr extends ConstantExpr
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Object getLiteralValue()
|
||||||
|
{
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
return String.valueOf(value);
|
return String.valueOf(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -133,6 +188,7 @@ class IdentifierExpr implements Expr
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -148,11 +204,13 @@ class IdentifierExpr implements Expr
|
||||||
|
|
||||||
class FunctionExpr implements Expr
|
class FunctionExpr implements Expr
|
||||||
{
|
{
|
||||||
|
final Function function;
|
||||||
final String name;
|
final String name;
|
||||||
final List<Expr> args;
|
final List<Expr> args;
|
||||||
|
|
||||||
public FunctionExpr(String name, List<Expr> args)
|
public FunctionExpr(Function function, String name, List<Expr> args)
|
||||||
{
|
{
|
||||||
|
this.function = function;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.args = args;
|
this.args = args;
|
||||||
}
|
}
|
||||||
|
@ -163,10 +221,11 @@ class FunctionExpr implements Expr
|
||||||
return "(" + name + " " + args + ")";
|
return "(" + name + " " + args + ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
return Parser.getFunction(name).apply(args, bindings);
|
return function.apply(args, bindings);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -203,6 +262,7 @@ class UnaryMinusExpr extends UnaryExpr
|
||||||
super(expr);
|
super(expr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -237,6 +297,7 @@ class UnaryNotExpr extends UnaryExpr
|
||||||
super(expr);
|
super(expr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -290,6 +351,7 @@ abstract class BinaryEvalOpExprBase extends BinaryOpExprBase
|
||||||
super(op, left, right);
|
super(op, left, right);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -347,7 +409,7 @@ class BinPowExpr extends BinaryEvalOpExprBase
|
||||||
@Override
|
@Override
|
||||||
protected final long evalLong(long left, long right)
|
protected final long evalLong(long left, long right)
|
||||||
{
|
{
|
||||||
return LongMath.pow(left, (int)right);
|
return LongMath.pow(left, Ints.checkedCast(right));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -606,6 +668,7 @@ class BinAndExpr extends BinaryOpExprBase
|
||||||
super(op, left, right);
|
super(op, left, right);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
@ -621,6 +684,7 @@ class BinOrExpr extends BinaryOpExprBase
|
||||||
super(op, left, right);
|
super(op, left, right);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
@Override
|
@Override
|
||||||
public ExprEval eval(ObjectBinding bindings)
|
public ExprEval eval(ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.math.expr;
|
package io.druid.math.expr;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.RE;
|
||||||
import io.druid.math.expr.antlr.ExprBaseListener;
|
import io.druid.math.expr.antlr.ExprBaseListener;
|
||||||
import io.druid.math.expr.antlr.ExprParser;
|
import io.druid.math.expr.antlr.ExprParser;
|
||||||
import org.antlr.v4.runtime.tree.ParseTree;
|
import org.antlr.v4.runtime.tree.ParseTree;
|
||||||
|
@ -36,11 +37,13 @@ import java.util.Map;
|
||||||
public class ExprListenerImpl extends ExprBaseListener
|
public class ExprListenerImpl extends ExprBaseListener
|
||||||
{
|
{
|
||||||
private final Map<ParseTree, Object> nodes;
|
private final Map<ParseTree, Object> nodes;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
private final ParseTree rootNodeKey;
|
private final ParseTree rootNodeKey;
|
||||||
|
|
||||||
ExprListenerImpl(ParseTree rootNodeKey)
|
ExprListenerImpl(ParseTree rootNodeKey, ExprMacroTable macroTable)
|
||||||
{
|
{
|
||||||
this.rootNodeKey = rootNodeKey;
|
this.rootNodeKey = rootNodeKey;
|
||||||
|
this.macroTable = macroTable;
|
||||||
this.nodes = new HashMap<>();
|
this.nodes = new HashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -285,15 +288,22 @@ public class ExprListenerImpl extends ExprBaseListener
|
||||||
public void exitFunctionExpr(ExprParser.FunctionExprContext ctx)
|
public void exitFunctionExpr(ExprParser.FunctionExprContext ctx)
|
||||||
{
|
{
|
||||||
String fnName = ctx.getChild(0).getText();
|
String fnName = ctx.getChild(0).getText();
|
||||||
if (!Parser.hasFunction(fnName)) {
|
final List<Expr> args = ctx.getChildCount() > 3
|
||||||
throw new RuntimeException("function " + fnName + " is not defined.");
|
? (List<Expr>) nodes.get(ctx.getChild(2))
|
||||||
|
: Collections.emptyList();
|
||||||
|
|
||||||
|
Expr expr = macroTable.get(fnName, args);
|
||||||
|
|
||||||
|
if (expr == null) {
|
||||||
|
// Built-in functions.
|
||||||
|
final Function function = Parser.getFunction(fnName);
|
||||||
|
if (function == null) {
|
||||||
|
throw new RE("function '%s' is not defined.", fnName);
|
||||||
|
}
|
||||||
|
expr = new FunctionExpr(function, fnName, args);
|
||||||
}
|
}
|
||||||
|
|
||||||
List<Expr> args = ctx.getChildCount() > 3 ? (List<Expr>) nodes.get(ctx.getChild(2)) : Collections.<Expr>emptyList();
|
nodes.put(ctx, expr);
|
||||||
nodes.put(
|
|
||||||
ctx,
|
|
||||||
new FunctionExpr(fnName, args)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,75 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.math.expr;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class ExprMacroTable
|
||||||
|
{
|
||||||
|
private static final ExprMacroTable NIL = new ExprMacroTable(Collections.emptyList());
|
||||||
|
|
||||||
|
private final Map<String, ExprMacro> macroMap;
|
||||||
|
|
||||||
|
public ExprMacroTable(final List<ExprMacro> macros)
|
||||||
|
{
|
||||||
|
this.macroMap = macros.stream().collect(
|
||||||
|
Collectors.toMap(
|
||||||
|
m -> m.name().toLowerCase(),
|
||||||
|
m -> m
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ExprMacroTable nil()
|
||||||
|
{
|
||||||
|
return NIL;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns an expr corresponding to a function call if this table has an entry for {@code functionName}.
|
||||||
|
* Otherwise, returns null.
|
||||||
|
*
|
||||||
|
* @param functionName function name
|
||||||
|
* @param args function arguments
|
||||||
|
*
|
||||||
|
* @return expr for this function call, or null
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public Expr get(final String functionName, final List<Expr> args)
|
||||||
|
{
|
||||||
|
final ExprMacro exprMacro = macroMap.get(functionName.toLowerCase());
|
||||||
|
if (exprMacro == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
return exprMacro.apply(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
public interface ExprMacro
|
||||||
|
{
|
||||||
|
String name();
|
||||||
|
|
||||||
|
Expr apply(final List<Expr> args);
|
||||||
|
}
|
||||||
|
}
|
|
@ -24,7 +24,6 @@ import org.joda.time.DateTime;
|
||||||
import org.joda.time.format.DateTimeFormat;
|
import org.joda.time.format.DateTimeFormat;
|
||||||
import org.joda.time.format.DateTimeFormatter;
|
import org.joda.time.format.DateTimeFormatter;
|
||||||
import org.joda.time.format.ISODateTimeFormat;
|
import org.joda.time.format.ISODateTimeFormat;
|
||||||
import com.google.common.base.Supplier;
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
@ -36,10 +35,6 @@ interface Function
|
||||||
|
|
||||||
ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings);
|
ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings);
|
||||||
|
|
||||||
// optional interface to be used when function should be created per reference in expression
|
|
||||||
interface FunctionFactory extends Supplier<Function>, Function {
|
|
||||||
}
|
|
||||||
|
|
||||||
abstract class SingleParam implements Function
|
abstract class SingleParam implements Function
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -20,13 +20,12 @@
|
||||||
package io.druid.math.expr;
|
package io.druid.math.expr;
|
||||||
|
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Suppliers;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import io.druid.java.util.common.IAE;
|
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
import io.druid.math.expr.antlr.ExprLexer;
|
import io.druid.math.expr.antlr.ExprLexer;
|
||||||
import io.druid.math.expr.antlr.ExprParser;
|
import io.druid.math.expr.antlr.ExprParser;
|
||||||
|
@ -43,47 +42,36 @@ import java.util.Set;
|
||||||
public class Parser
|
public class Parser
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(Parser.class);
|
private static final Logger log = new Logger(Parser.class);
|
||||||
private static final Map<String, Supplier<Function>> func;
|
private static final Map<String, Function> FUNCTIONS;
|
||||||
|
|
||||||
static {
|
static {
|
||||||
Map<String, Supplier<Function>> functionMap = Maps.newHashMap();
|
Map<String, Function> functionMap = Maps.newHashMap();
|
||||||
for (Class clazz : Function.class.getClasses()) {
|
for (Class clazz : Function.class.getClasses()) {
|
||||||
if (!Modifier.isAbstract(clazz.getModifiers()) && Function.class.isAssignableFrom(clazz)) {
|
if (!Modifier.isAbstract(clazz.getModifiers()) && Function.class.isAssignableFrom(clazz)) {
|
||||||
try {
|
try {
|
||||||
Function function = (Function)clazz.newInstance();
|
Function function = (Function) clazz.newInstance();
|
||||||
if (function instanceof Function.FunctionFactory) {
|
functionMap.put(function.name().toLowerCase(), function);
|
||||||
functionMap.put(function.name().toLowerCase(), (Supplier<Function>) function);
|
|
||||||
} else {
|
|
||||||
functionMap.put(function.name().toLowerCase(), Suppliers.ofInstance(function));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.info("failed to instantiate " + clazz.getName() + ".. ignoring", e);
|
log.info("failed to instantiate " + clazz.getName() + ".. ignoring", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
func = ImmutableMap.copyOf(functionMap);
|
FUNCTIONS = ImmutableMap.copyOf(functionMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Function getFunction(String name) {
|
public static Function getFunction(String name)
|
||||||
Supplier<Function> supplier = func.get(name.toLowerCase());
|
|
||||||
if (supplier == null) {
|
|
||||||
throw new IAE("Invalid function name '%s'", name);
|
|
||||||
}
|
|
||||||
return supplier.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static boolean hasFunction(String name)
|
|
||||||
{
|
{
|
||||||
return func.containsKey(name.toLowerCase());
|
return FUNCTIONS.get(name.toLowerCase());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Expr parse(String in)
|
public static Expr parse(String in, ExprMacroTable macroTable)
|
||||||
{
|
{
|
||||||
return parse(in, true);
|
return parse(in, macroTable, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Expr parse(String in, boolean withFlatten)
|
@VisibleForTesting
|
||||||
|
static Expr parse(String in, ExprMacroTable macroTable, boolean withFlatten)
|
||||||
{
|
{
|
||||||
ExprLexer lexer = new ExprLexer(new ANTLRInputStream(in));
|
ExprLexer lexer = new ExprLexer(new ANTLRInputStream(in));
|
||||||
CommonTokenStream tokens = new CommonTokenStream(lexer);
|
CommonTokenStream tokens = new CommonTokenStream(lexer);
|
||||||
|
@ -91,7 +79,7 @@ public class Parser
|
||||||
parser.setBuildParseTree(true);
|
parser.setBuildParseTree(true);
|
||||||
ParseTree parseTree = parser.expr();
|
ParseTree parseTree = parser.expr();
|
||||||
ParseTreeWalker walker = new ParseTreeWalker();
|
ParseTreeWalker walker = new ParseTreeWalker();
|
||||||
ExprListenerImpl listener = new ExprListenerImpl(parseTree);
|
ExprListenerImpl listener = new ExprListenerImpl(parseTree, macroTable);
|
||||||
walker.walk(listener, parseTree);
|
walker.walk(listener, parseTree);
|
||||||
return withFlatten ? flatten(listener.getAST()) : listener.getAST();
|
return withFlatten ? flatten(listener.getAST()) : listener.getAST();
|
||||||
}
|
}
|
||||||
|
@ -134,17 +122,12 @@ public class Parser
|
||||||
if (Evals.isAllConstants(flattening)) {
|
if (Evals.isAllConstants(flattening)) {
|
||||||
expr = expr.eval(null).toExpr();
|
expr = expr.eval(null).toExpr();
|
||||||
} else if (flattened) {
|
} else if (flattened) {
|
||||||
expr = new FunctionExpr(functionExpr.name, flattening);
|
expr = new FunctionExpr(functionExpr.function, functionExpr.name, flattening);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return expr;
|
return expr;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<String> findRequiredBindings(String in)
|
|
||||||
{
|
|
||||||
return findRequiredBindings(parse(in));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<String> findRequiredBindings(Expr expr)
|
public static List<String> findRequiredBindings(Expr expr)
|
||||||
{
|
{
|
||||||
final Set<String> found = Sets.newLinkedHashSet();
|
final Set<String> found = Sets.newLinkedHashSet();
|
||||||
|
|
|
@ -43,7 +43,7 @@ public class EvalTest
|
||||||
|
|
||||||
private ExprEval eval(String x, Expr.ObjectBinding bindings)
|
private ExprEval eval(String x, Expr.ObjectBinding bindings)
|
||||||
{
|
{
|
||||||
return Parser.parse(x).eval(bindings);
|
return Parser.parse(x, ExprMacroTable.nil()).eval(bindings);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -150,27 +150,27 @@ public class EvalTest
|
||||||
Expr.ObjectBinding bindings = Parser.withMap(
|
Expr.ObjectBinding bindings = Parser.withMap(
|
||||||
ImmutableMap.of("x", 100L, "y", 100L, "z", 100D, "w", 100D)
|
ImmutableMap.of("x", 100L, "y", 100L, "z", 100D, "w", 100D)
|
||||||
);
|
);
|
||||||
ExprEval eval = Parser.parse("x==y").eval(bindings);
|
ExprEval eval = Parser.parse("x==y", ExprMacroTable.nil()).eval(bindings);
|
||||||
Assert.assertTrue(eval.asBoolean());
|
Assert.assertTrue(eval.asBoolean());
|
||||||
Assert.assertEquals(ExprType.LONG, eval.type());
|
Assert.assertEquals(ExprType.LONG, eval.type());
|
||||||
|
|
||||||
eval = Parser.parse("x!=y").eval(bindings);
|
eval = Parser.parse("x!=y", ExprMacroTable.nil()).eval(bindings);
|
||||||
Assert.assertFalse(eval.asBoolean());
|
Assert.assertFalse(eval.asBoolean());
|
||||||
Assert.assertEquals(ExprType.LONG, eval.type());
|
Assert.assertEquals(ExprType.LONG, eval.type());
|
||||||
|
|
||||||
eval = Parser.parse("x==z").eval(bindings);
|
eval = Parser.parse("x==z", ExprMacroTable.nil()).eval(bindings);
|
||||||
Assert.assertTrue(eval.asBoolean());
|
Assert.assertTrue(eval.asBoolean());
|
||||||
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
||||||
|
|
||||||
eval = Parser.parse("x!=z").eval(bindings);
|
eval = Parser.parse("x!=z", ExprMacroTable.nil()).eval(bindings);
|
||||||
Assert.assertFalse(eval.asBoolean());
|
Assert.assertFalse(eval.asBoolean());
|
||||||
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
||||||
|
|
||||||
eval = Parser.parse("z==w").eval(bindings);
|
eval = Parser.parse("z==w", ExprMacroTable.nil()).eval(bindings);
|
||||||
Assert.assertTrue(eval.asBoolean());
|
Assert.assertTrue(eval.asBoolean());
|
||||||
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
||||||
|
|
||||||
eval = Parser.parse("z!=w").eval(bindings);
|
eval = Parser.parse("z!=w", ExprMacroTable.nil()).eval(bindings);
|
||||||
Assert.assertFalse(eval.asBoolean());
|
Assert.assertFalse(eval.asBoolean());
|
||||||
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
Assert.assertEquals(ExprType.DOUBLE, eval.type());
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,7 +33,7 @@ public class ParserTest
|
||||||
@Test
|
@Test
|
||||||
public void testSimple()
|
public void testSimple()
|
||||||
{
|
{
|
||||||
String actual = Parser.parse("1").toString();
|
String actual = Parser.parse("1", ExprMacroTable.nil()).toString();
|
||||||
String expected = "1";
|
String expected = "1";
|
||||||
Assert.assertEquals(expected, actual);
|
Assert.assertEquals(expected, actual);
|
||||||
}
|
}
|
||||||
|
@ -41,11 +41,11 @@ public class ParserTest
|
||||||
@Test
|
@Test
|
||||||
public void testSimpleUnaryOps1()
|
public void testSimpleUnaryOps1()
|
||||||
{
|
{
|
||||||
String actual = Parser.parse("-x").toString();
|
String actual = Parser.parse("-x", ExprMacroTable.nil()).toString();
|
||||||
String expected = "-x";
|
String expected = "-x";
|
||||||
Assert.assertEquals(expected, actual);
|
Assert.assertEquals(expected, actual);
|
||||||
|
|
||||||
actual = Parser.parse("!x").toString();
|
actual = Parser.parse("!x", ExprMacroTable.nil()).toString();
|
||||||
expected = "!x";
|
expected = "!x";
|
||||||
Assert.assertEquals(expected, actual);
|
Assert.assertEquals(expected, actual);
|
||||||
}
|
}
|
||||||
|
@ -188,14 +188,15 @@ public class ParserTest
|
||||||
|
|
||||||
private void validateFlatten(String expression, String withoutFlatten, String withFlatten)
|
private void validateFlatten(String expression, String withoutFlatten, String withFlatten)
|
||||||
{
|
{
|
||||||
Assert.assertEquals(expression, withoutFlatten, Parser.parse(expression, false).toString());
|
Assert.assertEquals(expression, withoutFlatten, Parser.parse(expression, ExprMacroTable.nil(), false).toString());
|
||||||
Assert.assertEquals(expression, withFlatten, Parser.parse(expression, true).toString());
|
Assert.assertEquals(expression, withFlatten, Parser.parse(expression, ExprMacroTable.nil(), true).toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void validateParser(String expression, String expected, List<String> identifiers)
|
private void validateParser(String expression, String expected, List<String> identifiers)
|
||||||
{
|
{
|
||||||
Assert.assertEquals(expression, expected, Parser.parse(expression).toString());
|
final Expr parsed = Parser.parse(expression, ExprMacroTable.nil());
|
||||||
Assert.assertEquals(expression, identifiers, Parser.findRequiredBindings(expression));
|
Assert.assertEquals(expression, expected, parsed.toString());
|
||||||
|
Assert.assertEquals(expression, identifiers, Parser.findRequiredBindings(parsed));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void validateConstantExpression(String expression, Object expected)
|
private void validateConstantExpression(String expression, Object expected)
|
||||||
|
@ -203,7 +204,7 @@ public class ParserTest
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
expression,
|
expression,
|
||||||
expected,
|
expected,
|
||||||
Parser.parse(expression).eval(Parser.withMap(ImmutableMap.<String, Object>of())).value()
|
Parser.parse(expression, ExprMacroTable.nil()).eval(Parser.withMap(ImmutableMap.of())).value()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,6 +26,7 @@ Also, the following built-in functions are supported.
|
||||||
|cast|cast(expr,'LONG' or 'DOUBLE' or 'STRING') returns expr with specified type. exception can be thrown |
|
|cast|cast(expr,'LONG' or 'DOUBLE' or 'STRING') returns expr with specified type. exception can be thrown |
|
||||||
|if|if(predicate,then,else) returns 'then' if 'predicate' evaluates to a positive number, otherwise it returns 'else' |
|
|if|if(predicate,then,else) returns 'then' if 'predicate' evaluates to a positive number, otherwise it returns 'else' |
|
||||||
|nvl|nvl(expr,expr-for-null) returns 'expr-for-null' if 'expr' is null (or empty string for string type) |
|
|nvl|nvl(expr,expr-for-null) returns 'expr-for-null' if 'expr' is null (or empty string for string type) |
|
||||||
|
|like|like(expr, pattern[, escape]) is equivalent to SQL `expr LIKE pattern`|
|
||||||
|timestamp|timestamp(expr[,format-string]) parses string expr into date then returns milli-seconds from java epoch. without 'format-string' it's regarded as ISO datetime format |
|
|timestamp|timestamp(expr[,format-string]) parses string expr into date then returns milli-seconds from java epoch. without 'format-string' it's regarded as ISO datetime format |
|
||||||
|unix_timestamp|same with 'timestamp' function but returns seconds instead |
|
|unix_timestamp|same with 'timestamp' function but returns seconds instead |
|
||||||
|
|
||||||
|
|
|
@ -64,6 +64,11 @@
|
||||||
<artifactId>junit</artifactId>
|
<artifactId>junit</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.easymock</groupId>
|
||||||
|
<artifactId>easymock</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -74,7 +74,6 @@ public class QuantileSqlAggregator implements SqlAggregator
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(
|
final RowExtraction rex = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowSignature.getRowOrder(),
|
rowSignature.getRowOrder(),
|
||||||
Expressions.fromFieldAccess(
|
Expressions.fromFieldAccess(
|
||||||
|
|
|
@ -136,7 +136,14 @@ public class QuantileSqlAggregatorTest
|
||||||
ImmutableSet.<SqlAggregator>of(new QuantileSqlAggregator()),
|
ImmutableSet.<SqlAggregator>of(new QuantileSqlAggregator()),
|
||||||
ImmutableSet.<SqlExtractionOperator>of()
|
ImmutableSet.<SqlExtractionOperator>of()
|
||||||
);
|
);
|
||||||
plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig, new ServerConfig());
|
plannerFactory = new PlannerFactory(
|
||||||
|
rootSchema,
|
||||||
|
walker,
|
||||||
|
operatorTable,
|
||||||
|
CalciteTests.createExprMacroTable(),
|
||||||
|
plannerConfig,
|
||||||
|
new ServerConfig()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
|
@ -74,6 +74,13 @@
|
||||||
<version>${apache.kafka.version}</version>
|
<version>${apache.kafka.version}</version>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>io.druid</groupId>
|
||||||
|
<artifactId>druid-processing</artifactId>
|
||||||
|
<version>${project.parent.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>io.druid</groupId>
|
<groupId>io.druid</groupId>
|
||||||
<artifactId>druid-server</artifactId>
|
<artifactId>druid-server</artifactId>
|
||||||
|
|
|
@ -102,6 +102,7 @@ import io.druid.query.timeseries.TimeseriesQueryRunnerFactory;
|
||||||
import io.druid.query.timeseries.TimeseriesResultValue;
|
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||||
import io.druid.segment.IndexIO;
|
import io.druid.segment.IndexIO;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -149,7 +150,7 @@ import java.util.concurrent.TimeoutException;
|
||||||
public class KafkaIndexTaskTest
|
public class KafkaIndexTaskTest
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(KafkaIndexTaskTest.class);
|
private static final Logger log = new Logger(KafkaIndexTaskTest.class);
|
||||||
private static final ObjectMapper objectMapper = new DefaultObjectMapper();
|
private static final ObjectMapper objectMapper = TestHelper.getJsonMapper();
|
||||||
private static final long POLL_RETRY_MS = 100;
|
private static final long POLL_RETRY_MS = 100;
|
||||||
|
|
||||||
private static TestingCluster zkServer;
|
private static TestingCluster zkServer;
|
||||||
|
|
|
@ -57,11 +57,11 @@ import io.druid.indexing.overlord.TaskRunnerListener;
|
||||||
import io.druid.indexing.overlord.TaskRunnerWorkItem;
|
import io.druid.indexing.overlord.TaskRunnerWorkItem;
|
||||||
import io.druid.indexing.overlord.TaskStorage;
|
import io.druid.indexing.overlord.TaskStorage;
|
||||||
import io.druid.indexing.overlord.supervisor.SupervisorReport;
|
import io.druid.indexing.overlord.supervisor.SupervisorReport;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.RealtimeIOConfig;
|
import io.druid.segment.indexing.RealtimeIOConfig;
|
||||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -110,7 +110,7 @@ import static org.easymock.EasyMock.reset;
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class KafkaSupervisorTest extends EasyMockSupport
|
public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
{
|
{
|
||||||
private static final ObjectMapper objectMapper = new DefaultObjectMapper();
|
private static final ObjectMapper objectMapper = TestHelper.getJsonMapper();
|
||||||
private static final String TOPIC_PREFIX = "testTopic";
|
private static final String TOPIC_PREFIX = "testTopic";
|
||||||
private static final String DATASOURCE = "testDS";
|
private static final String DATASOURCE = "testDS";
|
||||||
private static final int NUM_PARTITIONS = 3;
|
private static final int NUM_PARTITIONS = 3;
|
||||||
|
|
|
@ -104,7 +104,7 @@ public class LoadingLookupFactoryTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerDeser() throws IOException
|
public void testSerDeser() throws IOException
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
LoadingLookupFactory loadingLookupFactory = new LoadingLookupFactory(
|
LoadingLookupFactory loadingLookupFactory = new LoadingLookupFactory(
|
||||||
new MockDataFetcher(),
|
new MockDataFetcher(),
|
||||||
new OnHeapLoadingCache<String, String>(
|
new OnHeapLoadingCache<String, String>(
|
||||||
|
|
|
@ -35,7 +35,7 @@ import java.util.List;
|
||||||
*/
|
*/
|
||||||
public class DatasourceIngestionSpecTest
|
public class DatasourceIngestionSpecTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper MAPPER = TestHelper.getObjectMapper();
|
private static final ObjectMapper MAPPER = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSingleIntervalSerde() throws Exception
|
public void testSingleIntervalSerde() throws Exception
|
||||||
|
|
|
@ -67,6 +67,13 @@
|
||||||
<artifactId>hadoop-client</artifactId>
|
<artifactId>hadoop-client</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>io.druid</groupId>
|
||||||
|
<artifactId>druid-processing</artifactId>
|
||||||
|
<version>${project.parent.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>io.druid</groupId>
|
<groupId>io.druid</groupId>
|
||||||
<artifactId>druid-server</artifactId>
|
<artifactId>druid-server</artifactId>
|
||||||
|
|
|
@ -23,10 +23,11 @@ import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.Module;
|
import com.fasterxml.jackson.databind.Module;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Stopwatch;
|
import com.google.common.base.Stopwatch;
|
||||||
|
|
||||||
import io.druid.guice.ServerModule;
|
import io.druid.guice.ServerModule;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.segment.IndexIO;
|
import io.druid.segment.IndexIO;
|
||||||
import io.druid.segment.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
import io.druid.segment.IndexMergerV9;
|
import io.druid.segment.IndexMergerV9;
|
||||||
|
@ -70,6 +71,7 @@ public class TestUtils
|
||||||
|
|
||||||
jsonMapper.setInjectableValues(
|
jsonMapper.setInjectableValues(
|
||||||
new InjectableValues.Std()
|
new InjectableValues.Std()
|
||||||
|
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||||
.addValue(IndexIO.class, indexIO)
|
.addValue(IndexIO.class, indexIO)
|
||||||
.addValue(IndexMerger.class, indexMerger)
|
.addValue(IndexMerger.class, indexMerger)
|
||||||
.addValue(ObjectMapper.class, jsonMapper)
|
.addValue(ObjectMapper.class, jsonMapper)
|
||||||
|
@ -87,7 +89,8 @@ public class TestUtils
|
||||||
return indexMerger;
|
return indexMerger;
|
||||||
}
|
}
|
||||||
|
|
||||||
public IndexMergerV9 getTestIndexMergerV9() {
|
public IndexMergerV9 getTestIndexMergerV9()
|
||||||
|
{
|
||||||
return indexMergerV9;
|
return indexMergerV9;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.query.aggregation;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.java.util.common.Pair;
|
import io.druid.java.util.common.Pair;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
@ -92,6 +93,7 @@ public class AggregatorUtil
|
||||||
|
|
||||||
public static FloatColumnSelector getFloatColumnSelector(
|
public static FloatColumnSelector getFloatColumnSelector(
|
||||||
final ColumnSelectorFactory metricFactory,
|
final ColumnSelectorFactory metricFactory,
|
||||||
|
final ExprMacroTable macroTable,
|
||||||
final String fieldName,
|
final String fieldName,
|
||||||
final String fieldExpression,
|
final String fieldExpression,
|
||||||
final float nullValue
|
final float nullValue
|
||||||
|
@ -101,13 +103,18 @@ public class AggregatorUtil
|
||||||
return metricFactory.makeFloatColumnSelector(fieldName);
|
return metricFactory.makeFloatColumnSelector(fieldName);
|
||||||
}
|
}
|
||||||
if (fieldName == null && fieldExpression != null) {
|
if (fieldName == null && fieldExpression != null) {
|
||||||
return ExpressionSelectors.makeFloatColumnSelector(metricFactory, Parser.parse(fieldExpression), nullValue);
|
return ExpressionSelectors.makeFloatColumnSelector(
|
||||||
|
metricFactory,
|
||||||
|
Parser.parse(fieldExpression, macroTable),
|
||||||
|
nullValue
|
||||||
|
);
|
||||||
}
|
}
|
||||||
throw new IllegalArgumentException("Must have a valid, non-null fieldName or expression");
|
throw new IllegalArgumentException("Must have a valid, non-null fieldName or expression");
|
||||||
}
|
}
|
||||||
|
|
||||||
public static LongColumnSelector getLongColumnSelector(
|
public static LongColumnSelector getLongColumnSelector(
|
||||||
final ColumnSelectorFactory metricFactory,
|
final ColumnSelectorFactory metricFactory,
|
||||||
|
final ExprMacroTable macroTable,
|
||||||
final String fieldName,
|
final String fieldName,
|
||||||
final String fieldExpression,
|
final String fieldExpression,
|
||||||
final long nullValue
|
final long nullValue
|
||||||
|
@ -117,7 +124,11 @@ public class AggregatorUtil
|
||||||
return metricFactory.makeLongColumnSelector(fieldName);
|
return metricFactory.makeLongColumnSelector(fieldName);
|
||||||
}
|
}
|
||||||
if (fieldName == null && fieldExpression != null) {
|
if (fieldName == null && fieldExpression != null) {
|
||||||
return ExpressionSelectors.makeLongColumnSelector(metricFactory, Parser.parse(fieldExpression), nullValue);
|
return ExpressionSelectors.makeLongColumnSelector(
|
||||||
|
metricFactory,
|
||||||
|
Parser.parse(fieldExpression, macroTable),
|
||||||
|
nullValue
|
||||||
|
);
|
||||||
}
|
}
|
||||||
throw new IllegalArgumentException("Must have a valid, non-null fieldName or expression");
|
throw new IllegalArgumentException("Must have a valid, non-null fieldName or expression");
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,17 +19,18 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -44,12 +45,14 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
|
||||||
private final String name;
|
private final String name;
|
||||||
private final String fieldName;
|
private final String fieldName;
|
||||||
private final String expression;
|
private final String expression;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public DoubleMaxAggregatorFactory(
|
public DoubleMaxAggregatorFactory(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("fieldName") final String fieldName,
|
@JsonProperty("fieldName") final String fieldName,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
@ -61,11 +64,12 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName;
|
this.fieldName = fieldName;
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
|
this.macroTable = macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public DoubleMaxAggregatorFactory(String name, String fieldName)
|
public DoubleMaxAggregatorFactory(String name, String fieldName)
|
||||||
{
|
{
|
||||||
this(name, fieldName, null);
|
this(name, fieldName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -82,7 +86,7 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
|
||||||
|
|
||||||
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
|
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
|
||||||
{
|
{
|
||||||
return AggregatorUtil.getFloatColumnSelector(metricFactory, fieldName, expression, Float.MIN_VALUE);
|
return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, Float.MIN_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -100,7 +104,7 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public AggregatorFactory getCombiningFactory()
|
public AggregatorFactory getCombiningFactory()
|
||||||
{
|
{
|
||||||
return new DoubleMaxAggregatorFactory(name, name, null);
|
return new DoubleMaxAggregatorFactory(name, name, null, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,7 +120,7 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Arrays.<AggregatorFactory>asList(new DoubleMaxAggregatorFactory(fieldName, fieldName, expression));
|
return Collections.singletonList(new DoubleMaxAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -157,7 +161,9 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredFields()
|
public List<String> requiredFields()
|
||||||
{
|
{
|
||||||
return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
|
return fieldName != null
|
||||||
|
? Collections.singletonList(fieldName)
|
||||||
|
: Parser.findRequiredBindings(Parser.parse(expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,11 +19,13 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
@ -44,12 +46,14 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
|
||||||
private final String name;
|
private final String name;
|
||||||
private final String fieldName;
|
private final String fieldName;
|
||||||
private final String expression;
|
private final String expression;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public DoubleMinAggregatorFactory(
|
public DoubleMinAggregatorFactory(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("fieldName") final String fieldName,
|
@JsonProperty("fieldName") final String fieldName,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
@ -61,11 +65,12 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName;
|
this.fieldName = fieldName;
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
|
this.macroTable = macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public DoubleMinAggregatorFactory(String name, String fieldName)
|
public DoubleMinAggregatorFactory(String name, String fieldName)
|
||||||
{
|
{
|
||||||
this(name, fieldName, null);
|
this(name, fieldName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -82,7 +87,7 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
|
||||||
|
|
||||||
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
|
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
|
||||||
{
|
{
|
||||||
return AggregatorUtil.getFloatColumnSelector(metricFactory, fieldName, expression, Float.MAX_VALUE);
|
return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, Float.MAX_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -100,7 +105,7 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public AggregatorFactory getCombiningFactory()
|
public AggregatorFactory getCombiningFactory()
|
||||||
{
|
{
|
||||||
return new DoubleMinAggregatorFactory(name, name, null);
|
return new DoubleMinAggregatorFactory(name, name, null, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,7 +121,12 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Arrays.<AggregatorFactory>asList(new DoubleMinAggregatorFactory(fieldName, fieldName, expression));
|
return Arrays.<AggregatorFactory>asList(new DoubleMinAggregatorFactory(
|
||||||
|
fieldName,
|
||||||
|
fieldName,
|
||||||
|
expression,
|
||||||
|
macroTable
|
||||||
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -157,7 +167,9 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredFields()
|
public List<String> requiredFields()
|
||||||
{
|
{
|
||||||
return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
|
return fieldName != null
|
||||||
|
? Collections.singletonList(fieldName)
|
||||||
|
: Parser.findRequiredBindings(Parser.parse(expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,11 +19,13 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
@ -44,12 +46,14 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
|
||||||
private final String name;
|
private final String name;
|
||||||
private final String fieldName;
|
private final String fieldName;
|
||||||
private final String expression;
|
private final String expression;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public DoubleSumAggregatorFactory(
|
public DoubleSumAggregatorFactory(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("fieldName") String fieldName,
|
@JsonProperty("fieldName") String fieldName,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
@ -61,11 +65,12 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName;
|
this.fieldName = fieldName;
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
|
this.macroTable = macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public DoubleSumAggregatorFactory(String name, String fieldName)
|
public DoubleSumAggregatorFactory(String name, String fieldName)
|
||||||
{
|
{
|
||||||
this(name, fieldName, null);
|
this(name, fieldName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -82,7 +87,7 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
|
||||||
|
|
||||||
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
|
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
|
||||||
{
|
{
|
||||||
return AggregatorUtil.getFloatColumnSelector(metricFactory, fieldName, expression, 0f);
|
return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, 0f);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -100,7 +105,7 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public AggregatorFactory getCombiningFactory()
|
public AggregatorFactory getCombiningFactory()
|
||||||
{
|
{
|
||||||
return new DoubleSumAggregatorFactory(name, name, null);
|
return new DoubleSumAggregatorFactory(name, name, null, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,7 +121,7 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Arrays.<AggregatorFactory>asList(new DoubleSumAggregatorFactory(fieldName, fieldName, expression));
|
return Arrays.asList(new DoubleSumAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -157,7 +162,9 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredFields()
|
public List<String> requiredFields()
|
||||||
{
|
{
|
||||||
return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
|
return fieldName != null
|
||||||
|
? Collections.singletonList(fieldName)
|
||||||
|
: Parser.findRequiredBindings(Parser.parse(expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,11 +19,13 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
@ -44,12 +46,14 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
|
||||||
private final String name;
|
private final String name;
|
||||||
private final String fieldName;
|
private final String fieldName;
|
||||||
private final String expression;
|
private final String expression;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public LongMaxAggregatorFactory(
|
public LongMaxAggregatorFactory(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("fieldName") final String fieldName,
|
@JsonProperty("fieldName") final String fieldName,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
@ -61,11 +65,12 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName;
|
this.fieldName = fieldName;
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
|
this.macroTable = macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public LongMaxAggregatorFactory(String name, String fieldName)
|
public LongMaxAggregatorFactory(String name, String fieldName)
|
||||||
{
|
{
|
||||||
this(name, fieldName, null);
|
this(name, fieldName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -82,7 +87,7 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
|
||||||
|
|
||||||
private LongColumnSelector getLongColumnSelector(ColumnSelectorFactory metricFactory)
|
private LongColumnSelector getLongColumnSelector(ColumnSelectorFactory metricFactory)
|
||||||
{
|
{
|
||||||
return AggregatorUtil.getLongColumnSelector(metricFactory, fieldName, expression, Long.MIN_VALUE);
|
return AggregatorUtil.getLongColumnSelector(metricFactory, macroTable, fieldName, expression, Long.MIN_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -100,7 +105,7 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public AggregatorFactory getCombiningFactory()
|
public AggregatorFactory getCombiningFactory()
|
||||||
{
|
{
|
||||||
return new LongMaxAggregatorFactory(name, name, null);
|
return new LongMaxAggregatorFactory(name, name, null, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,7 +121,7 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Arrays.<AggregatorFactory>asList(new LongMaxAggregatorFactory(fieldName, fieldName, expression));
|
return Arrays.<AggregatorFactory>asList(new LongMaxAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -153,7 +158,9 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredFields()
|
public List<String> requiredFields()
|
||||||
{
|
{
|
||||||
return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
|
return fieldName != null
|
||||||
|
? Collections.singletonList(fieldName)
|
||||||
|
: Parser.findRequiredBindings(Parser.parse(expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,11 +19,13 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
@ -44,12 +46,14 @@ public class LongMinAggregatorFactory extends AggregatorFactory
|
||||||
private final String name;
|
private final String name;
|
||||||
private final String fieldName;
|
private final String fieldName;
|
||||||
private final String expression;
|
private final String expression;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public LongMinAggregatorFactory(
|
public LongMinAggregatorFactory(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("fieldName") final String fieldName,
|
@JsonProperty("fieldName") final String fieldName,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
@ -61,11 +65,12 @@ public class LongMinAggregatorFactory extends AggregatorFactory
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName;
|
this.fieldName = fieldName;
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
|
this.macroTable = macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public LongMinAggregatorFactory(String name, String fieldName)
|
public LongMinAggregatorFactory(String name, String fieldName)
|
||||||
{
|
{
|
||||||
this(name, fieldName, null);
|
this(name, fieldName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -82,7 +87,7 @@ public class LongMinAggregatorFactory extends AggregatorFactory
|
||||||
|
|
||||||
private LongColumnSelector getLongColumnSelector(ColumnSelectorFactory metricFactory)
|
private LongColumnSelector getLongColumnSelector(ColumnSelectorFactory metricFactory)
|
||||||
{
|
{
|
||||||
return AggregatorUtil.getLongColumnSelector(metricFactory, fieldName, expression, Long.MAX_VALUE);
|
return AggregatorUtil.getLongColumnSelector(metricFactory, macroTable, fieldName, expression, Long.MAX_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -100,7 +105,7 @@ public class LongMinAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public AggregatorFactory getCombiningFactory()
|
public AggregatorFactory getCombiningFactory()
|
||||||
{
|
{
|
||||||
return new LongMinAggregatorFactory(name, name, null);
|
return new LongMinAggregatorFactory(name, name, null, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,7 +121,7 @@ public class LongMinAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Arrays.<AggregatorFactory>asList(new LongMinAggregatorFactory(fieldName, fieldName, expression));
|
return Arrays.<AggregatorFactory>asList(new LongMinAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -153,7 +158,9 @@ public class LongMinAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredFields()
|
public List<String> requiredFields()
|
||||||
{
|
{
|
||||||
return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
|
return fieldName != null
|
||||||
|
? Collections.singletonList(fieldName)
|
||||||
|
: Parser.findRequiredBindings(Parser.parse(expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,11 +19,13 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
@ -44,12 +46,14 @@ public class LongSumAggregatorFactory extends AggregatorFactory
|
||||||
private final String name;
|
private final String name;
|
||||||
private final String fieldName;
|
private final String fieldName;
|
||||||
private final String expression;
|
private final String expression;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public LongSumAggregatorFactory(
|
public LongSumAggregatorFactory(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("fieldName") String fieldName,
|
@JsonProperty("fieldName") String fieldName,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
@ -61,11 +65,12 @@ public class LongSumAggregatorFactory extends AggregatorFactory
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName;
|
this.fieldName = fieldName;
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
|
this.macroTable = macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public LongSumAggregatorFactory(String name, String fieldName)
|
public LongSumAggregatorFactory(String name, String fieldName)
|
||||||
{
|
{
|
||||||
this(name, fieldName, null);
|
this(name, fieldName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -82,7 +87,7 @@ public class LongSumAggregatorFactory extends AggregatorFactory
|
||||||
|
|
||||||
private LongColumnSelector getLongColumnSelector(ColumnSelectorFactory metricFactory)
|
private LongColumnSelector getLongColumnSelector(ColumnSelectorFactory metricFactory)
|
||||||
{
|
{
|
||||||
return AggregatorUtil.getLongColumnSelector(metricFactory, fieldName, expression, 0L);
|
return AggregatorUtil.getLongColumnSelector(metricFactory, macroTable, fieldName, expression, 0L);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -100,7 +105,7 @@ public class LongSumAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public AggregatorFactory getCombiningFactory()
|
public AggregatorFactory getCombiningFactory()
|
||||||
{
|
{
|
||||||
return new LongSumAggregatorFactory(name, name, null);
|
return new LongSumAggregatorFactory(name, name, null, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,7 +121,7 @@ public class LongSumAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Arrays.<AggregatorFactory>asList(new LongSumAggregatorFactory(fieldName, fieldName, expression));
|
return Arrays.<AggregatorFactory>asList(new LongSumAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -153,7 +158,9 @@ public class LongSumAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredFields()
|
public List<String> requiredFields()
|
||||||
{
|
{
|
||||||
return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
|
return fieldName != null
|
||||||
|
? Collections.singletonList(fieldName)
|
||||||
|
: Parser.findRequiredBindings(Parser.parse(expression, macroTable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,18 +19,19 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation.post;
|
package io.druid.query.aggregation.post;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import io.druid.math.expr.Expr;
|
import io.druid.math.expr.Expr;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.PostAggregator;
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
import io.druid.query.cache.CacheKeyBuilder;
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
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.Set;
|
||||||
|
@ -55,15 +56,20 @@ public class ExpressionPostAggregator implements PostAggregator
|
||||||
private final String expression;
|
private final String expression;
|
||||||
private final Comparator comparator;
|
private final Comparator comparator;
|
||||||
private final String ordering;
|
private final String ordering;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
|
|
||||||
private final Expr parsed;
|
private final Expr parsed;
|
||||||
private final List<String> dependentFields;
|
private final Set<String> dependentFields;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor for serialization.
|
||||||
|
*/
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public ExpressionPostAggregator(
|
public ExpressionPostAggregator(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("expression") String expression,
|
@JsonProperty("expression") String expression,
|
||||||
@JsonProperty("ordering") String ordering
|
@JsonProperty("ordering") String ordering,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(expression != null, "expression cannot be null");
|
Preconditions.checkArgument(expression != null, "expression cannot be null");
|
||||||
|
@ -72,20 +78,20 @@ public class ExpressionPostAggregator implements PostAggregator
|
||||||
this.expression = expression;
|
this.expression = expression;
|
||||||
this.ordering = ordering;
|
this.ordering = ordering;
|
||||||
this.comparator = ordering == null ? DEFAULT_COMPARATOR : Ordering.valueOf(ordering);
|
this.comparator = ordering == null ? DEFAULT_COMPARATOR : Ordering.valueOf(ordering);
|
||||||
|
this.macroTable = macroTable;
|
||||||
this.parsed = Parser.parse(expression);
|
this.parsed = Parser.parse(expression, macroTable);
|
||||||
this.dependentFields = Parser.findRequiredBindings(parsed);
|
this.dependentFields = ImmutableSet.copyOf(Parser.findRequiredBindings(parsed));
|
||||||
}
|
}
|
||||||
|
|
||||||
public ExpressionPostAggregator(String name, String fnName)
|
public ExpressionPostAggregator(String name, String fnName)
|
||||||
{
|
{
|
||||||
this(name, fnName, null);
|
this(name, fnName, null, ExprMacroTable.nil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<String> getDependentFields()
|
public Set<String> getDependentFields()
|
||||||
{
|
{
|
||||||
return Sets.newHashSet(dependentFields);
|
return dependentFields;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,35 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.expression;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
|
|
||||||
|
import javax.inject.Inject;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class GuiceExprMacroTable extends ExprMacroTable
|
||||||
|
{
|
||||||
|
@Inject
|
||||||
|
public GuiceExprMacroTable(final Set<ExprMacro> macros)
|
||||||
|
{
|
||||||
|
super(ImmutableList.copyOf(macros));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.expression;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.math.expr.Expr;
|
||||||
|
import io.druid.math.expr.ExprEval;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
|
import io.druid.math.expr.ExprType;
|
||||||
|
import io.druid.query.filter.LikeDimFilter;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class LikeExprMacro implements ExprMacroTable.ExprMacro
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String name()
|
||||||
|
{
|
||||||
|
return "like";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Expr apply(final List<Expr> args)
|
||||||
|
{
|
||||||
|
if (args.size() < 2 || args.size() > 3) {
|
||||||
|
throw new IAE("Function[%s] must have 2 or 3 arguments", name());
|
||||||
|
}
|
||||||
|
|
||||||
|
final Expr arg = args.get(0);
|
||||||
|
final Expr patternExpr = args.get(1);
|
||||||
|
final Expr escapeExpr = args.size() > 2 ? args.get(2) : null;
|
||||||
|
|
||||||
|
if (!patternExpr.isLiteral() || (escapeExpr != null && !escapeExpr.isLiteral())) {
|
||||||
|
throw new IAE("pattern and escape must be literals");
|
||||||
|
}
|
||||||
|
|
||||||
|
final String escape = escapeExpr == null ? null : (String) escapeExpr.getLiteralValue();
|
||||||
|
final Character escapeChar;
|
||||||
|
|
||||||
|
if (escape != null && escape.length() != 1) {
|
||||||
|
throw new IllegalArgumentException("Escape must be null or a single character");
|
||||||
|
} else {
|
||||||
|
escapeChar = escape == null ? null : escape.charAt(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
final LikeDimFilter.LikeMatcher likeMatcher = LikeDimFilter.LikeMatcher.from(
|
||||||
|
(String) patternExpr.getLiteralValue(),
|
||||||
|
escapeChar
|
||||||
|
);
|
||||||
|
|
||||||
|
class LikeExtractExpr implements Expr
|
||||||
|
{
|
||||||
|
@Nonnull
|
||||||
|
@Override
|
||||||
|
public ExprEval eval(final ObjectBinding bindings)
|
||||||
|
{
|
||||||
|
return ExprEval.of(likeMatcher.matches(arg.eval(bindings).asString()), ExprType.LONG);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void visit(final Visitor visitor)
|
||||||
|
{
|
||||||
|
arg.visit(visitor);
|
||||||
|
visitor.visit(this);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new LikeExtractExpr();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -19,13 +19,14 @@
|
||||||
|
|
||||||
package io.druid.segment.virtual;
|
package io.druid.segment.virtual;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Ints;
|
|
||||||
import io.druid.java.util.common.StringUtils;
|
|
||||||
import io.druid.math.expr.Expr;
|
import io.druid.math.expr.Expr;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.math.expr.Parser;
|
import io.druid.math.expr.Parser;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
@ -37,8 +38,8 @@ import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
public class ExpressionVirtualColumn implements VirtualColumn
|
public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
{
|
{
|
||||||
|
@ -51,12 +52,13 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public ExpressionVirtualColumn(
|
public ExpressionVirtualColumn(
|
||||||
@JsonProperty("name") String name,
|
@JsonProperty("name") String name,
|
||||||
@JsonProperty("expression") String expression
|
@JsonProperty("expression") String expression,
|
||||||
|
@JacksonInject ExprMacroTable macroTable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = Preconditions.checkNotNull(name, "name");
|
this.name = Preconditions.checkNotNull(name, "name");
|
||||||
this.expression = Preconditions.checkNotNull(expression, "expression");
|
this.expression = Preconditions.checkNotNull(expression, "expression");
|
||||||
this.parsedExpression = Parser.parse(expression);
|
this.parsedExpression = Parser.parse(expression, macroTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("name")
|
@JsonProperty("name")
|
||||||
|
@ -78,7 +80,24 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
final ColumnSelectorFactory columnSelectorFactory
|
final ColumnSelectorFactory columnSelectorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return ExpressionSelectors.makeObjectColumnSelector(columnSelectorFactory, parsedExpression);
|
final ExpressionObjectSelector baseSelector = ExpressionSelectors.makeObjectColumnSelector(
|
||||||
|
columnSelectorFactory,
|
||||||
|
parsedExpression
|
||||||
|
);
|
||||||
|
return new ObjectColumnSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class classOfObject()
|
||||||
|
{
|
||||||
|
return Object.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object get()
|
||||||
|
{
|
||||||
|
return baseSelector.get();
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -123,7 +142,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
@Override
|
@Override
|
||||||
public List<String> requiredColumns()
|
public List<String> requiredColumns()
|
||||||
{
|
{
|
||||||
return Parser.findRequiredBindings(expression);
|
return Parser.findRequiredBindings(parsedExpression);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -135,21 +154,14 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
@Override
|
@Override
|
||||||
public byte[] getCacheKey()
|
public byte[] getCacheKey()
|
||||||
{
|
{
|
||||||
final byte[] nameBytes = StringUtils.toUtf8(name);
|
return new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_EXPRESSION)
|
||||||
final byte[] expressionBytes = StringUtils.toUtf8(expression);
|
.appendString(name)
|
||||||
|
.appendString(expression)
|
||||||
return ByteBuffer
|
.build();
|
||||||
.allocate(1 + Ints.BYTES * 2 + nameBytes.length + expressionBytes.length)
|
|
||||||
.put(VirtualColumnCacheHelper.CACHE_TYPE_ID_EXPRESSION)
|
|
||||||
.putInt(nameBytes.length)
|
|
||||||
.put(nameBytes)
|
|
||||||
.putInt(expressionBytes.length)
|
|
||||||
.put(expressionBytes)
|
|
||||||
.array();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o)
|
public boolean equals(final Object o)
|
||||||
{
|
{
|
||||||
if (this == o) {
|
if (this == o) {
|
||||||
return true;
|
return true;
|
||||||
|
@ -157,21 +169,15 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
if (o == null || getClass() != o.getClass()) {
|
if (o == null || getClass() != o.getClass()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
final ExpressionVirtualColumn that = (ExpressionVirtualColumn) o;
|
||||||
ExpressionVirtualColumn that = (ExpressionVirtualColumn) o;
|
return Objects.equals(name, that.name) &&
|
||||||
|
Objects.equals(expression, that.expression);
|
||||||
if (!name.equals(that.name)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return expression.equals(that.expression);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = name.hashCode();
|
return Objects.hash(name, expression);
|
||||||
result = 31 * result + expression.hashCode();
|
|
||||||
return result;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -180,6 +186,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||||
return "ExpressionVirtualColumn{" +
|
return "ExpressionVirtualColumn{" +
|
||||||
"name='" + name + '\'' +
|
"name='" + name + '\'' +
|
||||||
", expression='" + expression + '\'' +
|
", expression='" + expression + '\'' +
|
||||||
|
", parsedExpression=" + parsedExpression +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,7 +33,7 @@ public class ExtensionsConfigTest
|
||||||
public void testSerdeWithDefaults() throws Exception
|
public void testSerdeWithDefaults() throws Exception
|
||||||
{
|
{
|
||||||
String json = "{}";
|
String json = "{}";
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
ExtensionsConfig config = mapper.readValue(
|
ExtensionsConfig config = mapper.readValue(
|
||||||
mapper.writeValueAsString(
|
mapper.writeValueAsString(
|
||||||
|
@ -59,7 +59,7 @@ public class ExtensionsConfigTest
|
||||||
+ " \"hadoopContainerDruidClasspath\": \"testHadoopContainerClasspath\",\n"
|
+ " \"hadoopContainerDruidClasspath\": \"testHadoopContainerClasspath\",\n"
|
||||||
+ " \"loadList\": [\"a\",\"b\"]\n"
|
+ " \"loadList\": [\"a\",\"b\"]\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
ExtensionsConfig config = mapper.readValue(
|
ExtensionsConfig config = mapper.readValue(
|
||||||
mapper.writeValueAsString(
|
mapper.writeValueAsString(
|
||||||
|
|
|
@ -21,12 +21,12 @@ package io.druid.query;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.groupby.GroupByQuery;
|
import io.druid.query.groupby.GroupByQuery;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -35,7 +35,7 @@ import java.util.Arrays;
|
||||||
|
|
||||||
public class DataSourceTest
|
public class DataSourceTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSerialization() throws IOException
|
public void testSerialization() throws IOException
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
@ -29,6 +28,7 @@ import io.druid.query.dimension.ListFilteredDimensionSpec;
|
||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
import io.druid.query.topn.TopNQuery;
|
import io.druid.query.topn.TopNQuery;
|
||||||
import io.druid.query.topn.TopNQueryBuilder;
|
import io.druid.query.topn.TopNQueryBuilder;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -50,7 +50,7 @@ public class DefaultQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultQueryMetrics<Query<?>> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper());
|
DefaultQueryMetrics<Query<?>> queryMetrics = new DefaultQueryMetrics<>(TestHelper.getJsonMapper());
|
||||||
TopNQuery query = new TopNQueryBuilder()
|
TopNQuery query = new TopNQueryBuilder()
|
||||||
.dataSource("xx")
|
.dataSource("xx")
|
||||||
.granularity(Granularities.ALL)
|
.granularity(Granularities.ALL)
|
||||||
|
@ -92,7 +92,7 @@ public class DefaultQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultQueryMetrics<Query<?>> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper());
|
DefaultQueryMetrics<Query<?>> queryMetrics = new DefaultQueryMetrics<>(TestHelper.getJsonMapper());
|
||||||
testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,8 +21,8 @@ package io.druid.query;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -31,7 +31,7 @@ import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
public class QueryInterruptedExceptionTest
|
public class QueryInterruptedExceptionTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper MAPPER = new DefaultObjectMapper();
|
private static final ObjectMapper MAPPER = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testErrorCode()
|
public void testErrorCode()
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.MapMaker;
|
import com.google.common.collect.MapMaker;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.guava.Sequence;
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
|
@ -32,6 +31,7 @@ import io.druid.query.timeseries.TimeseriesQuery;
|
||||||
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
|
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
|
||||||
import io.druid.query.timeseries.TimeseriesResultValue;
|
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||||
import io.druid.segment.SegmentMissingException;
|
import io.druid.segment.SegmentMissingException;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -43,7 +43,7 @@ import java.util.Map;
|
||||||
|
|
||||||
public class RetryQueryRunnerTest
|
public class RetryQueryRunnerTest
|
||||||
{
|
{
|
||||||
private final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private final ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
|
|
@ -40,6 +40,7 @@ import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.filter.BoundDimFilter;
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
import io.druid.query.timeseries.TimeseriesQuery;
|
import io.druid.query.timeseries.TimeseriesQuery;
|
||||||
|
@ -261,8 +262,8 @@ public class SchemaEvolutionTest
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new LongSumAggregatorFactory("a", "c1"),
|
new LongSumAggregatorFactory("a", "c1"),
|
||||||
new DoubleSumAggregatorFactory("b", "c1"),
|
new DoubleSumAggregatorFactory("b", "c1"),
|
||||||
new LongSumAggregatorFactory("c", null, "c1 * 1"),
|
new LongSumAggregatorFactory("c", null, "c1 * 1", TestExprMacroTable.INSTANCE),
|
||||||
new DoubleSumAggregatorFactory("d", null, "c1 * 1")
|
new DoubleSumAggregatorFactory("d", null, "c1 * 1", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
|
|
|
@ -40,7 +40,6 @@ import io.druid.collections.StupidPool;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.data.input.impl.InputRowParser;
|
import io.druid.data.input.impl.InputRowParser;
|
||||||
import io.druid.data.input.impl.StringInputRowParser;
|
import io.druid.data.input.impl.StringInputRowParser;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.IAE;
|
import io.druid.java.util.common.IAE;
|
||||||
import io.druid.java.util.common.granularity.Granularity;
|
import io.druid.java.util.common.granularity.Granularity;
|
||||||
import io.druid.java.util.common.guava.CloseQuietly;
|
import io.druid.java.util.common.guava.CloseQuietly;
|
||||||
|
@ -75,6 +74,7 @@ import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.segment.QueryableIndexSegment;
|
import io.druid.segment.QueryableIndexSegment;
|
||||||
import io.druid.segment.Segment;
|
import io.druid.segment.Segment;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.column.ColumnConfig;
|
import io.druid.segment.column.ColumnConfig;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
|
@ -137,7 +137,7 @@ public class AggregationTestHelper
|
||||||
TemporaryFolder tempFolder
|
TemporaryFolder tempFolder
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = new DefaultObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(mapper, config);
|
GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(mapper, config);
|
||||||
|
|
||||||
IndexIO indexIO = new IndexIO(
|
IndexIO indexIO = new IndexIO(
|
||||||
|
@ -168,7 +168,7 @@ public class AggregationTestHelper
|
||||||
TemporaryFolder tempFolder
|
TemporaryFolder tempFolder
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = new DefaultObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
mapper.setInjectableValues(
|
mapper.setInjectableValues(
|
||||||
new InjectableValues.Std().addValue(
|
new InjectableValues.Std().addValue(
|
||||||
SelectQueryConfig.class,
|
SelectQueryConfig.class,
|
||||||
|
@ -179,14 +179,14 @@ public class AggregationTestHelper
|
||||||
Supplier<SelectQueryConfig> configSupplier = Suppliers.ofInstance(new SelectQueryConfig(true));
|
Supplier<SelectQueryConfig> configSupplier = Suppliers.ofInstance(new SelectQueryConfig(true));
|
||||||
|
|
||||||
SelectQueryQueryToolChest toolchest = new SelectQueryQueryToolChest(
|
SelectQueryQueryToolChest toolchest = new SelectQueryQueryToolChest(
|
||||||
new DefaultObjectMapper(),
|
TestHelper.getJsonMapper(),
|
||||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
|
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
|
||||||
configSupplier
|
configSupplier
|
||||||
);
|
);
|
||||||
|
|
||||||
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
|
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
|
||||||
new SelectQueryQueryToolChest(
|
new SelectQueryQueryToolChest(
|
||||||
new DefaultObjectMapper(),
|
TestHelper.getJsonMapper(),
|
||||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
|
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
|
||||||
configSupplier
|
configSupplier
|
||||||
),
|
),
|
||||||
|
@ -224,7 +224,7 @@ public class AggregationTestHelper
|
||||||
TemporaryFolder tempFolder
|
TemporaryFolder tempFolder
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = new DefaultObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest(
|
TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest(
|
||||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||||
|
@ -264,7 +264,7 @@ public class AggregationTestHelper
|
||||||
TemporaryFolder tempFolder
|
TemporaryFolder tempFolder
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = new DefaultObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest(
|
TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest(
|
||||||
new TopNQueryConfig(),
|
new TopNQueryConfig(),
|
||||||
|
|
|
@ -20,8 +20,8 @@
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -42,7 +42,7 @@ public class DoubleMaxAggregationTest
|
||||||
public DoubleMaxAggregationTest() throws Exception
|
public DoubleMaxAggregationTest() throws Exception
|
||||||
{
|
{
|
||||||
String aggSpecJson = "{\"type\": \"doubleMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
String aggSpecJson = "{\"type\": \"doubleMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
||||||
doubleMaxAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , DoubleMaxAggregatorFactory.class);
|
doubleMaxAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, DoubleMaxAggregatorFactory.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
|
|
@ -20,8 +20,8 @@
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -42,7 +42,7 @@ public class DoubleMinAggregationTest
|
||||||
public DoubleMinAggregationTest() throws Exception
|
public DoubleMinAggregationTest() throws Exception
|
||||||
{
|
{
|
||||||
String aggSpecJson = "{\"type\": \"doubleMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
String aggSpecJson = "{\"type\": \"doubleMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
||||||
doubleMinAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , DoubleMinAggregatorFactory.class);
|
doubleMinAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, DoubleMinAggregatorFactory.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
|
|
@ -19,9 +19,10 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -40,7 +41,7 @@ public class HistogramAggregatorTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
{
|
{
|
||||||
final DefaultObjectMapper objectMapper = new DefaultObjectMapper();
|
final ObjectMapper objectMapper = TestHelper.getJsonMapper();
|
||||||
String json0 = "{\"type\": \"histogram\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
String json0 = "{\"type\": \"histogram\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
||||||
HistogramAggregatorFactory agg0 = objectMapper.readValue(json0, HistogramAggregatorFactory.class);
|
HistogramAggregatorFactory agg0 = objectMapper.readValue(json0, HistogramAggregatorFactory.class);
|
||||||
Assert.assertEquals(ImmutableList.of(), agg0.getBreaks());
|
Assert.assertEquals(ImmutableList.of(), agg0.getBreaks());
|
||||||
|
|
|
@ -21,7 +21,7 @@ package io.druid.query.aggregation;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -63,7 +63,7 @@ public class HistogramTest
|
||||||
Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0};
|
Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0};
|
||||||
Double[] visualCounts = { 123., 4., 56., 7. };
|
Double[] visualCounts = { 123., 4., 56., 7. };
|
||||||
|
|
||||||
ObjectMapper objectMapper = new DefaultObjectMapper();
|
ObjectMapper objectMapper = TestHelper.getJsonMapper();
|
||||||
String json = objectMapper.writeValueAsString(h.asVisual());
|
String json = objectMapper.writeValueAsString(h.asVisual());
|
||||||
|
|
||||||
Map<String,Object> expectedObj = Maps.newLinkedHashMap();
|
Map<String,Object> expectedObj = Maps.newLinkedHashMap();
|
||||||
|
|
|
@ -20,8 +20,8 @@
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -42,7 +42,7 @@ public class LongMaxAggregationTest
|
||||||
public LongMaxAggregationTest() throws Exception
|
public LongMaxAggregationTest() throws Exception
|
||||||
{
|
{
|
||||||
String aggSpecJson = "{\"type\": \"longMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
String aggSpecJson = "{\"type\": \"longMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
||||||
longMaxAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , LongMaxAggregatorFactory.class);
|
longMaxAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, LongMaxAggregatorFactory.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
|
|
@ -20,8 +20,8 @@
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -42,7 +42,7 @@ public class LongMinAggregationTest
|
||||||
public LongMinAggregationTest() throws Exception
|
public LongMinAggregationTest() throws Exception
|
||||||
{
|
{
|
||||||
String aggSpecJson = "{\"type\": \"longMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
String aggSpecJson = "{\"type\": \"longMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
|
||||||
longMinAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , LongMinAggregatorFactory.class);
|
longMinAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, LongMinAggregatorFactory.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.query.aggregation.CountAggregator;
|
import io.druid.query.aggregation.CountAggregator;
|
||||||
import io.druid.query.aggregation.PostAggregator;
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -64,22 +65,22 @@ public class ArithmeticPostAggregatorTest
|
||||||
}
|
}
|
||||||
|
|
||||||
arithmeticPostAggregator = new ArithmeticPostAggregator("add", "+", postAggregatorList);
|
arithmeticPostAggregator = new ArithmeticPostAggregator("add", "+", postAggregatorList);
|
||||||
expressionPostAggregator = new ExpressionPostAggregator("add", "roku + rows");
|
expressionPostAggregator = new ExpressionPostAggregator("add", "roku + rows", null, TestExprMacroTable.INSTANCE);
|
||||||
Assert.assertEquals(9.0, arithmeticPostAggregator.compute(metricValues));
|
Assert.assertEquals(9.0, arithmeticPostAggregator.compute(metricValues));
|
||||||
Assert.assertEquals(9.0, expressionPostAggregator.compute(metricValues));
|
Assert.assertEquals(9.0, expressionPostAggregator.compute(metricValues));
|
||||||
|
|
||||||
arithmeticPostAggregator = new ArithmeticPostAggregator("subtract", "-", postAggregatorList);
|
arithmeticPostAggregator = new ArithmeticPostAggregator("subtract", "-", postAggregatorList);
|
||||||
expressionPostAggregator = new ExpressionPostAggregator("add", "roku - rows");
|
expressionPostAggregator = new ExpressionPostAggregator("add", "roku - rows", null, TestExprMacroTable.INSTANCE);
|
||||||
Assert.assertEquals(3.0, arithmeticPostAggregator.compute(metricValues));
|
Assert.assertEquals(3.0, arithmeticPostAggregator.compute(metricValues));
|
||||||
Assert.assertEquals(3.0, expressionPostAggregator.compute(metricValues));
|
Assert.assertEquals(3.0, expressionPostAggregator.compute(metricValues));
|
||||||
|
|
||||||
arithmeticPostAggregator = new ArithmeticPostAggregator("multiply", "*", postAggregatorList);
|
arithmeticPostAggregator = new ArithmeticPostAggregator("multiply", "*", postAggregatorList);
|
||||||
expressionPostAggregator = new ExpressionPostAggregator("add", "roku * rows");
|
expressionPostAggregator = new ExpressionPostAggregator("add", "roku * rows", null, TestExprMacroTable.INSTANCE);
|
||||||
Assert.assertEquals(18.0, arithmeticPostAggregator.compute(metricValues));
|
Assert.assertEquals(18.0, arithmeticPostAggregator.compute(metricValues));
|
||||||
Assert.assertEquals(18.0, expressionPostAggregator.compute(metricValues));
|
Assert.assertEquals(18.0, expressionPostAggregator.compute(metricValues));
|
||||||
|
|
||||||
arithmeticPostAggregator = new ArithmeticPostAggregator("divide", "/", postAggregatorList);
|
arithmeticPostAggregator = new ArithmeticPostAggregator("divide", "/", postAggregatorList);
|
||||||
expressionPostAggregator = new ExpressionPostAggregator("add", "roku / rows");
|
expressionPostAggregator = new ExpressionPostAggregator("add", "roku / rows", null, TestExprMacroTable.INSTANCE);
|
||||||
Assert.assertEquals(2.0, arithmeticPostAggregator.compute(metricValues));
|
Assert.assertEquals(2.0, arithmeticPostAggregator.compute(metricValues));
|
||||||
Assert.assertEquals(2.0, expressionPostAggregator.compute(metricValues));
|
Assert.assertEquals(2.0, expressionPostAggregator.compute(metricValues));
|
||||||
}
|
}
|
||||||
|
|
|
@ -37,7 +37,7 @@ public class ListFilteredDimensionSpecTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
//isWhitelist = true
|
//isWhitelist = true
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
|
|
|
@ -36,7 +36,7 @@ public class RegexFilteredDimensionSpecTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"regexFiltered\",\n"
|
+ " \"type\": \"regexFiltered\",\n"
|
||||||
|
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.expression;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
|
|
||||||
|
public class TestExprMacroTable extends ExprMacroTable
|
||||||
|
{
|
||||||
|
public static final ExprMacroTable INSTANCE = new TestExprMacroTable();
|
||||||
|
|
||||||
|
private TestExprMacroTable()
|
||||||
|
{
|
||||||
|
super(
|
||||||
|
ImmutableList.of(
|
||||||
|
new LikeExprMacro()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -22,7 +22,6 @@ package io.druid.query.groupby;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.PeriodGranularity;
|
import io.druid.java.util.common.granularity.PeriodGranularity;
|
||||||
import io.druid.query.CachingEmitter;
|
import io.druid.query.CachingEmitter;
|
||||||
import io.druid.query.DefaultQueryMetricsTest;
|
import io.druid.query.DefaultQueryMetricsTest;
|
||||||
|
@ -34,6 +33,7 @@ import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
import io.druid.query.extraction.MapLookupExtractor;
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
import io.druid.query.lookup.LookupExtractionFn;
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -55,7 +55,7 @@ public class DefaultGroupByQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(new DefaultObjectMapper());
|
DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(TestHelper.getJsonMapper());
|
||||||
GroupByQuery.Builder builder = GroupByQuery
|
GroupByQuery.Builder builder = GroupByQuery
|
||||||
.builder()
|
.builder()
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
@ -120,7 +120,7 @@ public class DefaultGroupByQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(new DefaultObjectMapper());
|
DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(TestHelper.getJsonMapper());
|
||||||
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,15 +21,15 @@ package io.druid.query.groupby;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
public class GroupByQueryConfigTest
|
public class GroupByQueryConfigTest
|
||||||
{
|
{
|
||||||
private final ObjectMapper MAPPER = new DefaultObjectMapper();
|
private final ObjectMapper MAPPER = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
private final ImmutableMap<String, String> CONFIG_MAP = ImmutableMap
|
private final ImmutableMap<String, String> CONFIG_MAP = ImmutableMap
|
||||||
.<String, String>builder()
|
.<String, String>builder()
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package io.druid.query.groupby;
|
package io.druid.query.groupby;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
|
||||||
import com.google.common.base.Functions;
|
import com.google.common.base.Functions;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Suppliers;
|
import com.google.common.base.Suppliers;
|
||||||
|
@ -35,7 +34,6 @@ import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import io.druid.collections.BlockingPool;
|
import io.druid.collections.BlockingPool;
|
||||||
import io.druid.collections.StupidPool;
|
import io.druid.collections.StupidPool;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.IAE;
|
import io.druid.java.util.common.IAE;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
import io.druid.java.util.common.granularity.DurationGranularity;
|
import io.druid.java.util.common.granularity.DurationGranularity;
|
||||||
|
@ -83,6 +81,7 @@ import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
import io.druid.query.dimension.ListFilteredDimensionSpec;
|
import io.druid.query.dimension.ListFilteredDimensionSpec;
|
||||||
import io.druid.query.dimension.RegexFilteredDimensionSpec;
|
import io.druid.query.dimension.RegexFilteredDimensionSpec;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.extraction.CascadeExtractionFn;
|
import io.druid.query.extraction.CascadeExtractionFn;
|
||||||
import io.druid.query.extraction.DimExtractionFn;
|
import io.druid.query.extraction.DimExtractionFn;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
@ -152,7 +151,7 @@ import static org.junit.Assert.assertEquals;
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class GroupByQueryRunnerTest
|
public class GroupByQueryRunnerTest
|
||||||
{
|
{
|
||||||
public static final ObjectMapper DEFAULT_MAPPER = new DefaultObjectMapper(new SmileFactory());
|
public static final ObjectMapper DEFAULT_MAPPER = TestHelper.getSmileMapper();
|
||||||
public static final DruidProcessingConfig DEFAULT_PROCESSING_CONFIG = new DruidProcessingConfig()
|
public static final DruidProcessingConfig DEFAULT_PROCESSING_CONFIG = new DruidProcessingConfig()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -2506,7 +2505,11 @@ public class GroupByQueryRunnerTest
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||||
.setVirtualColumns(
|
.setVirtualColumns(
|
||||||
new ExpressionVirtualColumn("expr", "index * 2 + indexMin / 10")
|
new ExpressionVirtualColumn(
|
||||||
|
"expr",
|
||||||
|
"index * 2 + indexMin / 10",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
)
|
||||||
)
|
)
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
|
@ -2719,7 +2722,7 @@ public class GroupByQueryRunnerTest
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
new DoubleSumAggregatorFactory("idx", null, "index / 2 + indexMin")
|
new DoubleSumAggregatorFactory("idx", null, "index / 2 + indexMin", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -2744,7 +2747,7 @@ public class GroupByQueryRunnerTest
|
||||||
// Now try it with an expression virtual column.
|
// Now try it with an expression virtual column.
|
||||||
builder.setLimit(Integer.MAX_VALUE)
|
builder.setLimit(Integer.MAX_VALUE)
|
||||||
.setVirtualColumns(
|
.setVirtualColumns(
|
||||||
new ExpressionVirtualColumn("expr", "index / 2 + indexMin")
|
new ExpressionVirtualColumn("expr", "index / 2 + indexMin", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
|
@ -3914,7 +3917,7 @@ public class GroupByQueryRunnerTest
|
||||||
|
|
||||||
fullQuery = fullQuery.withPostAggregatorSpecs(
|
fullQuery = fullQuery.withPostAggregatorSpecs(
|
||||||
Arrays.<PostAggregator>asList(
|
Arrays.<PostAggregator>asList(
|
||||||
new ExpressionPostAggregator("rows_times_10", "rows * 10.0")
|
new ExpressionPostAggregator("rows_times_10", "rows * 10.0", null, TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -4334,7 +4337,7 @@ public class GroupByQueryRunnerTest
|
||||||
|
|
||||||
subquery = new GroupByQuery.Builder(subquery)
|
subquery = new GroupByQuery.Builder(subquery)
|
||||||
.setVirtualColumns(
|
.setVirtualColumns(
|
||||||
new ExpressionVirtualColumn("expr", "-index + 100")
|
new ExpressionVirtualColumn("expr", "-index + 100", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
|
@ -5436,7 +5439,7 @@ public class GroupByQueryRunnerTest
|
||||||
.builder()
|
.builder()
|
||||||
.setDataSource(subquery)
|
.setDataSource(subquery)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
.setVirtualColumns(new ExpressionVirtualColumn("expr", "1"))
|
.setVirtualColumns(new ExpressionVirtualColumn("expr", "1", TestExprMacroTable.INSTANCE))
|
||||||
.setDimensions(Lists.<DimensionSpec>newArrayList())
|
.setDimensions(Lists.<DimensionSpec>newArrayList())
|
||||||
.setAggregatorSpecs(ImmutableList.<AggregatorFactory>of(new LongSumAggregatorFactory("count", "expr")))
|
.setAggregatorSpecs(ImmutableList.<AggregatorFactory>of(new LongSumAggregatorFactory("count", "expr")))
|
||||||
.setGranularity(QueryRunnerTestHelper.allGran)
|
.setGranularity(QueryRunnerTestHelper.allGran)
|
||||||
|
|
|
@ -26,7 +26,6 @@ import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import io.druid.data.input.MapBasedRow;
|
import io.druid.data.input.MapBasedRow;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunnerTestHelper;
|
import io.druid.query.QueryRunnerTestHelper;
|
||||||
|
@ -39,6 +38,7 @@ import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.groupby.orderby.DefaultLimitSpec;
|
import io.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||||
import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
import io.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -48,7 +48,7 @@ import java.util.Arrays;
|
||||||
|
|
||||||
public class GroupByQueryTest
|
public class GroupByQueryTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testQuerySerialization() throws IOException
|
public void testQuerySerialization() throws IOException
|
||||||
|
|
|
@ -36,6 +36,7 @@ import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
import io.druid.query.aggregation.post.ExpressionPostAggregator;
|
import io.druid.query.aggregation.post.ExpressionPostAggregator;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
import io.druid.segment.TestHelper;
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -67,7 +68,7 @@ public class DefaultLimitSpecTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
{
|
{
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
//defaults
|
//defaults
|
||||||
String json = "{\"type\": \"default\"}";
|
String json = "{\"type\": \"default\"}";
|
||||||
|
@ -260,7 +261,7 @@ public class DefaultLimitSpecTest
|
||||||
limitFn = limitSpec.build(
|
limitFn = limitSpec.build(
|
||||||
ImmutableList.<DimensionSpec>of(new DefaultDimensionSpec("k1", "k1")),
|
ImmutableList.<DimensionSpec>of(new DefaultDimensionSpec("k1", "k1")),
|
||||||
ImmutableList.<AggregatorFactory>of(new LongSumAggregatorFactory("k2", "k2")),
|
ImmutableList.<AggregatorFactory>of(new LongSumAggregatorFactory("k2", "k2")),
|
||||||
ImmutableList.<PostAggregator>of(new ExpressionPostAggregator("k1", "1 + 1"))
|
ImmutableList.<PostAggregator>of(new ExpressionPostAggregator("k1", "1 + 1", null, TestExprMacroTable.INSTANCE))
|
||||||
);
|
);
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
(List)ImmutableList.of(testRowsList.get(2), testRowsList.get(0)),
|
(List)ImmutableList.of(testRowsList.get(2), testRowsList.get(0)),
|
||||||
|
|
|
@ -31,7 +31,7 @@ import java.io.IOException;
|
||||||
public class LookupConfigTest
|
public class LookupConfigTest
|
||||||
{
|
{
|
||||||
|
|
||||||
ObjectMapper mapper = TestHelper.JSON_MAPPER;
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||||
|
|
|
@ -26,7 +26,7 @@ import org.junit.Test;
|
||||||
|
|
||||||
public class ColumnAnalysisTest
|
public class ColumnAnalysisTest
|
||||||
{
|
{
|
||||||
private final ObjectMapper MAPPER = TestHelper.getObjectMapper();
|
private final ObjectMapper MAPPER = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
private void assertSerDe(ColumnAnalysis analysis) throws Exception
|
private void assertSerDe(ColumnAnalysis analysis) throws Exception
|
||||||
{
|
{
|
||||||
|
|
|
@ -117,8 +117,8 @@ public class SearchQueryRunnerTest
|
||||||
public void testSearchHitSerDe() throws Exception
|
public void testSearchHitSerDe() throws Exception
|
||||||
{
|
{
|
||||||
for (SearchHit hit : Arrays.asList(new SearchHit("dim1", "val1"), new SearchHit("dim2", "val2", 3))) {
|
for (SearchHit hit : Arrays.asList(new SearchHit("dim1", "val1"), new SearchHit("dim2", "val2", 3))) {
|
||||||
SearchHit read = TestHelper.JSON_MAPPER.readValue(
|
SearchHit read = TestHelper.getJsonMapper().readValue(
|
||||||
TestHelper.JSON_MAPPER.writeValueAsString(hit),
|
TestHelper.getJsonMapper().writeValueAsString(hit),
|
||||||
SearchHit.class
|
SearchHit.class
|
||||||
);
|
);
|
||||||
Assert.assertEquals(hit, read);
|
Assert.assertEquals(hit, read);
|
||||||
|
|
|
@ -40,6 +40,7 @@ import io.druid.query.TableDataSource;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
import io.druid.query.extraction.MapLookupExtractor;
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
|
@ -508,7 +509,9 @@ public class SelectQueryRunnerTest
|
||||||
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
|
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
|
||||||
.metrics(Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric))
|
.metrics(Lists.<String>newArrayList(QueryRunnerTestHelper.indexMetric))
|
||||||
.pagingSpec(new PagingSpec(null, 10, true))
|
.pagingSpec(new PagingSpec(null, 10, true))
|
||||||
.virtualColumns(new ExpressionVirtualColumn("expr", "index / 10.0"))
|
.virtualColumns(
|
||||||
|
new ExpressionVirtualColumn("expr", "index / 10.0", TestExprMacroTable.INSTANCE)
|
||||||
|
)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||||
|
|
|
@ -20,12 +20,12 @@
|
||||||
package io.druid.query.timeseries;
|
package io.druid.query.timeseries;
|
||||||
|
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.CachingEmitter;
|
import io.druid.query.CachingEmitter;
|
||||||
import io.druid.query.DefaultQueryMetricsTest;
|
import io.druid.query.DefaultQueryMetricsTest;
|
||||||
import io.druid.query.DruidMetrics;
|
import io.druid.query.DruidMetrics;
|
||||||
import io.druid.query.Druids;
|
import io.druid.query.Druids;
|
||||||
import io.druid.query.QueryRunnerTestHelper;
|
import io.druid.query.QueryRunnerTestHelper;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -48,7 +48,7 @@ public class DefaultTimeseriesQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(new DefaultObjectMapper());
|
DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(TestHelper.getJsonMapper());
|
||||||
TimeseriesQuery query = Druids
|
TimeseriesQuery query = Druids
|
||||||
.newTimeseriesQueryBuilder()
|
.newTimeseriesQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
@ -96,7 +96,7 @@ public class DefaultTimeseriesQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(new DefaultObjectMapper());
|
DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(TestHelper.getJsonMapper());
|
||||||
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.query.timeseries;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.CacheStrategy;
|
import io.druid.query.CacheStrategy;
|
||||||
import io.druid.query.Druids;
|
import io.druid.query.Druids;
|
||||||
|
@ -32,6 +31,7 @@ import io.druid.query.TableDataSource;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.VirtualColumns;
|
import io.druid.segment.VirtualColumns;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -98,7 +98,7 @@ public class TimeseriesQueryQueryToolChestTest
|
||||||
|
|
||||||
Object preparedValue = strategy.prepareForCache().apply(result);
|
Object preparedValue = strategy.prepareForCache().apply(result);
|
||||||
|
|
||||||
ObjectMapper objectMapper = new DefaultObjectMapper();
|
ObjectMapper objectMapper = TestHelper.getJsonMapper();
|
||||||
Object fromCacheValue = objectMapper.readValue(
|
Object fromCacheValue = objectMapper.readValue(
|
||||||
objectMapper.writeValueAsBytes(preparedValue),
|
objectMapper.writeValueAsBytes(preparedValue),
|
||||||
strategy.getCacheObjectClazz()
|
strategy.getCacheObjectClazz()
|
||||||
|
|
|
@ -42,6 +42,7 @@ import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.PostAggregator;
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory;
|
import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory;
|
||||||
import io.druid.query.aggregation.last.DoubleLastAggregatorFactory;
|
import io.druid.query.aggregation.last.DoubleLastAggregatorFactory;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.extraction.MapLookupExtractor;
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
import io.druid.query.filter.AndDimFilter;
|
import io.druid.query.filter.AndDimFilter;
|
||||||
import io.druid.query.filter.BoundDimFilter;
|
import io.druid.query.filter.BoundDimFilter;
|
||||||
|
@ -416,7 +417,13 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.descending(descending)
|
.descending(descending)
|
||||||
.virtualColumns(new ExpressionVirtualColumn("expr", "index"))
|
.virtualColumns(
|
||||||
|
new ExpressionVirtualColumn(
|
||||||
|
"expr",
|
||||||
|
"index",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
)
|
||||||
|
)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||||
|
|
|
@ -20,11 +20,11 @@
|
||||||
package io.druid.query.timeseries;
|
package io.druid.query.timeseries;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.Druids;
|
import io.druid.query.Druids;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunnerTestHelper;
|
import io.druid.query.QueryRunnerTestHelper;
|
||||||
import io.druid.query.aggregation.PostAggregator;
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
|
@ -36,7 +36,7 @@ import java.util.Arrays;
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class TimeseriesQueryTest
|
public class TimeseriesQueryTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Parameterized.Parameters(name="descending={0}")
|
@Parameterized.Parameters(name="descending={0}")
|
||||||
public static Iterable<Object[]> constructorFeeder() throws IOException
|
public static Iterable<Object[]> constructorFeeder() throws IOException
|
||||||
|
|
|
@ -22,7 +22,7 @@ package io.druid.query.topn;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -102,7 +102,7 @@ public class AlphaNumericTopNMetricSpecTest
|
||||||
+ " \"type\": \"alphaNumeric\",\n"
|
+ " \"type\": \"alphaNumeric\",\n"
|
||||||
+ " \"previousStop\": \"test\"\n"
|
+ " \"previousStop\": \"test\"\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), AlphaNumericTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), AlphaNumericTopNMetricSpec.class);
|
||||||
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), AlphaNumericTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), AlphaNumericTopNMetricSpec.class);
|
||||||
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.topn;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.CachingEmitter;
|
import io.druid.query.CachingEmitter;
|
||||||
import io.druid.query.DefaultQueryMetricsTest;
|
import io.druid.query.DefaultQueryMetricsTest;
|
||||||
|
@ -31,6 +30,7 @@ import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.ListFilteredDimensionSpec;
|
import io.druid.query.dimension.ListFilteredDimensionSpec;
|
||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -52,7 +52,7 @@ public class DefaultTopNQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(new DefaultObjectMapper());
|
DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(TestHelper.getJsonMapper());
|
||||||
TopNQuery query = new TopNQueryBuilder()
|
TopNQuery query = new TopNQueryBuilder()
|
||||||
.dataSource("xx")
|
.dataSource("xx")
|
||||||
.granularity(Granularities.ALL)
|
.granularity(Granularities.ALL)
|
||||||
|
@ -102,7 +102,7 @@ public class DefaultTopNQueryMetricsTest
|
||||||
{
|
{
|
||||||
CachingEmitter cachingEmitter = new CachingEmitter();
|
CachingEmitter cachingEmitter = new CachingEmitter();
|
||||||
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter);
|
||||||
DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(new DefaultObjectMapper());
|
DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(TestHelper.getJsonMapper());
|
||||||
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,8 +20,8 @@
|
||||||
package io.druid.query.topn;
|
package io.druid.query.topn;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -42,7 +42,7 @@ public class DimensionTopNMetricSpecTest
|
||||||
+ " \"ordering\": \"alphanumeric\",\n"
|
+ " \"ordering\": \"alphanumeric\",\n"
|
||||||
+ " \"previousStop\": \"test\"\n"
|
+ " \"previousStop\": \"test\"\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
||||||
|
@ -62,7 +62,7 @@ public class DimensionTopNMetricSpecTest
|
||||||
+ " \"ordering\": \"lexicographic\",\n"
|
+ " \"ordering\": \"lexicographic\",\n"
|
||||||
+ " \"previousStop\": \"test\"\n"
|
+ " \"previousStop\": \"test\"\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
||||||
|
@ -82,7 +82,7 @@ public class DimensionTopNMetricSpecTest
|
||||||
+ " \"ordering\": \"strlen\",\n"
|
+ " \"ordering\": \"strlen\",\n"
|
||||||
+ " \"previousStop\": \"test\"\n"
|
+ " \"previousStop\": \"test\"\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
||||||
|
@ -102,7 +102,7 @@ public class DimensionTopNMetricSpecTest
|
||||||
+ " \"ordering\": \"numeric\",\n"
|
+ " \"ordering\": \"numeric\",\n"
|
||||||
+ " \"previousStop\": \"test\"\n"
|
+ " \"previousStop\": \"test\"\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class);
|
||||||
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
Assert.assertEquals(expectedMetricSpec, actualMetricSpec);
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.java.util.common.guava.Sequence;
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.query.CacheStrategy;
|
import io.druid.query.CacheStrategy;
|
||||||
|
@ -43,6 +42,7 @@ import io.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
import io.druid.segment.IncrementalIndexSegment;
|
import io.druid.segment.IncrementalIndexSegment;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.TestIndex;
|
import io.druid.segment.TestIndex;
|
||||||
import io.druid.segment.VirtualColumns;
|
import io.druid.segment.VirtualColumns;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -101,7 +101,7 @@ public class TopNQueryQueryToolChestTest
|
||||||
result
|
result
|
||||||
);
|
);
|
||||||
|
|
||||||
ObjectMapper objectMapper = new DefaultObjectMapper();
|
ObjectMapper objectMapper = TestHelper.getJsonMapper();
|
||||||
Object fromCacheValue = objectMapper.readValue(
|
Object fromCacheValue = objectMapper.readValue(
|
||||||
objectMapper.writeValueAsBytes(preparedValue),
|
objectMapper.writeValueAsBytes(preparedValue),
|
||||||
strategy.getCacheObjectClazz()
|
strategy.getCacheObjectClazz()
|
||||||
|
|
|
@ -39,6 +39,7 @@ import io.druid.java.util.common.granularity.Granularity;
|
||||||
import io.druid.java.util.common.guava.Sequence;
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.js.JavaScriptConfig;
|
import io.druid.js.JavaScriptConfig;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.query.BySegmentResultValue;
|
import io.druid.query.BySegmentResultValue;
|
||||||
import io.druid.query.BySegmentResultValueClass;
|
import io.druid.query.BySegmentResultValueClass;
|
||||||
import io.druid.query.Druids;
|
import io.druid.query.Druids;
|
||||||
|
@ -1976,7 +1977,7 @@ public class TopNQueryRunnerTest
|
||||||
query = query.withAggregatorSpecs(
|
query = query.withAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
QueryRunnerTestHelper.rowsCount,
|
QueryRunnerTestHelper.rowsCount,
|
||||||
new DoubleSumAggregatorFactory("index", null, "-index + 100")
|
new DoubleSumAggregatorFactory("index", null, "-index + 100", ExprMacroTable.nil())
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -4189,7 +4190,7 @@ public class TopNQueryRunnerTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFullOnTopNVirtualColumn()
|
public void testFullOnTopNLongVirtualColumn()
|
||||||
{
|
{
|
||||||
TopNQuery query = new TopNQueryBuilder()
|
TopNQuery query = new TopNQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
@ -4210,7 +4211,7 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.virtualColumns(new ExpressionVirtualColumn("ql_expr", "qualityLong"))
|
.virtualColumns(new ExpressionVirtualColumn("ql_expr", "qualityLong", ExprMacroTable.nil()))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||||
|
@ -34,6 +33,7 @@ import io.druid.query.dimension.LegacyDimensionSpec;
|
||||||
import io.druid.query.extraction.MapLookupExtractor;
|
import io.druid.query.extraction.MapLookupExtractor;
|
||||||
import io.druid.query.lookup.LookupExtractionFn;
|
import io.druid.query.lookup.LookupExtractionFn;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -51,7 +51,7 @@ import static io.druid.query.QueryRunnerTestHelper.rowsCount;
|
||||||
|
|
||||||
public class TopNQueryTest
|
public class TopNQueryTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testQuerySerialization() throws IOException
|
public void testQuerySerialization() throws IOException
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.segment;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import io.druid.data.input.impl.TimestampSpec;
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||||
|
@ -41,7 +40,7 @@ public class MetadataTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
{
|
{
|
||||||
ObjectMapper jsonMapper = new DefaultObjectMapper();
|
ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
Metadata metadata = new Metadata();
|
Metadata metadata = new Metadata();
|
||||||
metadata.put("k", "v");
|
metadata.put("k", "v");
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.data.input.MapBasedRow;
|
import io.druid.data.input.MapBasedRow;
|
||||||
|
@ -26,7 +27,9 @@ import io.druid.data.input.Row;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.java.util.common.guava.Sequence;
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.query.Result;
|
import io.druid.query.Result;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.timeseries.TimeseriesResultValue;
|
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||||
import io.druid.segment.column.ColumnConfig;
|
import io.druid.segment.column.ColumnConfig;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -41,12 +44,11 @@ public class TestHelper
|
||||||
private static final IndexMerger INDEX_MERGER;
|
private static final IndexMerger INDEX_MERGER;
|
||||||
private static final IndexMergerV9 INDEX_MERGER_V9;
|
private static final IndexMergerV9 INDEX_MERGER_V9;
|
||||||
private static final IndexIO INDEX_IO;
|
private static final IndexIO INDEX_IO;
|
||||||
public static final ObjectMapper JSON_MAPPER;
|
|
||||||
|
|
||||||
static {
|
static {
|
||||||
JSON_MAPPER = new DefaultObjectMapper();
|
final ObjectMapper jsonMapper = getJsonMapper();
|
||||||
INDEX_IO = new IndexIO(
|
INDEX_IO = new IndexIO(
|
||||||
JSON_MAPPER,
|
jsonMapper,
|
||||||
new ColumnConfig()
|
new ColumnConfig()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -56,16 +58,10 @@ public class TestHelper
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
INDEX_MERGER = new IndexMerger(JSON_MAPPER, INDEX_IO);
|
INDEX_MERGER = new IndexMerger(jsonMapper, INDEX_IO);
|
||||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
|
INDEX_MERGER_V9 = new IndexMergerV9(jsonMapper, INDEX_IO);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ObjectMapper getTestObjectMapper()
|
|
||||||
{
|
|
||||||
return JSON_MAPPER;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public static IndexMerger getTestIndexMerger()
|
public static IndexMerger getTestIndexMerger()
|
||||||
{
|
{
|
||||||
return INDEX_MERGER;
|
return INDEX_MERGER;
|
||||||
|
@ -81,11 +77,30 @@ public class TestHelper
|
||||||
return INDEX_IO;
|
return INDEX_IO;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ObjectMapper getObjectMapper() {
|
public static ObjectMapper getJsonMapper()
|
||||||
return JSON_MAPPER;
|
{
|
||||||
|
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||||
|
mapper.setInjectableValues(
|
||||||
|
new InjectableValues.Std()
|
||||||
|
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||||
|
.addValue(ObjectMapper.class.getName(), mapper)
|
||||||
|
);
|
||||||
|
return mapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <T> Iterable<T> revert(Iterable<T> input) {
|
public static ObjectMapper getSmileMapper()
|
||||||
|
{
|
||||||
|
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||||
|
mapper.setInjectableValues(
|
||||||
|
new InjectableValues.Std()
|
||||||
|
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||||
|
.addValue(ObjectMapper.class.getName(), mapper)
|
||||||
|
);
|
||||||
|
return mapper;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> Iterable<T> revert(Iterable<T> input)
|
||||||
|
{
|
||||||
return Lists.reverse(Lists.newArrayList(input));
|
return Lists.reverse(Lists.newArrayList(input));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -272,7 +287,11 @@ public class TestHelper
|
||||||
{
|
{
|
||||||
// Custom equals check to get fuzzy comparison of numerics, useful because different groupBy strategies don't
|
// Custom equals check to get fuzzy comparison of numerics, useful because different groupBy strategies don't
|
||||||
// always generate exactly the same results (different merge ordering / float vs double)
|
// always generate exactly the same results (different merge ordering / float vs double)
|
||||||
Assert.assertEquals(String.format("%s: timestamp", msg), expected.getTimestamp().getMillis(), actual.getTimestamp().getMillis());
|
Assert.assertEquals(
|
||||||
|
String.format("%s: timestamp", msg),
|
||||||
|
expected.getTimestamp().getMillis(),
|
||||||
|
actual.getTimestamp().getMillis()
|
||||||
|
);
|
||||||
|
|
||||||
final Map<String, Object> expectedMap = ((MapBasedRow) expected).getEvent();
|
final Map<String, Object> expectedMap = ((MapBasedRow) expected).getEvent();
|
||||||
final Map<String, Object> actualMap = ((MapBasedRow) actual).getEvent();
|
final Map<String, Object> actualMap = ((MapBasedRow) actual).getEvent();
|
||||||
|
|
|
@ -41,6 +41,7 @@ import io.druid.query.aggregation.DoubleMinAggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
|
@ -53,6 +54,7 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
@ -110,8 +112,8 @@ public class TestIndex
|
||||||
private static final Logger log = new Logger(TestIndex.class);
|
private static final Logger log = new Logger(TestIndex.class);
|
||||||
private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z");
|
private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z");
|
||||||
private static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create(
|
private static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create(
|
||||||
Arrays.<VirtualColumn>asList(
|
Collections.<VirtualColumn>singletonList(
|
||||||
new ExpressionVirtualColumn("expr", "index + 10")
|
new ExpressionVirtualColumn("expr", "index + 10", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
public static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
|
public static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
|
||||||
|
|
|
@ -38,6 +38,7 @@ import io.druid.query.aggregation.Aggregator;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.FilteredAggregatorFactory;
|
import io.druid.query.aggregation.FilteredAggregatorFactory;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.filter.BitmapIndexSelector;
|
import io.druid.query.filter.BitmapIndexSelector;
|
||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
import io.druid.query.filter.Filter;
|
import io.druid.query.filter.Filter;
|
||||||
|
@ -83,7 +84,7 @@ public abstract class BaseFilterTest
|
||||||
{
|
{
|
||||||
private static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create(
|
private static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create(
|
||||||
ImmutableList.<VirtualColumn>of(
|
ImmutableList.<VirtualColumn>of(
|
||||||
new ExpressionVirtualColumn("expr", "1.0 + 0.1")
|
new ExpressionVirtualColumn("expr", "1.0 + 0.1", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -41,7 +41,7 @@ public class DictionaryEncodedColumnPartSerdeTest
|
||||||
+ " \"byteOrder\": \"BIG_ENDIAN\"\n"
|
+ " \"byteOrder\": \"BIG_ENDIAN\"\n"
|
||||||
+ "}\n";
|
+ "}\n";
|
||||||
|
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
DictionaryEncodedColumnPartSerde serde = (DictionaryEncodedColumnPartSerde) mapper.readValue(
|
DictionaryEncodedColumnPartSerde serde = (DictionaryEncodedColumnPartSerde) mapper.readValue(
|
||||||
mapper.writeValueAsString(
|
mapper.writeValueAsString(
|
||||||
|
|
|
@ -21,9 +21,9 @@ package io.druid.segment.virtual;
|
||||||
|
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import io.druid.common.guava.SettableSupplier;
|
import io.druid.common.guava.SettableSupplier;
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
import io.druid.segment.TestFloatColumnSelector;
|
|
||||||
import io.druid.segment.TestLongColumnSelector;
|
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -31,46 +31,12 @@ import java.util.List;
|
||||||
|
|
||||||
public class ExpressionObjectSelectorTest
|
public class ExpressionObjectSelectorTest
|
||||||
{
|
{
|
||||||
@Test
|
|
||||||
public void testSupplierFromLongSelector()
|
|
||||||
{
|
|
||||||
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromLongSelector(
|
|
||||||
new TestLongColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public long get()
|
|
||||||
{
|
|
||||||
return 1L;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
Assert.assertEquals(1L, supplier.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testSupplierFromFloatSelector()
|
|
||||||
{
|
|
||||||
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromFloatSelector(
|
|
||||||
new TestFloatColumnSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public float get()
|
|
||||||
{
|
|
||||||
return 0.1f;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
Assert.assertEquals(0.1f, supplier.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSupplierFromObjectSelectorObject()
|
public void testSupplierFromObjectSelectorObject()
|
||||||
{
|
{
|
||||||
final SettableSupplier<Object> settableSupplier = new SettableSupplier<>();
|
final SettableSupplier<Object> settableSupplier = new SettableSupplier<>();
|
||||||
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
||||||
selectorFromSupplier(settableSupplier, Object.class)
|
objectSelectorFromSupplier(settableSupplier, Object.class)
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertNotNull(supplier);
|
Assert.assertNotNull(supplier);
|
||||||
|
@ -94,7 +60,7 @@ public class ExpressionObjectSelectorTest
|
||||||
{
|
{
|
||||||
final SettableSupplier<Number> settableSupplier = new SettableSupplier<>();
|
final SettableSupplier<Number> settableSupplier = new SettableSupplier<>();
|
||||||
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
||||||
selectorFromSupplier(settableSupplier, Number.class)
|
objectSelectorFromSupplier(settableSupplier, Number.class)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
|
@ -113,7 +79,7 @@ public class ExpressionObjectSelectorTest
|
||||||
{
|
{
|
||||||
final SettableSupplier<String> settableSupplier = new SettableSupplier<>();
|
final SettableSupplier<String> settableSupplier = new SettableSupplier<>();
|
||||||
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
||||||
selectorFromSupplier(settableSupplier, String.class)
|
objectSelectorFromSupplier(settableSupplier, String.class)
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertNotNull(supplier);
|
Assert.assertNotNull(supplier);
|
||||||
|
@ -131,14 +97,37 @@ public class ExpressionObjectSelectorTest
|
||||||
{
|
{
|
||||||
final SettableSupplier<List> settableSupplier = new SettableSupplier<>();
|
final SettableSupplier<List> settableSupplier = new SettableSupplier<>();
|
||||||
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
final Supplier<Number> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
|
||||||
selectorFromSupplier(settableSupplier, List.class)
|
objectSelectorFromSupplier(settableSupplier, List.class)
|
||||||
);
|
);
|
||||||
|
|
||||||
// List can't be a number, so supplierFromObjectSelector should return null.
|
// List can't be a number, so supplierFromObjectSelector should return null.
|
||||||
Assert.assertNull(supplier);
|
Assert.assertNull(supplier);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T> ObjectColumnSelector<T> selectorFromSupplier(final Supplier<T> supplier, final Class<T> clazz)
|
private static DimensionSelector dimensionSelectorFromSupplier(
|
||||||
|
final Supplier<String> supplier
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new BaseSingleValueDimensionSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
protected String getValue()
|
||||||
|
{
|
||||||
|
return supplier.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("supplier", supplier);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T> ObjectColumnSelector<T> objectSelectorFromSupplier(
|
||||||
|
final Supplier<T> supplier,
|
||||||
|
final Class<T> clazz
|
||||||
|
)
|
||||||
{
|
{
|
||||||
return new ObjectColumnSelector<T>()
|
return new ObjectColumnSelector<T>()
|
||||||
{
|
{
|
||||||
|
|
|
@ -26,6 +26,7 @@ import io.druid.data.input.InputRow;
|
||||||
import io.druid.data.input.MapBasedInputRow;
|
import io.druid.data.input.MapBasedInputRow;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.extraction.BucketExtractionFn;
|
import io.druid.query.extraction.BucketExtractionFn;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
|
import io.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
|
||||||
|
@ -56,7 +57,16 @@ public class ExpressionVirtualColumnTest
|
||||||
ImmutableMap.<String, Object>of("x", 2.1, "y", 3L)
|
ImmutableMap.<String, Object>of("x", 2.1, "y", 3L)
|
||||||
);
|
);
|
||||||
|
|
||||||
private static final ExpressionVirtualColumn XPLUSY = new ExpressionVirtualColumn("expr", "x + y");
|
private static final ExpressionVirtualColumn XPLUSY = new ExpressionVirtualColumn(
|
||||||
|
"expr",
|
||||||
|
"x + y",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
|
private static final ExpressionVirtualColumn CONSTANT_LIKE = new ExpressionVirtualColumn(
|
||||||
|
"expr",
|
||||||
|
"like('foo', 'f%')",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
private static final TestColumnSelectorFactory COLUMN_SELECTOR_FACTORY = new TestColumnSelectorFactory();
|
private static final TestColumnSelectorFactory COLUMN_SELECTOR_FACTORY = new TestColumnSelectorFactory();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -166,10 +176,18 @@ public class ExpressionVirtualColumnTest
|
||||||
Assert.assertEquals("5", selector.lookupName(selector.getRow().get(0)));
|
Assert.assertEquals("5", selector.lookupName(selector.getRow().get(0)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLongSelectorWithConstantLikeExprMacro()
|
||||||
|
{
|
||||||
|
final LongColumnSelector selector = CONSTANT_LIKE.makeLongColumnSelector("expr", COLUMN_SELECTOR_FACTORY);
|
||||||
|
|
||||||
|
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
|
||||||
|
Assert.assertEquals(1L, selector.get());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRequiredColumns()
|
public void testRequiredColumns()
|
||||||
{
|
{
|
||||||
final ExpressionVirtualColumn virtualColumn = new ExpressionVirtualColumn("expr", "x + y");
|
Assert.assertEquals(ImmutableList.of("x", "y"), XPLUSY.requiredColumns());
|
||||||
Assert.assertEquals(ImmutableList.of("x", "y"), virtualColumn.requiredColumns());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,10 +23,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
|
import io.druid.query.expression.TestExprMacroTable;
|
||||||
import io.druid.query.extraction.BucketExtractionFn;
|
import io.druid.query.extraction.BucketExtractionFn;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.ValueMatcher;
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
@ -39,6 +39,7 @@ import io.druid.segment.IdLookup;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
import io.druid.segment.TestFloatColumnSelector;
|
import io.druid.segment.TestFloatColumnSelector;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.TestLongColumnSelector;
|
import io.druid.segment.TestLongColumnSelector;
|
||||||
import io.druid.segment.VirtualColumn;
|
import io.druid.segment.VirtualColumn;
|
||||||
import io.druid.segment.VirtualColumns;
|
import io.druid.segment.VirtualColumns;
|
||||||
|
@ -123,7 +124,11 @@ public class VirtualColumnsTest
|
||||||
@Test
|
@Test
|
||||||
public void testTimeNotAllowed()
|
public void testTimeNotAllowed()
|
||||||
{
|
{
|
||||||
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn("__time", "x + y");
|
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn(
|
||||||
|
"__time",
|
||||||
|
"x + y",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
|
|
||||||
expectedException.expect(IllegalArgumentException.class);
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
expectedException.expectMessage("virtualColumn name[__time] not allowed");
|
expectedException.expectMessage("virtualColumn name[__time] not allowed");
|
||||||
|
@ -134,8 +139,17 @@ public class VirtualColumnsTest
|
||||||
@Test
|
@Test
|
||||||
public void testDuplicateNameDetection()
|
public void testDuplicateNameDetection()
|
||||||
{
|
{
|
||||||
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn("expr", "x + y");
|
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn(
|
||||||
final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn("expr", "x * 2");
|
"expr",
|
||||||
|
"x + y",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
|
|
||||||
|
final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn(
|
||||||
|
"expr",
|
||||||
|
"x * 2",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
|
|
||||||
expectedException.expect(IllegalArgumentException.class);
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
expectedException.expectMessage("Duplicate virtualColumn name[expr]");
|
expectedException.expectMessage("Duplicate virtualColumn name[expr]");
|
||||||
|
@ -146,8 +160,17 @@ public class VirtualColumnsTest
|
||||||
@Test
|
@Test
|
||||||
public void testCycleDetection()
|
public void testCycleDetection()
|
||||||
{
|
{
|
||||||
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn("expr", "x + expr2");
|
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn(
|
||||||
final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn("expr2", "expr * 2");
|
"expr",
|
||||||
|
"x + expr2",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
|
|
||||||
|
final ExpressionVirtualColumn expr2 = new ExpressionVirtualColumn(
|
||||||
|
"expr2",
|
||||||
|
"expr * 2",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
|
|
||||||
expectedException.expect(IllegalArgumentException.class);
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
expectedException.expectMessage("Self-referential column[expr]");
|
expectedException.expectMessage("Self-referential column[expr]");
|
||||||
|
@ -160,13 +183,13 @@ public class VirtualColumnsTest
|
||||||
{
|
{
|
||||||
final VirtualColumns virtualColumns = VirtualColumns.create(
|
final VirtualColumns virtualColumns = VirtualColumns.create(
|
||||||
ImmutableList.<VirtualColumn>of(
|
ImmutableList.<VirtualColumn>of(
|
||||||
new ExpressionVirtualColumn("expr", "x + y")
|
new ExpressionVirtualColumn("expr", "x + y", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
final VirtualColumns virtualColumns2 = VirtualColumns.create(
|
final VirtualColumns virtualColumns2 = VirtualColumns.create(
|
||||||
ImmutableList.<VirtualColumn>of(
|
ImmutableList.<VirtualColumn>of(
|
||||||
new ExpressionVirtualColumn("expr", "x + y")
|
new ExpressionVirtualColumn("expr", "x + y", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -179,13 +202,13 @@ public class VirtualColumnsTest
|
||||||
{
|
{
|
||||||
final VirtualColumns virtualColumns = VirtualColumns.create(
|
final VirtualColumns virtualColumns = VirtualColumns.create(
|
||||||
ImmutableList.<VirtualColumn>of(
|
ImmutableList.<VirtualColumn>of(
|
||||||
new ExpressionVirtualColumn("expr", "x + y")
|
new ExpressionVirtualColumn("expr", "x + y", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
final VirtualColumns virtualColumns2 = VirtualColumns.create(
|
final VirtualColumns virtualColumns2 = VirtualColumns.create(
|
||||||
ImmutableList.<VirtualColumn>of(
|
ImmutableList.<VirtualColumn>of(
|
||||||
new ExpressionVirtualColumn("expr", "x + y")
|
new ExpressionVirtualColumn("expr", "x + y", TestExprMacroTable.INSTANCE)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -202,10 +225,10 @@ public class VirtualColumnsTest
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
{
|
{
|
||||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
final ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
final ImmutableList<VirtualColumn> theColumns = ImmutableList.<VirtualColumn>of(
|
final ImmutableList<VirtualColumn> theColumns = ImmutableList.<VirtualColumn>of(
|
||||||
new ExpressionVirtualColumn("expr", "x + y"),
|
new ExpressionVirtualColumn("expr", "x + y", TestExprMacroTable.INSTANCE),
|
||||||
new ExpressionVirtualColumn("expr2", "x + z")
|
new ExpressionVirtualColumn("expr2", "x + z", TestExprMacroTable.INSTANCE)
|
||||||
);
|
);
|
||||||
final VirtualColumns virtualColumns = VirtualColumns.create(theColumns);
|
final VirtualColumns virtualColumns = VirtualColumns.create(theColumns);
|
||||||
|
|
||||||
|
@ -228,7 +251,11 @@ public class VirtualColumnsTest
|
||||||
|
|
||||||
private VirtualColumns makeVirtualColumns()
|
private VirtualColumns makeVirtualColumns()
|
||||||
{
|
{
|
||||||
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn("expr", "1");
|
final ExpressionVirtualColumn expr = new ExpressionVirtualColumn(
|
||||||
|
"expr",
|
||||||
|
"1",
|
||||||
|
TestExprMacroTable.INSTANCE
|
||||||
|
);
|
||||||
final DottyVirtualColumn dotty = new DottyVirtualColumn("foo");
|
final DottyVirtualColumn dotty = new DottyVirtualColumn("foo");
|
||||||
return VirtualColumns.create(ImmutableList.of(expr, dotty));
|
return VirtualColumns.create(ImmutableList.of(expr, dotty));
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,63 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.guice;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.Module;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.inject.Binder;
|
||||||
|
import com.google.inject.multibindings.Multibinder;
|
||||||
|
import io.druid.initialization.DruidModule;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
|
import io.druid.query.expression.GuiceExprMacroTable;
|
||||||
|
import io.druid.query.expression.LikeExprMacro;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class ExpressionModule implements DruidModule
|
||||||
|
{
|
||||||
|
public static final List<Class<? extends ExprMacroTable.ExprMacro>> EXPR_MACROS =
|
||||||
|
ImmutableList.<Class<? extends ExprMacroTable.ExprMacro>>builder()
|
||||||
|
.add(LikeExprMacro.class)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void configure(Binder binder)
|
||||||
|
{
|
||||||
|
binder.bind(ExprMacroTable.class).to(GuiceExprMacroTable.class).in(LazySingleton.class);
|
||||||
|
for (Class<? extends ExprMacroTable.ExprMacro> exprMacroClass : EXPR_MACROS) {
|
||||||
|
addExprMacro(binder, exprMacroClass);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<? extends Module> getJacksonModules()
|
||||||
|
{
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void addExprMacro(final Binder binder, final Class<? extends ExprMacroTable.ExprMacro> clazz)
|
||||||
|
{
|
||||||
|
Multibinder.newSetBinder(binder, ExprMacroTable.ExprMacro.class)
|
||||||
|
.addBinding()
|
||||||
|
.to(clazz);
|
||||||
|
}
|
||||||
|
}
|
|
@ -37,6 +37,7 @@ import io.druid.guice.AnnouncerModule;
|
||||||
import io.druid.guice.CoordinatorDiscoveryModule;
|
import io.druid.guice.CoordinatorDiscoveryModule;
|
||||||
import io.druid.guice.DruidProcessingModule;
|
import io.druid.guice.DruidProcessingModule;
|
||||||
import io.druid.guice.DruidSecondaryModule;
|
import io.druid.guice.DruidSecondaryModule;
|
||||||
|
import io.druid.guice.ExpressionModule;
|
||||||
import io.druid.guice.ExtensionsConfig;
|
import io.druid.guice.ExtensionsConfig;
|
||||||
import io.druid.guice.FirehoseModule;
|
import io.druid.guice.FirehoseModule;
|
||||||
import io.druid.guice.IndexingServiceDiscoveryModule;
|
import io.druid.guice.IndexingServiceDiscoveryModule;
|
||||||
|
@ -339,6 +340,7 @@ public class Initialization
|
||||||
new JettyServerModule(),
|
new JettyServerModule(),
|
||||||
new QueryableModule(),
|
new QueryableModule(),
|
||||||
new QueryRunnerFactoryModule(),
|
new QueryRunnerFactoryModule(),
|
||||||
|
new ExpressionModule(),
|
||||||
new DiscoveryModule(),
|
new DiscoveryModule(),
|
||||||
new ServerViewModule(),
|
new ServerViewModule(),
|
||||||
new MetadataConfigModule(),
|
new MetadataConfigModule(),
|
||||||
|
|
|
@ -29,7 +29,7 @@ import org.junit.Test;
|
||||||
*/
|
*/
|
||||||
public class BrokerSegmentWatcherConfigTest
|
public class BrokerSegmentWatcherConfigTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper MAPPER = TestHelper.getObjectMapper();
|
private static final ObjectMapper MAPPER = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testSerde() throws Exception
|
||||||
|
|
|
@ -43,7 +43,7 @@ public class LookupSnapshotTakerTest
|
||||||
{
|
{
|
||||||
@Rule
|
@Rule
|
||||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||||
private final ObjectMapper mapper = TestHelper.JSON_MAPPER;
|
private final ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
|
|
||||||
private LookupSnapshotTaker lookupSnapshotTaker;
|
private LookupSnapshotTaker lookupSnapshotTaker;
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package io.druid.segment.indexing;
|
package io.druid.segment.indexing;
|
||||||
|
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.InjectableValues;
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
@ -28,12 +27,12 @@ import io.druid.data.input.impl.DimensionsSpec;
|
||||||
import io.druid.data.input.impl.JSONParseSpec;
|
import io.druid.data.input.impl.JSONParseSpec;
|
||||||
import io.druid.data.input.impl.StringInputRowParser;
|
import io.druid.data.input.impl.StringInputRowParser;
|
||||||
import io.druid.data.input.impl.TimestampSpec;
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.java.util.common.IAE;
|
import io.druid.java.util.common.IAE;
|
||||||
import io.druid.java.util.common.granularity.DurationGranularity;
|
import io.druid.java.util.common.granularity.DurationGranularity;
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -44,13 +43,7 @@ import java.util.Map;
|
||||||
|
|
||||||
public class DataSchemaTest
|
public class DataSchemaTest
|
||||||
{
|
{
|
||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper = TestHelper.getJsonMapper();
|
||||||
|
|
||||||
public DataSchemaTest()
|
|
||||||
{
|
|
||||||
jsonMapper = new DefaultObjectMapper();
|
|
||||||
jsonMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDefaultExclusions() throws Exception
|
public void testDefaultExclusions() throws Exception
|
||||||
|
|
|
@ -53,7 +53,7 @@ public class RealtimeTuningConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\"type\":\"realtime\"}";
|
String jsonStr = "{\"type\":\"realtime\"}";
|
||||||
|
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue(
|
RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue(
|
||||||
mapper.writeValueAsString(
|
mapper.writeValueAsString(
|
||||||
mapper.readValue(
|
mapper.readValue(
|
||||||
|
@ -97,7 +97,7 @@ public class RealtimeTuningConfigTest
|
||||||
+ " \"alertTimeout\": 70\n"
|
+ " \"alertTimeout\": 70\n"
|
||||||
+ "}";
|
+ "}";
|
||||||
|
|
||||||
ObjectMapper mapper = TestHelper.getObjectMapper();
|
ObjectMapper mapper = TestHelper.getJsonMapper();
|
||||||
RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue(
|
RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue(
|
||||||
mapper.writeValueAsString(
|
mapper.writeValueAsString(
|
||||||
mapper.readValue(
|
mapper.readValue(
|
||||||
|
|
|
@ -114,7 +114,7 @@ public class FireDepartmentTest
|
||||||
TestHelper.getTestIndexIO(),
|
TestHelper.getTestIndexIO(),
|
||||||
MapCache.create(0),
|
MapCache.create(0),
|
||||||
NO_CACHE_CONFIG,
|
NO_CACHE_CONFIG,
|
||||||
TestHelper.getObjectMapper()
|
TestHelper.getJsonMapper()
|
||||||
|
|
||||||
),
|
),
|
||||||
null
|
null
|
||||||
|
|
|
@ -220,7 +220,7 @@ public class RealtimePlumberSchoolTest
|
||||||
TestHelper.getTestIndexIO(),
|
TestHelper.getTestIndexIO(),
|
||||||
MapCache.create(0),
|
MapCache.create(0),
|
||||||
FireDepartmentTest.NO_CACHE_CONFIG,
|
FireDepartmentTest.NO_CACHE_CONFIG,
|
||||||
TestHelper.getObjectMapper()
|
TestHelper.getJsonMapper()
|
||||||
);
|
);
|
||||||
|
|
||||||
metrics = new FireDepartmentMetrics();
|
metrics = new FireDepartmentMetrics();
|
||||||
|
|
|
@ -78,7 +78,6 @@ public class ApproxCountDistinctSqlAggregator implements SqlAggregator
|
||||||
Iterables.getOnlyElement(aggregateCall.getArgList())
|
Iterables.getOnlyElement(aggregateCall.getArgList())
|
||||||
);
|
);
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(
|
final RowExtraction rex = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowSignature.getRowOrder(),
|
rowSignature.getRowOrder(),
|
||||||
rexNode
|
rexNode
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package io.druid.sql.calcite.expression;
|
package io.druid.sql.calcite.expression;
|
||||||
|
|
||||||
import io.druid.query.extraction.StrlenExtractionFn;
|
import io.druid.query.extraction.StrlenExtractionFn;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.rex.RexCall;
|
import org.apache.calcite.rex.RexCall;
|
||||||
import org.apache.calcite.rex.RexNode;
|
import org.apache.calcite.rex.RexNode;
|
||||||
|
@ -39,7 +38,6 @@ public class CharacterLengthExtractionOperator implements SqlExtractionOperator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RowExtraction convert(
|
public RowExtraction convert(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -47,7 +45,6 @@ public class CharacterLengthExtractionOperator implements SqlExtractionOperator
|
||||||
{
|
{
|
||||||
final RexCall call = (RexCall) expression;
|
final RexCall call = (RexCall) expression;
|
||||||
final RowExtraction arg = Expressions.toRowExtraction(
|
final RowExtraction arg = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowOrder,
|
rowOrder,
|
||||||
call.getOperands().get(0)
|
call.getOperands().get(0)
|
||||||
|
|
|
@ -145,7 +145,6 @@ public class Expressions
|
||||||
* @return RowExtraction or null if not possible
|
* @return RowExtraction or null if not possible
|
||||||
*/
|
*/
|
||||||
public static RowExtraction toRowExtraction(
|
public static RowExtraction toRowExtraction(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -165,26 +164,26 @@ public class Expressions
|
||||||
&& operand.getType().getSqlTypeName() == SqlTypeName.TIMESTAMP) {
|
&& operand.getType().getSqlTypeName() == SqlTypeName.TIMESTAMP) {
|
||||||
// Handling casting TIMESTAMP to DATE by flooring to DAY.
|
// Handling casting TIMESTAMP to DATE by flooring to DAY.
|
||||||
return FloorExtractionOperator.applyTimestampFloor(
|
return FloorExtractionOperator.applyTimestampFloor(
|
||||||
toRowExtraction(operatorTable, plannerContext, rowOrder, operand),
|
toRowExtraction(plannerContext, rowOrder, operand),
|
||||||
TimeUnits.toQueryGranularity(TimeUnitRange.DAY, plannerContext.getTimeZone())
|
TimeUnits.toQueryGranularity(TimeUnitRange.DAY, plannerContext.getTimeZone())
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
// Ignore other casts.
|
// Ignore other casts.
|
||||||
// TODO(gianm): Probably not a good idea to ignore other CASTs like this.
|
// TODO(gianm): Probably not a good idea to ignore other CASTs like this.
|
||||||
return toRowExtraction(operatorTable, plannerContext, rowOrder, ((RexCall) expression).getOperands().get(0));
|
return toRowExtraction(plannerContext, rowOrder, ((RexCall) expression).getOperands().get(0));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// Try conversion using a SqlExtractionOperator.
|
// Try conversion using a SqlExtractionOperator.
|
||||||
final RowExtraction retVal;
|
final RowExtraction retVal;
|
||||||
|
|
||||||
if (expression instanceof RexCall) {
|
if (expression instanceof RexCall) {
|
||||||
final SqlExtractionOperator extractionOperator = operatorTable.lookupExtractionOperator(
|
final SqlExtractionOperator extractionOperator = plannerContext.getOperatorTable().lookupExtractionOperator(
|
||||||
expression.getKind(),
|
expression.getKind(),
|
||||||
((RexCall) expression).getOperator().getName()
|
((RexCall) expression).getOperator().getName()
|
||||||
);
|
);
|
||||||
|
|
||||||
retVal = extractionOperator != null
|
retVal = extractionOperator != null
|
||||||
? extractionOperator.convert(operatorTable, plannerContext, rowOrder, expression)
|
? extractionOperator.convert(plannerContext, rowOrder, expression)
|
||||||
: null;
|
: null;
|
||||||
} else {
|
} else {
|
||||||
retVal = null;
|
retVal = null;
|
||||||
|
@ -400,7 +399,6 @@ public class Expressions
|
||||||
* @param expression Calcite row expression
|
* @param expression Calcite row expression
|
||||||
*/
|
*/
|
||||||
public static DimFilter toFilter(
|
public static DimFilter toFilter(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final RowSignature rowSignature,
|
final RowSignature rowSignature,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -411,7 +409,7 @@ public class Expressions
|
||||||
|| expression.getKind() == SqlKind.NOT) {
|
|| expression.getKind() == SqlKind.NOT) {
|
||||||
final List<DimFilter> filters = Lists.newArrayList();
|
final List<DimFilter> filters = Lists.newArrayList();
|
||||||
for (final RexNode rexNode : ((RexCall) expression).getOperands()) {
|
for (final RexNode rexNode : ((RexCall) expression).getOperands()) {
|
||||||
final DimFilter nextFilter = toFilter(operatorTable, plannerContext, rowSignature, rexNode);
|
final DimFilter nextFilter = toFilter(plannerContext, rowSignature, rexNode);
|
||||||
if (nextFilter == null) {
|
if (nextFilter == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -428,7 +426,7 @@ public class Expressions
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// Handle filter conditions on everything else.
|
// Handle filter conditions on everything else.
|
||||||
return toLeafFilter(operatorTable, plannerContext, rowSignature, expression);
|
return toLeafFilter(plannerContext, rowSignature, expression);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -441,7 +439,6 @@ public class Expressions
|
||||||
* @param expression Calcite row expression
|
* @param expression Calcite row expression
|
||||||
*/
|
*/
|
||||||
private static DimFilter toLeafFilter(
|
private static DimFilter toLeafFilter(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final RowSignature rowSignature,
|
final RowSignature rowSignature,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -458,7 +455,6 @@ public class Expressions
|
||||||
if (kind == SqlKind.LIKE) {
|
if (kind == SqlKind.LIKE) {
|
||||||
final List<RexNode> operands = ((RexCall) expression).getOperands();
|
final List<RexNode> operands = ((RexCall) expression).getOperands();
|
||||||
final RowExtraction rex = toRowExtraction(
|
final RowExtraction rex = toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowSignature.getRowOrder(),
|
rowSignature.getRowOrder(),
|
||||||
operands.get(0)
|
operands.get(0)
|
||||||
|
@ -498,7 +494,7 @@ public class Expressions
|
||||||
}
|
}
|
||||||
|
|
||||||
// lhs must be translatable to a RowExtraction to be filterable
|
// lhs must be translatable to a RowExtraction to be filterable
|
||||||
final RowExtraction rex = toRowExtraction(operatorTable, plannerContext, rowSignature.getRowOrder(), lhs);
|
final RowExtraction rex = toRowExtraction(plannerContext, rowSignature.getRowOrder(), lhs);
|
||||||
if (rex == null || !rex.isFilterable(rowSignature)) {
|
if (rex == null || !rex.isFilterable(rowSignature)) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.sql.calcite.expression;
|
||||||
import io.druid.java.util.common.granularity.Granularity;
|
import io.druid.java.util.common.granularity.Granularity;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.extraction.TimeFormatExtractionFn;
|
import io.druid.query.extraction.TimeFormatExtractionFn;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.avatica.util.TimeUnitRange;
|
import org.apache.calcite.avatica.util.TimeUnitRange;
|
||||||
import org.apache.calcite.rex.RexCall;
|
import org.apache.calcite.rex.RexCall;
|
||||||
|
@ -43,7 +42,6 @@ public class ExtractExtractionOperator implements SqlExtractionOperator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RowExtraction convert(
|
public RowExtraction convert(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -55,7 +53,7 @@ public class ExtractExtractionOperator implements SqlExtractionOperator
|
||||||
final TimeUnitRange timeUnit = (TimeUnitRange) flag.getValue();
|
final TimeUnitRange timeUnit = (TimeUnitRange) flag.getValue();
|
||||||
final RexNode expr = call.getOperands().get(1);
|
final RexNode expr = call.getOperands().get(1);
|
||||||
|
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(operatorTable, plannerContext, rowOrder, expr);
|
final RowExtraction rex = Expressions.toRowExtraction(plannerContext, rowOrder, expr);
|
||||||
if (rex == null) {
|
if (rex == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.sql.calcite.expression;
|
||||||
|
|
||||||
import io.druid.java.util.common.granularity.Granularity;
|
import io.druid.java.util.common.granularity.Granularity;
|
||||||
import io.druid.query.extraction.BucketExtractionFn;
|
import io.druid.query.extraction.BucketExtractionFn;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.avatica.util.TimeUnitRange;
|
import org.apache.calcite.avatica.util.TimeUnitRange;
|
||||||
import org.apache.calcite.rex.RexCall;
|
import org.apache.calcite.rex.RexCall;
|
||||||
|
@ -60,7 +59,6 @@ public class FloorExtractionOperator implements SqlExtractionOperator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RowExtraction convert(
|
public RowExtraction convert(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -69,7 +67,7 @@ public class FloorExtractionOperator implements SqlExtractionOperator
|
||||||
final RexCall call = (RexCall) expression;
|
final RexCall call = (RexCall) expression;
|
||||||
final RexNode arg = call.getOperands().get(0);
|
final RexNode arg = call.getOperands().get(0);
|
||||||
|
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(operatorTable, plannerContext, rowOrder, arg);
|
final RowExtraction rex = Expressions.toRowExtraction(plannerContext, rowOrder, arg);
|
||||||
if (rex == null) {
|
if (rex == null) {
|
||||||
return null;
|
return null;
|
||||||
} else if (call.getOperands().size() == 1) {
|
} else if (call.getOperands().size() == 1) {
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.sql.calcite.expression;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import io.druid.query.lookup.LookupReferencesManager;
|
import io.druid.query.lookup.LookupReferencesManager;
|
||||||
import io.druid.query.lookup.RegisteredLookupExtractionFn;
|
import io.druid.query.lookup.RegisteredLookupExtractionFn;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.rex.RexCall;
|
import org.apache.calcite.rex.RexCall;
|
||||||
import org.apache.calcite.rex.RexLiteral;
|
import org.apache.calcite.rex.RexLiteral;
|
||||||
|
@ -58,7 +57,6 @@ public class LookupExtractionOperator implements SqlExtractionOperator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RowExtraction convert(
|
public RowExtraction convert(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -66,7 +64,6 @@ public class LookupExtractionOperator implements SqlExtractionOperator
|
||||||
{
|
{
|
||||||
final RexCall call = (RexCall) expression;
|
final RexCall call = (RexCall) expression;
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(
|
final RowExtraction rex = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowOrder,
|
rowOrder,
|
||||||
call.getOperands().get(0)
|
call.getOperands().get(0)
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.sql.calcite.expression;
|
||||||
|
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.extraction.RegexDimExtractionFn;
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.rex.RexCall;
|
import org.apache.calcite.rex.RexCall;
|
||||||
import org.apache.calcite.rex.RexLiteral;
|
import org.apache.calcite.rex.RexLiteral;
|
||||||
|
@ -49,7 +48,6 @@ public class RegexpExtractExtractionOperator implements SqlExtractionOperator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RowExtraction convert(
|
public RowExtraction convert(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -57,7 +55,6 @@ public class RegexpExtractExtractionOperator implements SqlExtractionOperator
|
||||||
{
|
{
|
||||||
final RexCall call = (RexCall) expression;
|
final RexCall call = (RexCall) expression;
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(
|
final RowExtraction rex = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowOrder,
|
rowOrder,
|
||||||
call.getOperands().get(0)
|
call.getOperands().get(0)
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package io.druid.sql.calcite.expression;
|
package io.druid.sql.calcite.expression;
|
||||||
|
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.rex.RexNode;
|
import org.apache.calcite.rex.RexNode;
|
||||||
import org.apache.calcite.sql.SqlFunction;
|
import org.apache.calcite.sql.SqlFunction;
|
||||||
|
@ -38,17 +37,15 @@ public interface SqlExtractionOperator
|
||||||
/**
|
/**
|
||||||
* Returns the Druid {@link RowExtraction} corresponding to a SQL {@code RexNode}.
|
* Returns the Druid {@link RowExtraction} corresponding to a SQL {@code RexNode}.
|
||||||
*
|
*
|
||||||
* @param operatorTable Operator table that can be used to convert sub-expressions
|
|
||||||
* @param plannerContext SQL planner context
|
* @param plannerContext SQL planner context
|
||||||
* @param rowOrder order of fields in the Druid rows to be extracted from
|
* @param rowOrder order of fields in the Druid rows to be extracted from
|
||||||
* @param expression expression meant to be applied on top of the table
|
* @param expression expression meant to be applied on top of the table
|
||||||
*
|
*
|
||||||
* @return (columnName, extractionFn) or null
|
* @return (columnName, extractionFn) or null
|
||||||
*
|
*
|
||||||
* @see ExpressionConversion#convert(ExpressionConverter, PlannerContext, List, RexNode)
|
* @see Expressions#toRowExtraction(PlannerContext, List, RexNode)
|
||||||
*/
|
*/
|
||||||
RowExtraction convert(
|
RowExtraction convert(
|
||||||
DruidOperatorTable operatorTable,
|
|
||||||
PlannerContext plannerContext,
|
PlannerContext plannerContext,
|
||||||
List<String> rowOrder,
|
List<String> rowOrder,
|
||||||
RexNode expression
|
RexNode expression
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package io.druid.sql.calcite.expression;
|
package io.druid.sql.calcite.expression;
|
||||||
|
|
||||||
import io.druid.query.extraction.SubstringDimExtractionFn;
|
import io.druid.query.extraction.SubstringDimExtractionFn;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import org.apache.calcite.rex.RexCall;
|
import org.apache.calcite.rex.RexCall;
|
||||||
import org.apache.calcite.rex.RexLiteral;
|
import org.apache.calcite.rex.RexLiteral;
|
||||||
|
@ -40,7 +39,6 @@ public class SubstringExtractionOperator implements SqlExtractionOperator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RowExtraction convert(
|
public RowExtraction convert(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode expression
|
final RexNode expression
|
||||||
|
@ -48,7 +46,6 @@ public class SubstringExtractionOperator implements SqlExtractionOperator
|
||||||
{
|
{
|
||||||
final RexCall call = (RexCall) expression;
|
final RexCall call = (RexCall) expression;
|
||||||
final RowExtraction arg = Expressions.toRowExtraction(
|
final RowExtraction arg = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
rowOrder,
|
rowOrder,
|
||||||
call.getOperands().get(0)
|
call.getOperands().get(0)
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.sql.calcite.planner;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.calcite.DataContext;
|
import org.apache.calcite.DataContext;
|
||||||
import org.apache.calcite.adapter.java.JavaTypeFactory;
|
import org.apache.calcite.adapter.java.JavaTypeFactory;
|
||||||
import org.apache.calcite.linq4j.QueryProvider;
|
import org.apache.calcite.linq4j.QueryProvider;
|
||||||
|
@ -40,17 +41,23 @@ public class PlannerContext
|
||||||
public static final String CTX_SQL_CURRENT_TIMESTAMP = "sqlCurrentTimestamp";
|
public static final String CTX_SQL_CURRENT_TIMESTAMP = "sqlCurrentTimestamp";
|
||||||
public static final String CTX_SQL_TIME_ZONE = "sqlTimeZone";
|
public static final String CTX_SQL_TIME_ZONE = "sqlTimeZone";
|
||||||
|
|
||||||
|
private final DruidOperatorTable operatorTable;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
private final PlannerConfig plannerConfig;
|
private final PlannerConfig plannerConfig;
|
||||||
private final DateTime localNow;
|
private final DateTime localNow;
|
||||||
private final long queryStartTimeMillis;
|
private final long queryStartTimeMillis;
|
||||||
private final Map<String, Object> queryContext;
|
private final Map<String, Object> queryContext;
|
||||||
|
|
||||||
private PlannerContext(
|
private PlannerContext(
|
||||||
|
final DruidOperatorTable operatorTable,
|
||||||
|
final ExprMacroTable macroTable,
|
||||||
final PlannerConfig plannerConfig,
|
final PlannerConfig plannerConfig,
|
||||||
final DateTime localNow,
|
final DateTime localNow,
|
||||||
final Map<String, Object> queryContext
|
final Map<String, Object> queryContext
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
this.operatorTable = operatorTable;
|
||||||
|
this.macroTable = macroTable;
|
||||||
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
|
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
|
||||||
this.queryContext = queryContext != null ? ImmutableMap.copyOf(queryContext) : ImmutableMap.<String, Object>of();
|
this.queryContext = queryContext != null ? ImmutableMap.copyOf(queryContext) : ImmutableMap.<String, Object>of();
|
||||||
this.localNow = Preconditions.checkNotNull(localNow, "localNow");
|
this.localNow = Preconditions.checkNotNull(localNow, "localNow");
|
||||||
|
@ -58,6 +65,8 @@ public class PlannerContext
|
||||||
}
|
}
|
||||||
|
|
||||||
public static PlannerContext create(
|
public static PlannerContext create(
|
||||||
|
final DruidOperatorTable operatorTable,
|
||||||
|
final ExprMacroTable macroTable,
|
||||||
final PlannerConfig plannerConfig,
|
final PlannerConfig plannerConfig,
|
||||||
final Map<String, Object> queryContext
|
final Map<String, Object> queryContext
|
||||||
)
|
)
|
||||||
|
@ -85,7 +94,23 @@ public class PlannerContext
|
||||||
timeZone = DateTimeZone.UTC;
|
timeZone = DateTimeZone.UTC;
|
||||||
}
|
}
|
||||||
|
|
||||||
return new PlannerContext(plannerConfig.withOverrides(queryContext), utcNow.withZone(timeZone), queryContext);
|
return new PlannerContext(
|
||||||
|
operatorTable,
|
||||||
|
macroTable,
|
||||||
|
plannerConfig.withOverrides(queryContext),
|
||||||
|
utcNow.withZone(timeZone),
|
||||||
|
queryContext
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DruidOperatorTable getOperatorTable()
|
||||||
|
{
|
||||||
|
return operatorTable;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ExprMacroTable getExprMacroTable()
|
||||||
|
{
|
||||||
|
return macroTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
public PlannerConfig getPlannerConfig()
|
public PlannerConfig getPlannerConfig()
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.sql.calcite.planner;
|
package io.druid.sql.calcite.planner;
|
||||||
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.query.QuerySegmentWalker;
|
import io.druid.query.QuerySegmentWalker;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
import io.druid.server.initialization.ServerConfig;
|
||||||
import io.druid.sql.calcite.rel.QueryMaker;
|
import io.druid.sql.calcite.rel.QueryMaker;
|
||||||
|
@ -41,9 +42,18 @@ import java.util.Map;
|
||||||
|
|
||||||
public class PlannerFactory
|
public class PlannerFactory
|
||||||
{
|
{
|
||||||
|
private static final SqlParser.Config PARSER_CONFIG = SqlParser
|
||||||
|
.configBuilder()
|
||||||
|
.setCaseSensitive(true)
|
||||||
|
.setUnquotedCasing(Casing.UNCHANGED)
|
||||||
|
.setQuotedCasing(Casing.UNCHANGED)
|
||||||
|
.setQuoting(Quoting.DOUBLE_QUOTE)
|
||||||
|
.build();
|
||||||
|
|
||||||
private final SchemaPlus rootSchema;
|
private final SchemaPlus rootSchema;
|
||||||
private final QuerySegmentWalker walker;
|
private final QuerySegmentWalker walker;
|
||||||
private final DruidOperatorTable operatorTable;
|
private final DruidOperatorTable operatorTable;
|
||||||
|
private final ExprMacroTable macroTable;
|
||||||
private final PlannerConfig plannerConfig;
|
private final PlannerConfig plannerConfig;
|
||||||
private final ServerConfig serverConfig;
|
private final ServerConfig serverConfig;
|
||||||
|
|
||||||
|
@ -52,6 +62,7 @@ public class PlannerFactory
|
||||||
final SchemaPlus rootSchema,
|
final SchemaPlus rootSchema,
|
||||||
final QuerySegmentWalker walker,
|
final QuerySegmentWalker walker,
|
||||||
final DruidOperatorTable operatorTable,
|
final DruidOperatorTable operatorTable,
|
||||||
|
final ExprMacroTable macroTable,
|
||||||
final PlannerConfig plannerConfig,
|
final PlannerConfig plannerConfig,
|
||||||
final ServerConfig serverConfig
|
final ServerConfig serverConfig
|
||||||
)
|
)
|
||||||
|
@ -59,29 +70,23 @@ public class PlannerFactory
|
||||||
this.rootSchema = rootSchema;
|
this.rootSchema = rootSchema;
|
||||||
this.walker = walker;
|
this.walker = walker;
|
||||||
this.operatorTable = operatorTable;
|
this.operatorTable = operatorTable;
|
||||||
|
this.macroTable = macroTable;
|
||||||
this.plannerConfig = plannerConfig;
|
this.plannerConfig = plannerConfig;
|
||||||
this.serverConfig = serverConfig;
|
this.serverConfig = serverConfig;
|
||||||
}
|
}
|
||||||
|
|
||||||
public DruidPlanner createPlanner(final Map<String, Object> queryContext)
|
public DruidPlanner createPlanner(final Map<String, Object> queryContext)
|
||||||
{
|
{
|
||||||
final PlannerContext plannerContext = PlannerContext.create(plannerConfig, queryContext);
|
final PlannerContext plannerContext = PlannerContext.create(operatorTable, macroTable, plannerConfig, queryContext);
|
||||||
final QueryMaker queryMaker = new QueryMaker(walker, plannerContext, serverConfig);
|
final QueryMaker queryMaker = new QueryMaker(walker, plannerContext, serverConfig);
|
||||||
final FrameworkConfig frameworkConfig = Frameworks
|
final FrameworkConfig frameworkConfig = Frameworks
|
||||||
.newConfigBuilder()
|
.newConfigBuilder()
|
||||||
.parserConfig(
|
.parserConfig(PARSER_CONFIG)
|
||||||
SqlParser.configBuilder()
|
|
||||||
.setCaseSensitive(true)
|
|
||||||
.setUnquotedCasing(Casing.UNCHANGED)
|
|
||||||
.setQuotedCasing(Casing.UNCHANGED)
|
|
||||||
.setQuoting(Quoting.DOUBLE_QUOTE)
|
|
||||||
.build()
|
|
||||||
)
|
|
||||||
.defaultSchema(rootSchema)
|
.defaultSchema(rootSchema)
|
||||||
.traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE)
|
.traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE)
|
||||||
.convertletTable(new DruidConvertletTable(plannerContext))
|
.convertletTable(new DruidConvertletTable(plannerContext))
|
||||||
.operatorTable(operatorTable)
|
.operatorTable(operatorTable)
|
||||||
.programs(Rules.programs(queryMaker, operatorTable))
|
.programs(Rules.programs(plannerContext, queryMaker))
|
||||||
.executor(new RexExecutorImpl(Schemas.createDataContext(null)))
|
.executor(new RexExecutorImpl(Schemas.createDataContext(null)))
|
||||||
.context(Contexts.EMPTY_CONTEXT)
|
.context(Contexts.EMPTY_CONTEXT)
|
||||||
.typeSystem(RelDataTypeSystem.DEFAULT)
|
.typeSystem(RelDataTypeSystem.DEFAULT)
|
||||||
|
|
|
@ -159,42 +159,42 @@ public class Rules
|
||||||
// No instantiation.
|
// No instantiation.
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<Program> programs(final QueryMaker queryMaker, final DruidOperatorTable operatorTable)
|
public static List<Program> programs(final PlannerContext plannerContext, final QueryMaker queryMaker)
|
||||||
{
|
{
|
||||||
return ImmutableList.of(
|
return ImmutableList.of(
|
||||||
Programs.ofRules(druidConventionRuleSet(queryMaker, operatorTable)),
|
Programs.ofRules(druidConventionRuleSet(plannerContext, queryMaker)),
|
||||||
Programs.ofRules(bindableConventionRuleSet(queryMaker, operatorTable))
|
Programs.ofRules(bindableConventionRuleSet(plannerContext, queryMaker))
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static List<RelOptRule> druidConventionRuleSet(
|
private static List<RelOptRule> druidConventionRuleSet(
|
||||||
final QueryMaker queryMaker,
|
final PlannerContext plannerContext,
|
||||||
final DruidOperatorTable operatorTable
|
final QueryMaker queryMaker
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return ImmutableList.<RelOptRule>builder()
|
return ImmutableList.<RelOptRule>builder()
|
||||||
.addAll(baseRuleSet(queryMaker, operatorTable))
|
.addAll(baseRuleSet(plannerContext, queryMaker))
|
||||||
.add(DruidRelToDruidRule.instance())
|
.add(DruidRelToDruidRule.instance())
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static List<RelOptRule> bindableConventionRuleSet(
|
private static List<RelOptRule> bindableConventionRuleSet(
|
||||||
final QueryMaker queryMaker,
|
final PlannerContext plannerContext,
|
||||||
final DruidOperatorTable operatorTable
|
final QueryMaker queryMaker
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return ImmutableList.<RelOptRule>builder()
|
return ImmutableList.<RelOptRule>builder()
|
||||||
.addAll(baseRuleSet(queryMaker, operatorTable))
|
.addAll(baseRuleSet(plannerContext, queryMaker))
|
||||||
.addAll(Bindables.RULES)
|
.addAll(Bindables.RULES)
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static List<RelOptRule> baseRuleSet(
|
private static List<RelOptRule> baseRuleSet(
|
||||||
final QueryMaker queryMaker,
|
final PlannerContext plannerContext,
|
||||||
final DruidOperatorTable operatorTable
|
final QueryMaker queryMaker
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final PlannerConfig plannerConfig = queryMaker.getPlannerContext().getPlannerConfig();
|
final PlannerConfig plannerConfig = plannerContext.getPlannerConfig();
|
||||||
final ImmutableList.Builder<RelOptRule> rules = ImmutableList.builder();
|
final ImmutableList.Builder<RelOptRule> rules = ImmutableList.builder();
|
||||||
|
|
||||||
// Calcite rules.
|
// Calcite rules.
|
||||||
|
@ -217,15 +217,15 @@ public class Rules
|
||||||
rules.add(SortCollapseRule.instance());
|
rules.add(SortCollapseRule.instance());
|
||||||
|
|
||||||
// Druid-specific rules.
|
// Druid-specific rules.
|
||||||
rules.add(new DruidTableScanRule(queryMaker));
|
rules.add(new DruidTableScanRule(plannerContext, queryMaker));
|
||||||
rules.add(new DruidFilterRule(operatorTable));
|
rules.add(new DruidFilterRule());
|
||||||
|
|
||||||
if (plannerConfig.getMaxSemiJoinRowsInMemory() > 0) {
|
if (plannerConfig.getMaxSemiJoinRowsInMemory() > 0) {
|
||||||
rules.add(DruidSemiJoinRule.instance());
|
rules.add(DruidSemiJoinRule.instance());
|
||||||
}
|
}
|
||||||
|
|
||||||
rules.addAll(SelectRules.rules(operatorTable));
|
rules.addAll(SelectRules.rules());
|
||||||
rules.addAll(GroupByRules.rules(operatorTable));
|
rules.addAll(GroupByRules.rules());
|
||||||
|
|
||||||
return rules.build();
|
return rules.build();
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.query.QueryDataSource;
|
import io.druid.query.QueryDataSource;
|
||||||
import io.druid.query.groupby.GroupByQuery;
|
import io.druid.query.groupby.GroupByQuery;
|
||||||
import io.druid.sql.calcite.filtration.Filtration;
|
import io.druid.sql.calcite.filtration.Filtration;
|
||||||
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import io.druid.sql.calcite.table.DruidTable;
|
import io.druid.sql.calcite.table.DruidTable;
|
||||||
import io.druid.sql.calcite.table.RowSignature;
|
import io.druid.sql.calcite.table.RowSignature;
|
||||||
import org.apache.calcite.interpreter.BindableConvention;
|
import org.apache.calcite.interpreter.BindableConvention;
|
||||||
|
@ -73,6 +74,7 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
||||||
final RelOptCluster cluster,
|
final RelOptCluster cluster,
|
||||||
final RelOptTable table,
|
final RelOptTable table,
|
||||||
final DruidTable druidTable,
|
final DruidTable druidTable,
|
||||||
|
final PlannerContext plannerContext,
|
||||||
final QueryMaker queryMaker
|
final QueryMaker queryMaker
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -193,8 +195,7 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
||||||
pw.item("filter", filtration.getDimFilter());
|
pw.item("filter", filtration.getDimFilter());
|
||||||
}
|
}
|
||||||
if (queryBuilder.getSelectProjection() != null) {
|
if (queryBuilder.getSelectProjection() != null) {
|
||||||
pw.item("selectDimensions", queryBuilder.getSelectProjection().getDimensions());
|
pw.item("selectProjection", queryBuilder.getSelectProjection());
|
||||||
pw.item("selectMetrics", queryBuilder.getSelectProjection().getMetrics());
|
|
||||||
}
|
}
|
||||||
if (queryBuilder.getGrouping() != null) {
|
if (queryBuilder.getGrouping() != null) {
|
||||||
pw.item("dimensions", queryBuilder.getGrouping().getDimensions());
|
pw.item("dimensions", queryBuilder.getGrouping().getDimensions());
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.sql.calcite.rule;
|
||||||
|
|
||||||
import io.druid.query.filter.DimFilter;
|
import io.druid.query.filter.DimFilter;
|
||||||
import io.druid.sql.calcite.expression.Expressions;
|
import io.druid.sql.calcite.expression.Expressions;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.rel.DruidRel;
|
import io.druid.sql.calcite.rel.DruidRel;
|
||||||
import org.apache.calcite.plan.RelOptRule;
|
import org.apache.calcite.plan.RelOptRule;
|
||||||
import org.apache.calcite.plan.RelOptRuleCall;
|
import org.apache.calcite.plan.RelOptRuleCall;
|
||||||
|
@ -29,12 +28,9 @@ import org.apache.calcite.rel.core.Filter;
|
||||||
|
|
||||||
public class DruidFilterRule extends RelOptRule
|
public class DruidFilterRule extends RelOptRule
|
||||||
{
|
{
|
||||||
private final DruidOperatorTable operatorTable;
|
public DruidFilterRule()
|
||||||
|
|
||||||
public DruidFilterRule(final DruidOperatorTable operatorTable)
|
|
||||||
{
|
{
|
||||||
super(operand(Filter.class, operand(DruidRel.class, none())));
|
super(operand(Filter.class, operand(DruidRel.class, none())));
|
||||||
this.operatorTable = operatorTable;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -50,14 +46,16 @@ public class DruidFilterRule extends RelOptRule
|
||||||
}
|
}
|
||||||
|
|
||||||
final DimFilter dimFilter = Expressions.toFilter(
|
final DimFilter dimFilter = Expressions.toFilter(
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext(),
|
druidRel.getPlannerContext(),
|
||||||
druidRel.getSourceRowSignature(),
|
druidRel.getSourceRowSignature(),
|
||||||
filter.getCondition()
|
filter.getCondition()
|
||||||
);
|
);
|
||||||
if (dimFilter != null) {
|
if (dimFilter != null) {
|
||||||
call.transformTo(
|
call.transformTo(
|
||||||
druidRel.withQueryBuilder(druidRel.getQueryBuilder().withFilter(dimFilter))
|
druidRel.withQueryBuilder(
|
||||||
|
druidRel.getQueryBuilder()
|
||||||
|
.withFilter(dimFilter)
|
||||||
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.sql.calcite.rule;
|
package io.druid.sql.calcite.rule;
|
||||||
|
|
||||||
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import io.druid.sql.calcite.rel.DruidQueryRel;
|
import io.druid.sql.calcite.rel.DruidQueryRel;
|
||||||
import io.druid.sql.calcite.rel.QueryMaker;
|
import io.druid.sql.calcite.rel.QueryMaker;
|
||||||
import io.druid.sql.calcite.table.DruidTable;
|
import io.druid.sql.calcite.table.DruidTable;
|
||||||
|
@ -29,13 +30,16 @@ import org.apache.calcite.rel.logical.LogicalTableScan;
|
||||||
|
|
||||||
public class DruidTableScanRule extends RelOptRule
|
public class DruidTableScanRule extends RelOptRule
|
||||||
{
|
{
|
||||||
|
private final PlannerContext plannerContext;
|
||||||
private final QueryMaker queryMaker;
|
private final QueryMaker queryMaker;
|
||||||
|
|
||||||
public DruidTableScanRule(
|
public DruidTableScanRule(
|
||||||
|
final PlannerContext plannerContext,
|
||||||
final QueryMaker queryMaker
|
final QueryMaker queryMaker
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(operand(LogicalTableScan.class, any()));
|
super(operand(LogicalTableScan.class, any()));
|
||||||
|
this.plannerContext = plannerContext;
|
||||||
this.queryMaker = queryMaker;
|
this.queryMaker = queryMaker;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -47,7 +51,7 @@ public class DruidTableScanRule extends RelOptRule
|
||||||
final DruidTable druidTable = table.unwrap(DruidTable.class);
|
final DruidTable druidTable = table.unwrap(DruidTable.class);
|
||||||
if (druidTable != null) {
|
if (druidTable != null) {
|
||||||
call.transformTo(
|
call.transformTo(
|
||||||
DruidQueryRel.fullScan(scan.getCluster(), table, druidTable, queryMaker)
|
DruidQueryRel.fullScan(scan.getCluster(), table, druidTable, plannerContext, queryMaker)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||||
|
@ -53,7 +54,6 @@ import io.druid.sql.calcite.expression.Expressions;
|
||||||
import io.druid.sql.calcite.expression.RowExtraction;
|
import io.druid.sql.calcite.expression.RowExtraction;
|
||||||
import io.druid.sql.calcite.filtration.Filtration;
|
import io.druid.sql.calcite.filtration.Filtration;
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import io.druid.sql.calcite.rel.DruidNestedGroupBy;
|
import io.druid.sql.calcite.rel.DruidNestedGroupBy;
|
||||||
import io.druid.sql.calcite.rel.DruidRel;
|
import io.druid.sql.calcite.rel.DruidRel;
|
||||||
|
@ -87,14 +87,14 @@ public class GroupByRules
|
||||||
// No instantiation.
|
// No instantiation.
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<RelOptRule> rules(final DruidOperatorTable operatorTable)
|
public static List<RelOptRule> rules()
|
||||||
{
|
{
|
||||||
return ImmutableList.of(
|
return ImmutableList.of(
|
||||||
new DruidAggregateRule(operatorTable),
|
new DruidAggregateRule(),
|
||||||
new DruidAggregateProjectRule(operatorTable),
|
new DruidAggregateProjectRule(),
|
||||||
new DruidAggregateProjectFilterRule(operatorTable),
|
new DruidAggregateProjectFilterRule(),
|
||||||
new DruidGroupByPostAggregationRule(),
|
new DruidGroupByPostAggregationRule(),
|
||||||
new DruidGroupByHavingRule(operatorTable),
|
new DruidGroupByHavingRule(),
|
||||||
new DruidGroupByLimitRule()
|
new DruidGroupByLimitRule()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -116,13 +116,12 @@ public class GroupByRules
|
||||||
}
|
}
|
||||||
|
|
||||||
public static FieldOrExpression fromRexNode(
|
public static FieldOrExpression fromRexNode(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext,
|
||||||
final List<String> rowOrder,
|
final List<String> rowOrder,
|
||||||
final RexNode rexNode
|
final RexNode rexNode
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(operatorTable, plannerContext, rowOrder, rexNode);
|
final RowExtraction rex = Expressions.toRowExtraction(plannerContext, rowOrder, rexNode);
|
||||||
if (rex != null && rex.getExtractionFn() == null) {
|
if (rex != null && rex.getExtractionFn() == null) {
|
||||||
// This was a simple field access.
|
// This was a simple field access.
|
||||||
return fieldName(rex.getColumn());
|
return fieldName(rex.getColumn());
|
||||||
|
@ -160,12 +159,9 @@ public class GroupByRules
|
||||||
|
|
||||||
public static class DruidAggregateRule extends RelOptRule
|
public static class DruidAggregateRule extends RelOptRule
|
||||||
{
|
{
|
||||||
private final DruidOperatorTable operatorTable;
|
private DruidAggregateRule()
|
||||||
|
|
||||||
private DruidAggregateRule(final DruidOperatorTable operatorTable)
|
|
||||||
{
|
{
|
||||||
super(operand(Aggregate.class, operand(DruidRel.class, none())));
|
super(operand(Aggregate.class, operand(DruidRel.class, none())));
|
||||||
this.operatorTable = operatorTable;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -185,9 +181,7 @@ public class GroupByRules
|
||||||
druidRel,
|
druidRel,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
aggregate,
|
aggregate
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext().getPlannerConfig().isUseApproximateCountDistinct()
|
|
||||||
);
|
);
|
||||||
if (newDruidRel != null) {
|
if (newDruidRel != null) {
|
||||||
call.transformTo(newDruidRel);
|
call.transformTo(newDruidRel);
|
||||||
|
@ -197,12 +191,9 @@ public class GroupByRules
|
||||||
|
|
||||||
public static class DruidAggregateProjectRule extends RelOptRule
|
public static class DruidAggregateProjectRule extends RelOptRule
|
||||||
{
|
{
|
||||||
private final DruidOperatorTable operatorTable;
|
private DruidAggregateProjectRule()
|
||||||
|
|
||||||
private DruidAggregateProjectRule(final DruidOperatorTable operatorTable)
|
|
||||||
{
|
{
|
||||||
super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, none()))));
|
super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, none()))));
|
||||||
this.operatorTable = operatorTable;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -224,9 +215,7 @@ public class GroupByRules
|
||||||
druidRel,
|
druidRel,
|
||||||
null,
|
null,
|
||||||
project,
|
project,
|
||||||
aggregate,
|
aggregate
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext().getPlannerConfig().isUseApproximateCountDistinct()
|
|
||||||
);
|
);
|
||||||
if (newDruidRel != null) {
|
if (newDruidRel != null) {
|
||||||
call.transformTo(newDruidRel);
|
call.transformTo(newDruidRel);
|
||||||
|
@ -236,12 +225,9 @@ public class GroupByRules
|
||||||
|
|
||||||
public static class DruidAggregateProjectFilterRule extends RelOptRule
|
public static class DruidAggregateProjectFilterRule extends RelOptRule
|
||||||
{
|
{
|
||||||
private final DruidOperatorTable operatorTable;
|
private DruidAggregateProjectFilterRule()
|
||||||
|
|
||||||
private DruidAggregateProjectFilterRule(final DruidOperatorTable operatorTable)
|
|
||||||
{
|
{
|
||||||
super(operand(Aggregate.class, operand(Project.class, operand(Filter.class, operand(DruidRel.class, none())))));
|
super(operand(Aggregate.class, operand(Project.class, operand(Filter.class, operand(DruidRel.class, none())))));
|
||||||
this.operatorTable = operatorTable;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -265,9 +251,7 @@ public class GroupByRules
|
||||||
druidRel,
|
druidRel,
|
||||||
filter,
|
filter,
|
||||||
project,
|
project,
|
||||||
aggregate,
|
aggregate
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext().getPlannerConfig().isUseApproximateCountDistinct()
|
|
||||||
);
|
);
|
||||||
if (newDruidRel != null) {
|
if (newDruidRel != null) {
|
||||||
call.transformTo(newDruidRel);
|
call.transformTo(newDruidRel);
|
||||||
|
@ -303,12 +287,9 @@ public class GroupByRules
|
||||||
|
|
||||||
public static class DruidGroupByHavingRule extends RelOptRule
|
public static class DruidGroupByHavingRule extends RelOptRule
|
||||||
{
|
{
|
||||||
private final DruidOperatorTable operatorTable;
|
private DruidGroupByHavingRule()
|
||||||
|
|
||||||
private DruidGroupByHavingRule(final DruidOperatorTable operatorTable)
|
|
||||||
{
|
{
|
||||||
super(operand(Filter.class, operand(DruidRel.class, none())));
|
super(operand(Filter.class, operand(DruidRel.class, none())));
|
||||||
this.operatorTable = operatorTable;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -323,7 +304,7 @@ public class GroupByRules
|
||||||
{
|
{
|
||||||
final Filter postFilter = call.rel(0);
|
final Filter postFilter = call.rel(0);
|
||||||
final DruidRel druidRel = call.rel(1);
|
final DruidRel druidRel = call.rel(1);
|
||||||
final DruidRel newDruidRel = GroupByRules.applyHaving(operatorTable, druidRel, postFilter);
|
final DruidRel newDruidRel = GroupByRules.applyHaving(druidRel, postFilter);
|
||||||
if (newDruidRel != null) {
|
if (newDruidRel != null) {
|
||||||
call.transformTo(newDruidRel);
|
call.transformTo(newDruidRel);
|
||||||
}
|
}
|
||||||
|
@ -379,9 +360,7 @@ public class GroupByRules
|
||||||
final DruidRel druidRel,
|
final DruidRel druidRel,
|
||||||
final Filter filter0,
|
final Filter filter0,
|
||||||
final Project project0,
|
final Project project0,
|
||||||
final Aggregate aggregate,
|
final Aggregate aggregate
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final boolean approximateCountDistinct
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkState(canApplyAggregate(druidRel, filter0, project0, aggregate), "Cannot applyAggregate.");
|
Preconditions.checkState(canApplyAggregate(druidRel, filter0, project0, aggregate), "Cannot applyAggregate.");
|
||||||
|
@ -400,7 +379,6 @@ public class GroupByRules
|
||||||
final DimFilter filter;
|
final DimFilter filter;
|
||||||
if (filter0 != null) {
|
if (filter0 != null) {
|
||||||
filter = Expressions.toFilter(
|
filter = Expressions.toFilter(
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext(),
|
druidRel.getPlannerContext(),
|
||||||
sourceRowSignature,
|
sourceRowSignature,
|
||||||
filter0.getCondition()
|
filter0.getCondition()
|
||||||
|
@ -444,7 +422,6 @@ public class GroupByRules
|
||||||
} else {
|
} else {
|
||||||
final RexNode rexNode = Expressions.fromFieldAccess(sourceRowSignature, project, i);
|
final RexNode rexNode = Expressions.fromFieldAccess(sourceRowSignature, project, i);
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(
|
final RowExtraction rex = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext(),
|
druidRel.getPlannerContext(),
|
||||||
sourceRowSignature.getRowOrder(),
|
sourceRowSignature.getRowOrder(),
|
||||||
rexNode
|
rexNode
|
||||||
|
@ -480,10 +457,8 @@ public class GroupByRules
|
||||||
sourceRowSignature,
|
sourceRowSignature,
|
||||||
project,
|
project,
|
||||||
aggCall,
|
aggCall,
|
||||||
operatorTable,
|
|
||||||
aggregations,
|
aggregations,
|
||||||
i,
|
i
|
||||||
approximateCountDistinct
|
|
||||||
);
|
);
|
||||||
|
|
||||||
if (aggregation == null) {
|
if (aggregation == null) {
|
||||||
|
@ -601,7 +576,6 @@ public class GroupByRules
|
||||||
* @return new rel, or null if the filter cannot be applied
|
* @return new rel, or null if the filter cannot be applied
|
||||||
*/
|
*/
|
||||||
private static DruidRel applyHaving(
|
private static DruidRel applyHaving(
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final DruidRel druidRel,
|
final DruidRel druidRel,
|
||||||
final Filter postFilter
|
final Filter postFilter
|
||||||
)
|
)
|
||||||
|
@ -609,7 +583,6 @@ public class GroupByRules
|
||||||
Preconditions.checkState(canApplyHaving(druidRel), "Cannot applyHaving.");
|
Preconditions.checkState(canApplyHaving(druidRel), "Cannot applyHaving.");
|
||||||
|
|
||||||
final DimFilter dimFilter = Expressions.toFilter(
|
final DimFilter dimFilter = Expressions.toFilter(
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext(),
|
druidRel.getPlannerContext(),
|
||||||
druidRel.getOutputRowSignature(),
|
druidRel.getOutputRowSignature(),
|
||||||
postFilter.getCondition()
|
postFilter.getCondition()
|
||||||
|
@ -746,10 +719,8 @@ public class GroupByRules
|
||||||
final RowSignature sourceRowSignature,
|
final RowSignature sourceRowSignature,
|
||||||
final Project project,
|
final Project project,
|
||||||
final AggregateCall call,
|
final AggregateCall call,
|
||||||
final DruidOperatorTable operatorTable,
|
|
||||||
final List<Aggregation> existingAggregations,
|
final List<Aggregation> existingAggregations,
|
||||||
final int aggNumber,
|
final int aggNumber
|
||||||
final boolean approximateCountDistinct
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final List<DimFilter> filters = Lists.newArrayList();
|
final List<DimFilter> filters = Lists.newArrayList();
|
||||||
|
@ -766,7 +737,7 @@ public class GroupByRules
|
||||||
}
|
}
|
||||||
|
|
||||||
final RexNode expression = project.getChildExps().get(call.filterArg);
|
final RexNode expression = project.getChildExps().get(call.filterArg);
|
||||||
final DimFilter filter = Expressions.toFilter(operatorTable, plannerContext, sourceRowSignature, expression);
|
final DimFilter filter = Expressions.toFilter(plannerContext, sourceRowSignature, expression);
|
||||||
if (filter == null) {
|
if (filter == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -779,16 +750,22 @@ public class GroupByRules
|
||||||
return Aggregation.create(new CountAggregatorFactory(name)).filter(makeFilter(filters, sourceRowSignature));
|
return Aggregation.create(new CountAggregatorFactory(name)).filter(makeFilter(filters, sourceRowSignature));
|
||||||
} else if (kind == SqlKind.COUNT && call.isDistinct()) {
|
} else if (kind == SqlKind.COUNT && call.isDistinct()) {
|
||||||
// COUNT(DISTINCT x)
|
// COUNT(DISTINCT x)
|
||||||
return approximateCountDistinct ? APPROX_COUNT_DISTINCT.toDruidAggregation(
|
if (plannerContext.getPlannerConfig().isUseApproximateCountDistinct()) {
|
||||||
name,
|
return APPROX_COUNT_DISTINCT.toDruidAggregation(
|
||||||
sourceRowSignature,
|
name,
|
||||||
operatorTable,
|
sourceRowSignature,
|
||||||
plannerContext,
|
plannerContext.getOperatorTable(),
|
||||||
existingAggregations,
|
plannerContext,
|
||||||
project,
|
existingAggregations,
|
||||||
call,
|
project,
|
||||||
makeFilter(filters, sourceRowSignature)
|
call,
|
||||||
) : null;
|
makeFilter(filters, sourceRowSignature)
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
// Can't do exact distinct count as an aggregator. Return null here and give Calcite's rules a chance
|
||||||
|
// to rewrite this query as a nested groupBy.
|
||||||
|
return null;
|
||||||
|
}
|
||||||
} else if (kind == SqlKind.COUNT
|
} else if (kind == SqlKind.COUNT
|
||||||
|| kind == SqlKind.SUM
|
|| kind == SqlKind.SUM
|
||||||
|| kind == SqlKind.SUM0
|
|| kind == SqlKind.SUM0
|
||||||
|
@ -801,7 +778,7 @@ public class GroupByRules
|
||||||
|
|
||||||
final int inputField = Iterables.getOnlyElement(call.getArgList());
|
final int inputField = Iterables.getOnlyElement(call.getArgList());
|
||||||
final RexNode rexNode = Expressions.fromFieldAccess(sourceRowSignature, project, inputField);
|
final RexNode rexNode = Expressions.fromFieldAccess(sourceRowSignature, project, inputField);
|
||||||
final FieldOrExpression foe = FieldOrExpression.fromRexNode(operatorTable, plannerContext, rowOrder, rexNode);
|
final FieldOrExpression foe = FieldOrExpression.fromRexNode(plannerContext, rowOrder, rexNode);
|
||||||
|
|
||||||
if (foe != null) {
|
if (foe != null) {
|
||||||
input = foe;
|
input = foe;
|
||||||
|
@ -821,7 +798,6 @@ public class GroupByRules
|
||||||
|
|
||||||
// Operand 1: Filter
|
// Operand 1: Filter
|
||||||
final DimFilter filter = Expressions.toFilter(
|
final DimFilter filter = Expressions.toFilter(
|
||||||
operatorTable,
|
|
||||||
plannerContext,
|
plannerContext,
|
||||||
sourceRowSignature,
|
sourceRowSignature,
|
||||||
caseCall.getOperands().get(0)
|
caseCall.getOperands().get(0)
|
||||||
|
@ -849,7 +825,7 @@ public class GroupByRules
|
||||||
|| (kind == SqlKind.SUM
|
|| (kind == SqlKind.SUM
|
||||||
&& Calcites.isIntLiteral(arg2)
|
&& Calcites.isIntLiteral(arg2)
|
||||||
&& RexLiteral.intValue(arg2) == 0) /* Case A2 */) {
|
&& RexLiteral.intValue(arg2) == 0) /* Case A2 */) {
|
||||||
input = FieldOrExpression.fromRexNode(operatorTable, plannerContext, rowOrder, arg1);
|
input = FieldOrExpression.fromRexNode(plannerContext, rowOrder, arg1);
|
||||||
if (input == null) {
|
if (input == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -874,6 +850,7 @@ public class GroupByRules
|
||||||
final Aggregation retVal;
|
final Aggregation retVal;
|
||||||
final String fieldName = input.getFieldName();
|
final String fieldName = input.getFieldName();
|
||||||
final String expression = input.getExpression();
|
final String expression = input.getExpression();
|
||||||
|
final ExprMacroTable macroTable = plannerContext.getExprMacroTable();
|
||||||
|
|
||||||
final boolean isLong = SqlTypeName.INT_TYPES.contains(outputType)
|
final boolean isLong = SqlTypeName.INT_TYPES.contains(outputType)
|
||||||
|| SqlTypeName.TIMESTAMP == outputType
|
|| SqlTypeName.TIMESTAMP == outputType
|
||||||
|
@ -881,22 +858,22 @@ public class GroupByRules
|
||||||
|
|
||||||
if (kind == SqlKind.SUM || kind == SqlKind.SUM0) {
|
if (kind == SqlKind.SUM || kind == SqlKind.SUM0) {
|
||||||
retVal = isLong
|
retVal = isLong
|
||||||
? Aggregation.create(new LongSumAggregatorFactory(name, fieldName, expression))
|
? Aggregation.create(new LongSumAggregatorFactory(name, fieldName, expression, macroTable))
|
||||||
: Aggregation.create(new DoubleSumAggregatorFactory(name, fieldName, expression));
|
: Aggregation.create(new DoubleSumAggregatorFactory(name, fieldName, expression, macroTable));
|
||||||
} else if (kind == SqlKind.MIN) {
|
} else if (kind == SqlKind.MIN) {
|
||||||
retVal = isLong
|
retVal = isLong
|
||||||
? Aggregation.create(new LongMinAggregatorFactory(name, fieldName, expression))
|
? Aggregation.create(new LongMinAggregatorFactory(name, fieldName, expression, macroTable))
|
||||||
: Aggregation.create(new DoubleMinAggregatorFactory(name, fieldName, expression));
|
: Aggregation.create(new DoubleMinAggregatorFactory(name, fieldName, expression, macroTable));
|
||||||
} else if (kind == SqlKind.MAX) {
|
} else if (kind == SqlKind.MAX) {
|
||||||
retVal = isLong
|
retVal = isLong
|
||||||
? Aggregation.create(new LongMaxAggregatorFactory(name, fieldName, expression))
|
? Aggregation.create(new LongMaxAggregatorFactory(name, fieldName, expression, macroTable))
|
||||||
: Aggregation.create(new DoubleMaxAggregatorFactory(name, fieldName, expression));
|
: Aggregation.create(new DoubleMaxAggregatorFactory(name, fieldName, expression, macroTable));
|
||||||
} else if (kind == SqlKind.AVG) {
|
} else if (kind == SqlKind.AVG) {
|
||||||
final String sumName = aggInternalName(aggNumber, "sum");
|
final String sumName = aggInternalName(aggNumber, "sum");
|
||||||
final String countName = aggInternalName(aggNumber, "count");
|
final String countName = aggInternalName(aggNumber, "count");
|
||||||
final AggregatorFactory sum = isLong
|
final AggregatorFactory sum = isLong
|
||||||
? new LongSumAggregatorFactory(sumName, fieldName, expression)
|
? new LongSumAggregatorFactory(sumName, fieldName, expression, macroTable)
|
||||||
: new DoubleSumAggregatorFactory(sumName, fieldName, expression);
|
: new DoubleSumAggregatorFactory(sumName, fieldName, expression, macroTable);
|
||||||
final AggregatorFactory count = new CountAggregatorFactory(countName);
|
final AggregatorFactory count = new CountAggregatorFactory(countName);
|
||||||
retVal = Aggregation.create(
|
retVal = Aggregation.create(
|
||||||
ImmutableList.of(sum, count),
|
ImmutableList.of(sum, count),
|
||||||
|
@ -918,11 +895,13 @@ public class GroupByRules
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// Not a built-in aggregator, check operator table.
|
// Not a built-in aggregator, check operator table.
|
||||||
final SqlAggregator sqlAggregator = operatorTable.lookupAggregator(call.getAggregation().getName());
|
final SqlAggregator sqlAggregator = plannerContext.getOperatorTable()
|
||||||
|
.lookupAggregator(call.getAggregation().getName());
|
||||||
|
|
||||||
return sqlAggregator != null ? sqlAggregator.toDruidAggregation(
|
return sqlAggregator != null ? sqlAggregator.toDruidAggregation(
|
||||||
name,
|
name,
|
||||||
sourceRowSignature,
|
sourceRowSignature,
|
||||||
operatorTable,
|
plannerContext.getOperatorTable(),
|
||||||
plannerContext,
|
plannerContext,
|
||||||
existingAggregations,
|
existingAggregations,
|
||||||
project,
|
project,
|
||||||
|
|
|
@ -31,7 +31,6 @@ import io.druid.segment.column.ValueType;
|
||||||
import io.druid.sql.calcite.expression.Expressions;
|
import io.druid.sql.calcite.expression.Expressions;
|
||||||
import io.druid.sql.calcite.expression.RowExtraction;
|
import io.druid.sql.calcite.expression.RowExtraction;
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
|
||||||
import io.druid.sql.calcite.rel.DruidRel;
|
import io.druid.sql.calcite.rel.DruidRel;
|
||||||
import io.druid.sql.calcite.rel.SelectProjection;
|
import io.druid.sql.calcite.rel.SelectProjection;
|
||||||
import io.druid.sql.calcite.table.RowSignature;
|
import io.druid.sql.calcite.table.RowSignature;
|
||||||
|
@ -51,22 +50,19 @@ public class SelectRules
|
||||||
// No instantiation.
|
// No instantiation.
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<RelOptRule> rules(final DruidOperatorTable operatorTable)
|
public static List<RelOptRule> rules()
|
||||||
{
|
{
|
||||||
return ImmutableList.of(
|
return ImmutableList.of(
|
||||||
new DruidSelectProjectionRule(operatorTable),
|
new DruidSelectProjectionRule(),
|
||||||
new DruidSelectSortRule()
|
new DruidSelectSortRule()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
static class DruidSelectProjectionRule extends RelOptRule
|
static class DruidSelectProjectionRule extends RelOptRule
|
||||||
{
|
{
|
||||||
private final DruidOperatorTable operatorTable;
|
public DruidSelectProjectionRule()
|
||||||
|
|
||||||
public DruidSelectProjectionRule(final DruidOperatorTable operatorTable)
|
|
||||||
{
|
{
|
||||||
super(operand(Project.class, operand(DruidRel.class, none())));
|
super(operand(Project.class, operand(DruidRel.class, none())));
|
||||||
this.operatorTable = operatorTable;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -97,7 +93,6 @@ public class SelectRules
|
||||||
for (int i = 0; i < project.getRowType().getFieldCount(); i++) {
|
for (int i = 0; i < project.getRowType().getFieldCount(); i++) {
|
||||||
final RexNode rexNode = project.getChildExps().get(i);
|
final RexNode rexNode = project.getChildExps().get(i);
|
||||||
final RowExtraction rex = Expressions.toRowExtraction(
|
final RowExtraction rex = Expressions.toRowExtraction(
|
||||||
operatorTable,
|
|
||||||
druidRel.getPlannerContext(),
|
druidRel.getPlannerContext(),
|
||||||
sourceRowSignature.getRowOrder(),
|
sourceRowSignature.getRowOrder(),
|
||||||
rexNode
|
rexNode
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue