diff --git a/README.md b/README.md index 12228c473f..ea22e4669e 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,7 @@ The following projects have been merged into this repository as separate folders * [SQL CLI](https://github.com/opendistro-for-elasticsearch/sql/tree/master/sql-cli) * [SQL JDBC](https://github.com/opendistro-for-elasticsearch/sql/tree/master/sql-jdbc) * [SQL ODBC](https://github.com/opendistro-for-elasticsearch/sql/tree/master/sql-odbc) -* [SQL Workbench](https://github.com/opendistro-for-elasticsearch/sql/tree/master/sql-workbench) +* [SQL Workbench](https://github.com/opendistro-for-elasticsearch/sql/tree/master/workbench) ## Documentation diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/Analyzer.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/Analyzer.java index a3c3f37365..054ae0e004 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/Analyzer.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/Analyzer.java @@ -25,11 +25,13 @@ import com.amazon.opendistroforelasticsearch.sql.ast.expression.Let; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Literal; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Map; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedArgument; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Aggregation; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Dedupe; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Eval; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Head; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation; @@ -46,10 +48,12 @@ import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalAggregation; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalDedupe; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalEval; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalFilter; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalHead; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalProject; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRareTopN; @@ -80,6 +84,8 @@ public class Analyzer extends AbstractNodeVisitor private final SelectExpressionAnalyzer selectExpressionAnalyzer; + private final NamedExpressionAnalyzer namedExpressionAnalyzer; + private final StorageEngine storageEngine; /** @@ -91,6 +97,7 @@ public Analyzer( this.expressionAnalyzer = expressionAnalyzer; this.storageEngine = storageEngine; this.selectExpressionAnalyzer = new SelectExpressionAnalyzer(expressionAnalyzer); + this.namedExpressionAnalyzer = new NamedExpressionAnalyzer(expressionAnalyzer); } public LogicalPlan analyze(UnresolvedPlan unresolved, AnalysisContext context) { @@ -153,25 +160,27 @@ public LogicalPlan visitRename(Rename node, AnalysisContext context) { @Override public LogicalPlan visitAggregation(Aggregation node, AnalysisContext context) { final LogicalPlan child = node.getChild().get(0).accept(this, context); - ImmutableList.Builder aggregatorBuilder = new ImmutableList.Builder<>(); + ImmutableList.Builder aggregatorBuilder = new ImmutableList.Builder<>(); for (UnresolvedExpression expr : node.getAggExprList()) { - aggregatorBuilder.add((Aggregator) expressionAnalyzer.analyze(expr, context)); + NamedExpression aggExpr = namedExpressionAnalyzer.analyze(expr, context); + aggregatorBuilder + .add(new NamedAggregator(aggExpr.getName(), (Aggregator) aggExpr.getDelegated())); } - ImmutableList aggregators = aggregatorBuilder.build(); + ImmutableList aggregators = aggregatorBuilder.build(); - ImmutableList.Builder groupbyBuilder = new ImmutableList.Builder<>(); + ImmutableList.Builder groupbyBuilder = new ImmutableList.Builder<>(); for (UnresolvedExpression expr : node.getGroupExprList()) { - groupbyBuilder.add(expressionAnalyzer.analyze(expr, context)); + groupbyBuilder.add(namedExpressionAnalyzer.analyze(expr, context)); } - ImmutableList groupBys = groupbyBuilder.build(); + ImmutableList groupBys = groupbyBuilder.build(); // new context context.push(); TypeEnvironment newEnv = context.peek(); aggregators.forEach(aggregator -> newEnv.define(new Symbol(Namespace.FIELD_NAME, - aggregator.toString()), aggregator.type())); + aggregator.getName()), aggregator.type())); groupBys.forEach(group -> newEnv.define(new Symbol(Namespace.FIELD_NAME, - group.toString()), group.type())); + group.getName()), group.type())); return new LogicalAggregation(child, aggregators, groupBys); } @@ -311,6 +320,23 @@ public LogicalPlan visitDedupe(Dedupe node, AnalysisContext context) { consecutive); } + /** + * Build {@link LogicalHead}. + */ + public LogicalPlan visitHead(Head node, AnalysisContext context) { + LogicalPlan child = node.getChild().get(0).accept(this, context); + List options = node.getOptions(); + Boolean keeplast = (Boolean) getOptionAsLiteral(options, 0).getValue(); + Expression whileExpr = expressionAnalyzer.analyze(options.get(1).getValue(), context); + Integer number = (Integer) getOptionAsLiteral(options, 2).getValue(); + + return new LogicalHead(child, keeplast, whileExpr, number); + } + + private static Literal getOptionAsLiteral(List options, int optionIdx) { + return (Literal) options.get(optionIdx).getValue(); + } + @Override public LogicalPlan visitValues(Values node, AnalysisContext context) { List> values = node.getValues(); diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzer.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzer.java index 7b14f3a21b..3aec63a6be 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzer.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzer.java @@ -25,7 +25,6 @@ import com.amazon.opendistroforelasticsearch.sql.ast.expression.Field; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Function; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Interval; -import com.amazon.opendistroforelasticsearch.sql.ast.expression.IntervalUnit; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Literal; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Not; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Or; diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/NamedExpressionAnalyzer.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/NamedExpressionAnalyzer.java new file mode 100644 index 0000000000..b7ac75519f --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/NamedExpressionAnalyzer.java @@ -0,0 +1,61 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.analysis; + +import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.Alias; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.QualifiedName; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import lombok.RequiredArgsConstructor; + +/** + * Analyze the Alias node in the {@link AnalysisContext} to construct the list of + * {@link NamedExpression}. + */ +@RequiredArgsConstructor +public class NamedExpressionAnalyzer extends + AbstractNodeVisitor { + private final ExpressionAnalyzer expressionAnalyzer; + + /** + * Analyze Select fields. + */ + public NamedExpression analyze(UnresolvedExpression expression, + AnalysisContext analysisContext) { + return expression.accept(this, analysisContext); + } + + @Override + public NamedExpression visitAlias(Alias node, AnalysisContext context) { + return DSL.named( + unqualifiedNameIfFieldOnly(node, context), + node.getDelegated().accept(expressionAnalyzer, context), + node.getAlias()); + } + + private String unqualifiedNameIfFieldOnly(Alias node, AnalysisContext context) { + UnresolvedExpression selectItem = node.getDelegated(); + if (selectItem instanceof QualifiedName) { + QualifierAnalyzer qualifierAnalyzer = new QualifierAnalyzer(context); + return qualifierAnalyzer.unqualified((QualifiedName) selectItem); + } + return node.getName(); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzer.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzer.java index c49cbca552..ed40b830d3 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzer.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzer.java @@ -20,9 +20,11 @@ import com.amazon.opendistroforelasticsearch.sql.analysis.symbol.Namespace; import com.amazon.opendistroforelasticsearch.sql.analysis.symbol.Symbol; import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.AggregateFunction; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Alias; import com.amazon.opendistroforelasticsearch.sql.ast.expression.AllFields; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Field; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.Function; import com.amazon.opendistroforelasticsearch.sql.ast.expression.QualifiedName; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; @@ -67,23 +69,42 @@ public List visitField(Field node, AnalysisContext context) { @Override public List visitAlias(Alias node, AnalysisContext context) { - Expression expr = referenceIfSymbolDefined(node.getDelegated(), context); + Expression expr = referenceIfSymbolDefined(node, context); return Collections.singletonList(DSL.named( unqualifiedNameIfFieldOnly(node, context), expr, node.getAlias())); } - private Expression referenceIfSymbolDefined(UnresolvedExpression expr, + /** + * The Alias could be + * 1. SELECT name, AVG(age) FROM s BY name -> + * Project(Alias("name", expr), Alias("AVG(age)", aggExpr)) + * Agg(Alias("AVG(age)", aggExpr)) + * 2. SELECT length(name), AVG(age) FROM s BY length(name) + * Project(Alias("name", expr), Alias("AVG(age)", aggExpr)) + * Agg(Alias("AVG(age)", aggExpr)) + * 3. SELECT length(name) as l, AVG(age) FROM s BY l + * Project(Alias("name", expr, l), Alias("AVG(age)", aggExpr)) + * Agg(Alias("AVG(age)", aggExpr), Alias("length(name)", groupExpr)) + */ + private Expression referenceIfSymbolDefined(Alias expr, AnalysisContext context) { + UnresolvedExpression delegatedExpr = expr.getDelegated(); try { - // Since resolved aggregator.toString() is used as symbol name, unresolved expression - // needs to be analyzed too to get toString() name for consistency - String symbolName = expressionAnalyzer.analyze(expr, context).toString(); - ExprType type = context.peek().resolve(new Symbol(Namespace.FIELD_NAME, symbolName)); - return DSL.ref(symbolName, type); + if ((delegatedExpr instanceof AggregateFunction) + || (delegatedExpr instanceof Function)) { + ExprType type = context.peek().resolve(new Symbol(Namespace.FIELD_NAME, expr.getName())); + return DSL.ref(expr.getName(), type); + } else { + // Since resolved aggregator.toString() is used as symbol name, unresolved expression + // needs to be analyzed too to get toString() name for consistency + String symbolName = expressionAnalyzer.analyze(delegatedExpr, context).toString(); + ExprType type = context.peek().resolve(new Symbol(Namespace.FIELD_NAME, symbolName)); + return DSL.ref(symbolName, type); + } } catch (SemanticCheckException e) { - return expr.accept(expressionAnalyzer, context); + return delegatedExpr.accept(expressionAnalyzer, context); } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/AbstractNodeVisitor.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/AbstractNodeVisitor.java index ac099b5f94..a6b640bdb8 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/AbstractNodeVisitor.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/AbstractNodeVisitor.java @@ -33,12 +33,14 @@ import com.amazon.opendistroforelasticsearch.sql.ast.expression.Not; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Or; import com.amazon.opendistroforelasticsearch.sql.ast.expression.QualifiedName; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedArgument; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedAttribute; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Xor; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Aggregation; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Dedupe; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Eval; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Head; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation; @@ -179,6 +181,10 @@ public T visitDedupe(Dedupe node, C context) { return visitChildren(node, context); } + public T visitHead(Head node, C context) { + return visitChildren(node, context); + } + public T visitRareTopN(RareTopN node, C context) { return visitChildren(node, context); } @@ -198,4 +204,8 @@ public T visitAllFields(AllFields node, C context) { public T visitInterval(Interval node, C context) { return visitChildren(node, context); } + + public T visitUnresolvedArgument(UnresolvedArgument node, C context) { + return visitChildren(node, context); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/dsl/AstDSL.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/dsl/AstDSL.java index 2d6b70920c..8d04fcb3ac 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/dsl/AstDSL.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/dsl/AstDSL.java @@ -33,6 +33,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.expression.Not; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Or; import com.amazon.opendistroforelasticsearch.sql.ast.expression.QualifiedName; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedArgument; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedAttribute; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Xor; @@ -40,6 +41,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.Dedupe; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Eval; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Head; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN.CommandType; @@ -214,6 +216,10 @@ public static Argument argument(String argName, Literal argValue) { return new Argument(argName, argValue); } + public static UnresolvedArgument unresolvedArg(String argName, UnresolvedExpression argValue) { + return new UnresolvedArgument(argName, argValue); + } + public static UnresolvedExpression field(UnresolvedExpression field) { return new Field((QualifiedName) field); } @@ -254,6 +260,10 @@ public static List exprList(Argument... exprList) { return Arrays.asList(exprList); } + public static List unresolvedArgList(UnresolvedArgument... exprList) { + return Arrays.asList(exprList); + } + public static List defaultFieldsArgs() { return exprList(argument("exclude", booleanLiteral(false))); } @@ -299,6 +309,20 @@ public static Dedupe dedupe(UnresolvedPlan input, List options, Field. return new Dedupe(input, options, Arrays.asList(fields)); } + public static Head head(UnresolvedPlan input, List options) { + return new Head(input, options); + } + + /** + * Default Head Command Args. + */ + public static List defaultHeadArgs() { + return unresolvedArgList( + unresolvedArg("keeplast", booleanLiteral(true)), + unresolvedArg("whileExpr", booleanLiteral(true)), + unresolvedArg("number", intLiteral(10))); + } + public static List defaultTopArgs() { return exprList(argument("noOfResults", intLiteral(10))); } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Function.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Function.java index 1732734512..d0edb4a21a 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Function.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Function.java @@ -18,17 +18,17 @@ import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.RequiredArgsConstructor; -import lombok.ToString; + /** * Expression node of scalar function. * Params include function name (@funcName) and function arguments (@funcArgs) */ @Getter -@ToString @EqualsAndHashCode(callSuper = false) @RequiredArgsConstructor public class Function extends UnresolvedExpression { @@ -44,4 +44,12 @@ public List getChild() { public R accept(AbstractNodeVisitor nodeVisitor, C context) { return nodeVisitor.visitFunction(this, context); } + + @Override + public String toString() { + return String.format("%s(%s)", funcName, + funcArgs.stream() + .map(Object::toString) + .collect(Collectors.joining(", "))); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Literal.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Literal.java index 5a0f60175c..d76c8a8a34 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Literal.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/Literal.java @@ -29,7 +29,6 @@ * literal data type (@type) which can be selected from {@link DataType}. */ @Getter -@ToString @EqualsAndHashCode(callSuper = false) @RequiredArgsConstructor public class Literal extends UnresolvedExpression { @@ -46,4 +45,9 @@ public List getChild() { public R accept(AbstractNodeVisitor nodeVisitor, C context) { return nodeVisitor.visitLiteral(this, context); } + + @Override + public String toString() { + return value.toString(); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/UnresolvedArgument.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/UnresolvedArgument.java new file mode 100644 index 0000000000..471686c429 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/expression/UnresolvedArgument.java @@ -0,0 +1,49 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.ast.expression; + +import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; +import java.util.Arrays; +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; + +/** + * Argument. + */ +@Getter +@ToString +@EqualsAndHashCode(callSuper = false) +public class UnresolvedArgument extends UnresolvedExpression { + private final String argName; + private final UnresolvedExpression value; + + public UnresolvedArgument(String argName, UnresolvedExpression value) { + this.argName = argName; + this.value = value; + } + + @Override + public List getChild() { + return Arrays.asList(value); + } + + @Override + public R accept(AbstractNodeVisitor nodeVisitor, C context) { + return nodeVisitor.visitUnresolvedArgument(this, context); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Head.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Head.java new file mode 100644 index 0000000000..89a5b36e67 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Head.java @@ -0,0 +1,58 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.ast.tree; + +import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedArgument; +import com.google.common.collect.ImmutableList; +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.Setter; +import lombok.ToString; + +/** + * AST node represent Head operation. + */ +@Getter +@Setter +@ToString +@EqualsAndHashCode(callSuper = false) +@RequiredArgsConstructor +@AllArgsConstructor +public class Head extends UnresolvedPlan { + + private UnresolvedPlan child; + private final List options; + + @Override + public Head attach(UnresolvedPlan child) { + this.child = child; + return this; + } + + @Override + public List getChild() { + return ImmutableList.of(this.child); + } + + @Override + public T accept(AbstractNodeVisitor nodeVisitor, C context) { + return nodeVisitor.visitHead(this, context); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/RareTopN.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/RareTopN.java index 8f04d6bea1..33c6989966 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/RareTopN.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/RareTopN.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"). * You may not use this file except in compliance with the License. diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/ExecutionEngine.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/ExecutionEngine.java index d75379ee1e..87e9c1499b 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/ExecutionEngine.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/ExecutionEngine.java @@ -21,7 +21,10 @@ import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; import lombok.Data; +import lombok.RequiredArgsConstructor; /** * Execution engine that encapsulates execution details. @@ -36,6 +39,16 @@ public interface ExecutionEngine { */ void execute(PhysicalPlan plan, ResponseListener listener); + /** + * Explain physical plan and call back response listener. The reason why this has to + * be part of execution engine interface is that the physical plan probably needs to + * be executed to get more info for profiling, such as actual execution time, rows fetched etc. + * + * @param plan physical plan to explain + * @param listener response listener + */ + void explain(PhysicalPlan plan, ResponseListener listener); + /** * Data class that encapsulates ExprValue. */ @@ -57,4 +70,22 @@ public static class Column { } } + /** + * Data class that encapsulates explain result. This can help decouple core engine + * from concrete explain response format. + */ + @Data + class ExplainResponse { + private final ExplainResponseNode root; + } + + @AllArgsConstructor + @Data + @RequiredArgsConstructor + class ExplainResponseNode { + private final String name; + private Map description; + private List children; + } + } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/Explain.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/Explain.java new file mode 100644 index 0000000000..76ef42c776 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/Explain.java @@ -0,0 +1,181 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.executor; + +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponseNode; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.AggregationOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.DedupeOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.EvalOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.FilterOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.HeadOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.ProjectOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.RareTopNOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.RemoveOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.RenameOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.SortOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.ValuesOperator; +import com.amazon.opendistroforelasticsearch.sql.storage.TableScanOperator; +import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Visitor that explains a physical plan to JSON format. + */ +public class Explain extends PhysicalPlanNodeVisitor + implements Function { + + @Override + public ExplainResponse apply(PhysicalPlan plan) { + return new ExplainResponse(plan.accept(this, null)); + } + + @Override + public ExplainResponseNode visitProject(ProjectOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "fields", node.getProjectList().toString()))); + } + + @Override + public ExplainResponseNode visitFilter(FilterOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "conditions", node.getConditions().toString()))); + } + + @Override + public ExplainResponseNode visitSort(SortOperator node, Object context) { + Map> sortListDescription = + node.getSortList() + .stream() + .collect(Collectors.toMap( + p -> p.getRight().toString(), + p -> ImmutableMap.of( + "sortOrder", p.getLeft().getSortOrder().toString(), + "nullOrder", p.getLeft().getNullOrder().toString()))); + + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "count", node.getCount(), + "sortList", sortListDescription))); + } + + @Override + public ExplainResponseNode visitTableScan(TableScanOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "request", node.toString()))); + } + + @Override + public ExplainResponseNode visitAggregation(AggregationOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "aggregators", node.getAggregatorList().toString(), + "groupBy", node.getGroupByExprList().toString()))); + } + + @Override + public ExplainResponseNode visitRename(RenameOperator node, Object context) { + Map renameMappingDescription = + node.getMapping() + .entrySet() + .stream() + .collect(Collectors.toMap( + e -> e.getKey().toString(), + e -> e.getValue().toString())); + + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "mapping", renameMappingDescription))); + } + + @Override + public ExplainResponseNode visitRemove(RemoveOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "removeList", node.getRemoveList().toString()))); + } + + @Override + public ExplainResponseNode visitEval(EvalOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "expressions", convertPairListToMap(node.getExpressionList())))); + } + + @Override + public ExplainResponseNode visitDedupe(DedupeOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "dedupeList", node.getDedupeList().toString(), + "allowedDuplication", node.getAllowedDuplication(), + "keepEmpty", node.getKeepEmpty(), + "consecutive", node.getConsecutive()))); + } + + @Override + public ExplainResponseNode visitRareTopN(RareTopNOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "commandType", node.getCommandType(), + "noOfResults", node.getNoOfResults(), + "fields", node.getFieldExprList().toString(), + "groupBy", node.getGroupByExprList().toString() + ))); + } + + @Override + public ExplainResponseNode visitHead(HeadOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "keepLast", node.getKeepLast(), + "whileExpr", node.getWhileExpr().toString(), + "number", node.getNumber() + ))); + } + + @Override + public ExplainResponseNode visitValues(ValuesOperator node, Object context) { + return explain(node, context, explainNode -> explainNode.setDescription(ImmutableMap.of( + "values", node.getValues()))); + } + + protected ExplainResponseNode explain(PhysicalPlan node, Object context, + Consumer doExplain) { + ExplainResponseNode explainNode = new ExplainResponseNode(getOperatorName(node)); + + List children = new ArrayList<>(); + for (PhysicalPlan child : node.getChild()) { + children.add(child.accept(this, context)); + } + explainNode.setChildren(children); + + doExplain.accept(explainNode); + return explainNode; + } + + private String getOperatorName(PhysicalPlan node) { + return node.getClass().getSimpleName(); + } + + private Map convertPairListToMap(List> pairs) { + return pairs.stream() + .collect(Collectors.toMap( + p -> p.getLeft().toString(), + p -> p.getRight().toString())); + } + +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/DSL.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/DSL.java index edb1748f6e..cc2c5e21cf 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/DSL.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/DSL.java @@ -20,6 +20,7 @@ import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; import java.util.Arrays; @@ -104,6 +105,10 @@ public static NamedExpression named(String name, Expression expression, String a return new NamedExpression(name, expression, alias); } + public static NamedAggregator named(String name, Aggregator aggregator) { + return new NamedAggregator(name, aggregator); + } + public FunctionExpression abs(Expression... expressions) { return function(BuiltinFunctionName.ABS, expressions); } @@ -256,6 +261,10 @@ public FunctionExpression timestamp(Expression... expressions) { return function(BuiltinFunctionName.TIMESTAMP, expressions); } + public FunctionExpression adddate(Expression... expressions) { + return function(BuiltinFunctionName.ADDDATE, expressions); + } + public FunctionExpression divide(Expression... expressions) { return function(BuiltinFunctionName.DIVIDE, expressions); } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitor.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitor.java index f2b1618357..4c3799c25f 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitor.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitor.java @@ -17,6 +17,7 @@ package com.amazon.opendistroforelasticsearch.sql.expression; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionImplementation; /** @@ -74,4 +75,7 @@ public T visitAggregator(Aggregator node, C context) { return visitChildren(node, context); } + public T visitNamedAggregator(NamedAggregator node, C context) { + return visitChildren(node, context); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/NamedExpression.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/NamedExpression.java index da38d4b9cc..17fd1225eb 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/NamedExpression.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/NamedExpression.java @@ -34,7 +34,6 @@ @AllArgsConstructor @EqualsAndHashCode @RequiredArgsConstructor -@ToString public class NamedExpression implements Expression { /** @@ -45,6 +44,7 @@ public class NamedExpression implements Expression { /** * Expression that being named. */ + @Getter private final Expression delegated; /** @@ -76,4 +76,9 @@ public T accept(ExpressionNodeVisitor visitor, C context) { return visitor.visitNamed(this, context); } + @Override + public String toString() { + return getName(); + } + } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/NamedAggregator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/NamedAggregator.java new file mode 100644 index 0000000000..912015594f --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/NamedAggregator.java @@ -0,0 +1,87 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.expression.aggregation; + +import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import com.google.common.base.Strings; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; + +/** + * NamedAggregator expression that represents expression with name. + * Please see more details in associated unresolved expression operator + * {@link com.amazon.opendistroforelasticsearch.sql.ast.expression.Alias}. + */ +@EqualsAndHashCode +public class NamedAggregator extends Aggregator { + + /** + * Aggregator name. + */ + private final String name; + + /** + * Aggregator that being named. + */ + private final Aggregator delegated; + + /** + * NamedAggregator. + * + * @param name name + * @param delegated delegated + */ + public NamedAggregator( + String name, + Aggregator delegated) { + super(delegated.getFunctionName(), delegated.getArguments(), delegated.returnType); + this.name = name; + this.delegated = delegated; + } + + @Override + public AggregationState create() { + return delegated.create(); + } + + @Override + public AggregationState iterate(BindingTuple tuple, AggregationState state) { + return delegated.iterate(tuple, state); + } + + /** + * Get expression name using name or its alias (if it's present). + * @return expression name + */ + public String getName() { + return name; + } + + @Override + public T accept(ExpressionNodeVisitor visitor, C context) { + return visitor.visitNamedAggregator(this, context); + } + + @Override + public String toString() { + return getName(); + } + +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunction.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunction.java index ea4716e9a2..6fdac81ddd 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunction.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunction.java @@ -17,10 +17,11 @@ package com.amazon.opendistroforelasticsearch.sql.expression.datetime; -import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.getStringValue; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DATE; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DATETIME; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTERVAL; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.LONG; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.STRING; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.TIME; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.TIMESTAMP; @@ -30,6 +31,7 @@ import static com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionDSL.nullMissingHandling; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprDateValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprDatetimeValue; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprIntegerValue; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprStringValue; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTimeValue; @@ -57,6 +59,7 @@ public void register(BuiltinFunctionRepository repository) { repository.register(dayOfMonth()); repository.register(time()); repository.register(timestamp()); + repository.register(adddate()); } /** @@ -78,7 +81,8 @@ private FunctionResolver date() { private FunctionResolver dayOfMonth() { return define(DAYOFMONTH.getName(), impl(nullMissingHandling(DateTimeFunction::exprDayOfMonth), - INTEGER, DATE) + INTEGER, DATE), + impl(nullMissingHandling(DateTimeFunction::exprDayOfMonth), INTEGER, STRING) ); } @@ -109,6 +113,27 @@ private FunctionResolver timestamp() { impl(nullMissingHandling(DateTimeFunction::exprTimestamp), TIMESTAMP, TIMESTAMP)); } + /** + * Specify a start date and add a temporal amount to the date. + * The return type depends on the date type and the interval unit. Detailed supported signatures: + * (DATE, DATETIME/TIMESTAMP, INTERVAL) -> DATETIME + * (DATE, LONG) -> DATE + * (DATETIME/TIMESTAMP, LONG) -> DATETIME + */ + private FunctionResolver adddate() { + return define(BuiltinFunctionName.ADDDATE.getName(), + impl(nullMissingHandling(DateTimeFunction::exprAddDateInterval), DATE, DATE, INTERVAL), + impl(nullMissingHandling(DateTimeFunction::exprAddDateInterval), DATETIME, DATE, INTERVAL), + impl(nullMissingHandling(DateTimeFunction::exprAddDateInterval), + DATETIME, DATETIME, INTERVAL), + impl(nullMissingHandling(DateTimeFunction::exprAddDateInterval), + DATETIME, TIMESTAMP, INTERVAL), + impl(nullMissingHandling(DateTimeFunction::exprAddDateDays), DATE, DATE, LONG), + impl(nullMissingHandling(DateTimeFunction::exprAddDateDays), DATETIME, DATETIME, LONG), + impl(nullMissingHandling(DateTimeFunction::exprAddDateDays), DATETIME, TIMESTAMP, LONG) + ); + } + /** * Date implementation for ExprValue. * @param exprValue ExprValue of Date type or String type. @@ -128,7 +153,11 @@ private ExprValue exprDate(ExprValue exprValue) { * @return ExprValue. */ private ExprValue exprDayOfMonth(ExprValue date) { - return new ExprIntegerValue(date.dateValue().getMonthValue()); + if (date instanceof ExprStringValue) { + return new ExprIntegerValue( + new ExprDateValue(date.stringValue()).dateValue().getDayOfMonth()); + } + return new ExprIntegerValue(date.dateValue().getDayOfMonth()); } /** @@ -156,4 +185,29 @@ private ExprValue exprTimestamp(ExprValue exprValue) { return new ExprTimestampValue(exprValue.timestampValue()); } } + + /** + * ADDDATE function implementation for ExprValue. + * + * @param date ExprValue of Date/Datetime/Timestamp type. + * @param expr ExprValue of Interval type, the temporal amount to add. + * @return Date/Datetime resulted from expr added to date. + */ + private ExprValue exprAddDateInterval(ExprValue date, ExprValue expr) { + return new ExprDatetimeValue(date.datetimeValue().plus(expr.intervalValue())); + } + + /** + * ADDDATE function implementation for ExprValue. + * + * @param date ExprValue of Date/Datetime/Timestamp type. + * @param days ExprValue of Long type, representing the number of days to add. + * @return Date/Datetime resulted from days added to date. + */ + private ExprValue exprAddDateDays(ExprValue date, ExprValue days) { + if (date instanceof ExprDateValue) { + return new ExprDateValue(date.dateValue().plusDays(days.longValue())); + } + return new ExprDatetimeValue(date.datetimeValue().plusDays(days.longValue())); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionName.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionName.java index eefc13c0d8..b853b9c855 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionName.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionName.java @@ -55,6 +55,7 @@ public enum BuiltinFunctionName { DAYOFMONTH(FunctionName.of("dayofmonth")), TIME(FunctionName.of("time")), TIMESTAMP(FunctionName.of("timestamp")), + ADDDATE(FunctionName.of("adddate")), /** * Text Functions. diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementor.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementor.java index 85f358e57b..e7c68df204 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementor.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementor.java @@ -20,6 +20,7 @@ import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalDedupe; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalEval; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalFilter; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalHead; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanNodeVisitor; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalProject; @@ -33,6 +34,7 @@ import com.amazon.opendistroforelasticsearch.sql.planner.physical.DedupeOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.EvalOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.FilterOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.HeadOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.physical.ProjectOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.RareTopNOperator; @@ -74,6 +76,15 @@ public PhysicalPlan visitDedupe(LogicalDedupe node, C context) { node.getConsecutive()); } + @Override + public PhysicalPlan visitHead(LogicalHead node, C context) { + return new HeadOperator( + visitChild(node, context), + node.getKeeplast(), + node.getWhileExpr(), + node.getNumber()); + } + @Override public PhysicalPlan visitProject(LogicalProject node, C context) { return new ProjectOperator(visitChild(node, context), node.getProjectList()); diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalAggregation.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalAggregation.java index ef89d91cef..e4d9b690ac 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalAggregation.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalAggregation.java @@ -15,8 +15,8 @@ package com.amazon.opendistroforelasticsearch.sql.planner.logical; -import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import java.util.Collections; import java.util.List; import lombok.EqualsAndHashCode; @@ -33,9 +33,9 @@ public class LogicalAggregation extends LogicalPlan { private final LogicalPlan child; @Getter - private final List aggregatorList; + private final List aggregatorList; @Getter - private final List groupByList; + private final List groupByList; @Override public List getChild() { diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalHead.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalHead.java new file mode 100644 index 0000000000..bb66f688f6 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalHead.java @@ -0,0 +1,47 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.planner.logical; + +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import java.util.Arrays; +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + + +@Getter +@ToString +@EqualsAndHashCode(callSuper = false) +@RequiredArgsConstructor +public class LogicalHead extends LogicalPlan { + + private final LogicalPlan child; + private final Boolean keeplast; + private final Expression whileExpr; + private final Integer number; + + @Override + public List getChild() { + return Arrays.asList(child); + } + + @Override + public R accept(LogicalPlanNodeVisitor visitor, C context) { + return visitor.visitHead(this, context); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanDSL.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanDSL.java index 99cd707577..a19ce69112 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanDSL.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanDSL.java @@ -21,7 +21,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.LiteralExpression; import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; -import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.google.common.collect.ImmutableSet; import java.util.Arrays; import java.util.List; @@ -36,7 +36,7 @@ public class LogicalPlanDSL { public static LogicalPlan aggregation( - LogicalPlan input, List aggregatorList, List groupByList) { + LogicalPlan input, List aggregatorList, List groupByList) { return new LogicalAggregation(input, aggregatorList, groupByList); } @@ -85,6 +85,11 @@ public static LogicalPlan dedupe( input, Arrays.asList(fields), allowedDuplication, keepEmpty, consecutive); } + public static LogicalPlan head( + LogicalPlan input, boolean keeplast, Expression whileExpr, int number) { + return new LogicalHead(input, keeplast, whileExpr, number); + } + public static LogicalPlan rareTopN(LogicalPlan input, CommandType commandType, List groupByList, Expression... fields) { return rareTopN(input, commandType, 10, groupByList, fields); diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java index 1b9ce5208f..601b466909 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java @@ -43,6 +43,10 @@ public R visitDedupe(LogicalDedupe plan, C context) { return visitNode(plan, context); } + public R visitHead(LogicalHead plan, C context) { + return visitNode(plan, context); + } + public R visitRename(LogicalRename plan, C context) { return visitNode(plan, context); } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRareTopN.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRareTopN.java index 22de70894d..89c75d774a 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRareTopN.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRareTopN.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"). * You may not use this file except in compliance with the License. diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperator.java index 9fd395e8e3..3bcf8301f5 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperator.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperator.java @@ -18,8 +18,10 @@ import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AggregationState; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; @@ -47,9 +49,9 @@ public class AggregationOperator extends PhysicalPlan { @Getter private final PhysicalPlan input; @Getter - private final List aggregatorList; + private final List aggregatorList; @Getter - private final List groupByExprList; + private final List groupByExprList; @EqualsAndHashCode.Exclude private final Group group; @EqualsAndHashCode.Exclude @@ -57,12 +59,13 @@ public class AggregationOperator extends PhysicalPlan { /** * AggregationOperator Constructor. - * @param input Input {@link PhysicalPlan} - * @param aggregatorList List of {@link Aggregator} + * + * @param input Input {@link PhysicalPlan} + * @param aggregatorList List of {@link Aggregator} * @param groupByExprList List of group by {@link Expression} */ - public AggregationOperator(PhysicalPlan input, List aggregatorList, - List groupByExprList) { + public AggregationOperator(PhysicalPlan input, List aggregatorList, + List groupByExprList) { this.input = input; this.aggregatorList = aggregatorList; this.groupByExprList = groupByExprList; @@ -103,7 +106,7 @@ public void open() { @RequiredArgsConstructor public class Group { - private final Map>> groupListMap = + private final Map>> groupListMap = new HashMap<>(); /** @@ -131,12 +134,12 @@ public void push(ExprValue inputValue) { */ public List result() { ImmutableList.Builder resultBuilder = new ImmutableList.Builder<>(); - for (Map.Entry>> entry : groupListMap - .entrySet()) { + for (Map.Entry>> + entry : groupListMap.entrySet()) { LinkedHashMap map = new LinkedHashMap<>(); map.putAll(entry.getKey().groupKeyMap()); - for (Map.Entry stateEntry : entry.getValue()) { - map.put(stateEntry.getKey().toString(), stateEntry.getValue().result()); + for (Map.Entry stateEntry : entry.getValue()) { + map.put(stateEntry.getKey().getName(), stateEntry.getValue().result()); } resultBuilder.add(ExprTupleValue.fromExprValueMap(map)); } @@ -169,7 +172,7 @@ public GroupKey(ExprValue value) { public LinkedHashMap groupKeyMap() { LinkedHashMap map = new LinkedHashMap<>(); for (int i = 0; i < groupByExprList.size(); i++) { - map.put(groupByExprList.get(i).toString(), groupByValueList.get(i)); + map.put(groupByExprList.get(i).getName(), groupByValueList.get(i)); } return map; } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/HeadOperator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/HeadOperator.java new file mode 100644 index 0000000000..48e35eacab --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/HeadOperator.java @@ -0,0 +1,114 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.planner.physical; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprBooleanValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.LiteralExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.operator.predicate.BinaryPredicateOperator; +import java.util.Collections; +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; + +/** + * The Head operator returns the first {@link HeadOperator#number} number of results until the + * {@link HeadOperator#whileExpr} evaluates to true. If {@link HeadOperator#keepLast} is true then + * first result which evalutes {@link HeadOperator#whileExpr} to false is also returned. + * The NULL and MISSING are handled by the logic defined in {@link BinaryPredicateOperator}. + */ +@Getter +@EqualsAndHashCode +public class HeadOperator extends PhysicalPlan { + + @Getter + private final PhysicalPlan input; + @Getter + private final Boolean keepLast; + @Getter + private final Expression whileExpr; + @Getter + private final Integer number; + + private static final Integer DEFAULT_LIMIT = 10; + private static final Boolean IGNORE_LAST = false; + + @EqualsAndHashCode.Exclude + private int recordCount = 0; + @EqualsAndHashCode.Exclude + private boolean foundFirstFalse = false; + @EqualsAndHashCode.Exclude + private ExprValue next; + + @NonNull + public HeadOperator(PhysicalPlan input) { + this(input, IGNORE_LAST, new LiteralExpression(ExprBooleanValue.of(true)), DEFAULT_LIMIT); + } + + /** + * HeadOperator Constructor. + * + * @param input Input {@link PhysicalPlan} + * @param keepLast Controls whether the last result in the result set is retained. The last + * result returned is the result that caused the whileExpr to evaluate to false + * or NULL. + * @param whileExpr The search returns results until this expression evaluates to false + * @param number Number of specified results + */ + @NonNull + public HeadOperator(PhysicalPlan input, Boolean keepLast, Expression whileExpr, Integer number) { + this.input = input; + this.keepLast = keepLast; + this.whileExpr = whileExpr; + this.number = number; + } + + @Override + public R accept(PhysicalPlanNodeVisitor visitor, C context) { + return visitor.visitHead(this, context); + } + + @Override + public List getChild() { + return Collections.singletonList(input); + } + + @Override + public boolean hasNext() { + if (!input.hasNext() || foundFirstFalse || (recordCount >= number)) { + return false; + } + ExprValue inputVal = input.next(); + ExprValue exprValue = whileExpr.valueOf(inputVal.bindingTuples()); + if (exprValue.isNull() || exprValue.isMissing() || !(exprValue.booleanValue())) { + // First false is when we decide whether to keep the last value + foundFirstFalse = true; + if (!keepLast) { + return false; + } + } + this.next = inputVal; + recordCount++; + return true; + } + + @Override + public ExprValue next() { + return this.next; + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanDSL.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanDSL.java index 515711a84f..03b22d6d7a 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanDSL.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanDSL.java @@ -22,6 +22,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.google.common.collect.ImmutableSet; import java.util.Arrays; import java.util.List; @@ -36,7 +37,7 @@ public class PhysicalPlanDSL { public static AggregationOperator agg( - PhysicalPlan input, List aggregators, List groups) { + PhysicalPlan input, List aggregators, List groups) { return new AggregationOperator(input, aggregators, groups); } @@ -81,6 +82,11 @@ public static DedupeOperator dedupe( input, Arrays.asList(expressions), allowedDuplication, keepEmpty, consecutive); } + public static HeadOperator head(PhysicalPlan input, boolean keepLast, Expression whileExpr, + int number) { + return new HeadOperator(input, keepLast, whileExpr, number); + } + public static RareTopNOperator rareTopN(PhysicalPlan input, CommandType commandType, List groups, Expression... expressions) { return new RareTopNOperator(input, commandType, Arrays.asList(expressions), groups); diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitor.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitor.java index 62b35af003..13bab7eb34 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitor.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitor.java @@ -69,7 +69,12 @@ public R visitSort(SortOperator node, C context) { return visitNode(node, context); } + public R visitHead(HeadOperator node, C context) { + return visitNode(node, context); + } + public R visitRareTopN(RareTopNOperator node, C context) { return visitNode(node, context); } + } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperator.java index 4e98787c76..99f5e79855 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperator.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperator.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"). * You may not use this file except in compliance with the License. diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/ValuesOperator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/ValuesOperator.java index 215589e5ef..5ba041e574 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/ValuesOperator.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/ValuesOperator.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.stream.Collectors; import lombok.EqualsAndHashCode; +import lombok.Getter; import lombok.ToString; /** @@ -36,6 +37,7 @@ public class ValuesOperator extends PhysicalPlan { /** * Original values list for print and equality check. */ + @Getter private final List> values; /** diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTest.java index b41818ac5e..28b4725861 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTest.java @@ -15,25 +15,36 @@ package com.amazon.opendistroforelasticsearch.sql.analysis; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.aggregate; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.alias; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.argument; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.booleanLiteral; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.compare; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.field; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.filter; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.function; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.intLiteral; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.qualifiedName; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.relation; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.unresolvedArg; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.unresolvedArgList; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.booleanValue; import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.integerValue; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DOUBLE; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.LONG; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.STRING; +import static java.util.Collections.emptyList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN.CommandType; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan; import com.amazon.opendistroforelasticsearch.sql.exception.SemanticCheckException; import com.amazon.opendistroforelasticsearch.sql.expression.DSL; import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -61,6 +72,20 @@ public void filter_relation() { AstDSL.equalTo(AstDSL.field("integer_value"), AstDSL.intLiteral(1)))); } + @Test + public void head_relation() { + assertAnalyzeEqual( + LogicalPlanDSL.head( + LogicalPlanDSL.relation("schema"), + false, dsl.equal(DSL.ref("integer_value", INTEGER), DSL.literal(integerValue(1))), 10), + AstDSL.head( + AstDSL.relation("schema"), + unresolvedArgList( + unresolvedArg("keeplast", booleanLiteral(false)), + unresolvedArg("whileExpr", compare("=", field("integer_value"), intLiteral(1))), + unresolvedArg("number", intLiteral(10))))); + } + @Test public void analyze_filter_relation() { assertAnalyzeEqual( @@ -81,37 +106,24 @@ public void rename_relation() { AstDSL.map(AstDSL.field("integer_value"), AstDSL.field("ivalue")))); } - @Test - public void rename_stats_source() { - assertAnalyzeEqual( - LogicalPlanDSL.rename( - LogicalPlanDSL.aggregation( - LogicalPlanDSL.relation("schema"), - ImmutableList.of(dsl.avg(DSL.ref("integer_value", INTEGER))), - ImmutableList.of()), - ImmutableMap.of(DSL.ref("avg(integer_value)", DOUBLE), DSL.ref("ivalue", DOUBLE))), - AstDSL.rename( - AstDSL.agg( - AstDSL.relation("schema"), - AstDSL.exprList(AstDSL.aggregate("avg", field("integer_value"))), - null, - ImmutableList.of(), - AstDSL.defaultStatsArgs()), - AstDSL.map(AstDSL.aggregate("avg", field("integer_value")), field("ivalue")))); - } - @Test public void stats_source() { assertAnalyzeEqual( LogicalPlanDSL.aggregation( LogicalPlanDSL.relation("schema"), - ImmutableList.of(dsl.avg(DSL.ref("integer_value", INTEGER))), - ImmutableList.of(DSL.ref("string_value", STRING))), + ImmutableList + .of(DSL.named("avg(integer_value)", dsl.avg(DSL.ref("integer_value", INTEGER)))), + ImmutableList.of(DSL.named("string_value", DSL.ref("string_value", STRING)))), AstDSL.agg( AstDSL.relation("schema"), - AstDSL.exprList(AstDSL.aggregate("avg", field("integer_value"))), + AstDSL.exprList( + AstDSL.alias( + "avg(integer_value)", + AstDSL.aggregate("avg", field("integer_value"))) + ), null, - ImmutableList.of(field("string_value")), + ImmutableList.of( + AstDSL.alias("string_value", field("string_value"))), AstDSL.defaultStatsArgs())); } @@ -165,7 +177,9 @@ public void rename_to_invalid_expression() { AstDSL.rename( AstDSL.agg( AstDSL.relation("schema"), - AstDSL.exprList(AstDSL.aggregate("avg", field("integer_value"))), + AstDSL.exprList( + AstDSL.alias("avg(integer_value)", AstDSL.aggregate("avg", field( + "integer_value")))), Collections.emptyList(), ImmutableList.of(), AstDSL.defaultStatsArgs()), @@ -241,4 +255,61 @@ public void project_values() { ); } + /** + * SELECT name, AVG(age) FROM test GROUP BY name. + */ + @Test + public void sql_group_by_field() { + assertAnalyzeEqual( + LogicalPlanDSL.project( + LogicalPlanDSL.aggregation( + LogicalPlanDSL.relation("schema"), + ImmutableList + .of(DSL + .named("AVG(integer_value)", dsl.avg(DSL.ref("integer_value", INTEGER)))), + ImmutableList.of(DSL.named("string_value", DSL.ref("string_value", STRING)))), + DSL.named("string_value", DSL.ref("string_value", STRING)), + DSL.named("AVG(integer_value)", DSL.ref("AVG(integer_value)", DOUBLE))), + AstDSL.project( + AstDSL.agg( + AstDSL.relation("schema"), + ImmutableList.of(alias("AVG(integer_value)", + aggregate("AVG", qualifiedName("integer_value")))), + emptyList(), + ImmutableList.of(alias("string_value", qualifiedName("string_value"))), + emptyList()), + AstDSL.alias("string_value", qualifiedName("string_value")), + AstDSL.alias("AVG(integer_value)", aggregate("AVG", qualifiedName("integer_value")))) + ); + } + + /** + * SELECT abs(name), AVG(age) FROM test GROUP BY abs(name). + */ + @Test + public void sql_group_by_function() { + assertAnalyzeEqual( + LogicalPlanDSL.project( + LogicalPlanDSL.aggregation( + LogicalPlanDSL.relation("schema"), + ImmutableList + .of(DSL + .named("AVG(integer_value)", dsl.avg(DSL.ref("integer_value", INTEGER)))), + ImmutableList.of(DSL.named("abs(long_value)", + dsl.abs(DSL.ref("long_value", LONG))))), + DSL.named("abs(long_value)", DSL.ref("abs(long_value)", LONG)), + DSL.named("AVG(integer_value)", DSL.ref("AVG(integer_value)", DOUBLE))), + AstDSL.project( + AstDSL.agg( + AstDSL.relation("schema"), + ImmutableList.of(alias("AVG(integer_value)", + aggregate("AVG", qualifiedName("integer_value")))), + emptyList(), + ImmutableList + .of(alias("abs(long_value)", function("abs", qualifiedName("long_value")))), + emptyList()), + AstDSL.alias("abs(long_value)", function("abs", qualifiedName("long_value"))), + AstDSL.alias("AVG(integer_value)", aggregate("AVG", qualifiedName("integer_value")))) + ); + } } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/NamedExpressionAnalyzerTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/NamedExpressionAnalyzerTest.java new file mode 100644 index 0000000000..4386ec7501 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/NamedExpressionAnalyzerTest.java @@ -0,0 +1,46 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.analysis; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.Alias; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.springframework.context.annotation.Configuration; +import org.springframework.test.context.ContextConfiguration; +import org.springframework.test.context.junit.jupiter.SpringExtension; + +@Configuration +@ExtendWith(SpringExtension.class) +@ContextConfiguration(classes = {ExpressionConfig.class, AnalyzerTestBase.class}) +class NamedExpressionAnalyzerTest extends AnalyzerTestBase { + @Test + void visit_named_seleteitem() { + Alias alias = AstDSL.alias("integer_value", AstDSL.qualifiedName("integer_value")); + + NamedExpressionAnalyzer analyzer = + new NamedExpressionAnalyzer(expressionAnalyzer); + + NamedExpression analyze = analyzer.analyze(alias, analysisContext); + assertEquals("integer_value", analyze.getName()); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectAnalyzeTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectAnalyzeTest.java index 376e7538c9..814fb63a83 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectAnalyzeTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectAnalyzeTest.java @@ -124,17 +124,19 @@ public void stats_and_project_all() { LogicalPlanDSL.project( LogicalPlanDSL.aggregation( LogicalPlanDSL.relation("schema"), - ImmutableList.of(dsl.avg(DSL.ref("integer_value", INTEGER))), - ImmutableList.of(DSL.ref("string_value", STRING))), + ImmutableList.of(DSL + .named("avg(integer_value)", dsl.avg(DSL.ref("integer_value", INTEGER)))), + ImmutableList.of(DSL.named("string_value", DSL.ref("string_value", STRING)))), DSL.named("string_value", DSL.ref("string_value", STRING)), DSL.named("avg(integer_value)", DSL.ref("avg(integer_value)", DOUBLE)) ), AstDSL.projectWithArg( AstDSL.agg( AstDSL.relation("schema"), - AstDSL.exprList(AstDSL.aggregate("avg", field("integer_value"))), + AstDSL.exprList(AstDSL.alias("avg(integer_value)", AstDSL.aggregate("avg", + field("integer_value")))), null, - ImmutableList.of(field("string_value")), + ImmutableList.of(AstDSL.alias("string_value", field("string_value"))), AstDSL.defaultStatsArgs()), AstDSL.defaultFieldsArgs(), AllFields.of())); } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzerTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzerTest.java index d4e59c8219..a5f9fe5a65 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzerTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/SelectExpressionAnalyzerTest.java @@ -61,10 +61,10 @@ public void named_expression_with_alias() { @Test public void named_expression_with_delegated_expression_defined_in_symbol_table() { analysisContext.push(); - analysisContext.peek().define(new Symbol(Namespace.FIELD_NAME, "avg(integer_value)"), FLOAT); + analysisContext.peek().define(new Symbol(Namespace.FIELD_NAME, "AVG(integer_value)"), FLOAT); assertAnalyzeEqual( - DSL.named("AVG(integer_value)", DSL.ref("avg(integer_value)", FLOAT)), + DSL.named("AVG(integer_value)", DSL.ref("AVG(integer_value)", FLOAT)), AstDSL.alias("AVG(integer_value)", AstDSL.aggregate("AVG", AstDSL.qualifiedName("integer_value"))) ); diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/executor/ExplainTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/executor/ExplainTest.java new file mode 100644 index 0000000000..9eee96b082 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/executor/ExplainTest.java @@ -0,0 +1,247 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.executor; + +import static com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN.CommandType.TOP; +import static com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort.SortOption.PPL_ASC; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DOUBLE; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.STRING; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.literal; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.named; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.ref; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.agg; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.dedupe; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.eval; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.filter; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.head; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.project; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.rareTopN; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.remove; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.rename; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.sort; +import static com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL.values; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponseNode; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionTestBase; +import com.amazon.opendistroforelasticsearch.sql.expression.LiteralExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; +import com.amazon.opendistroforelasticsearch.sql.storage.TableScanOperator; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +class ExplainTest extends ExpressionTestBase { + + private final Explain explain = new Explain(); + + private final FakeTableScan tableScan = new FakeTableScan(); + + @Test + void can_explain_project_filter_table_scan() { + Expression filterExpr = + dsl.and( + dsl.equal(ref("balance", INTEGER), literal(10000)), + dsl.greater(ref("age", INTEGER), literal(30))); + NamedExpression[] projectList = { + named("full_name", ref("full_name", STRING), "name"), + named("age", ref("age", INTEGER)) + }; + + PhysicalPlan plan = + project( + filter( + tableScan, + filterExpr), + projectList); + + assertEquals( + new ExplainResponse( + new ExplainResponseNode( + "ProjectOperator", + ImmutableMap.of("fields", "[name, age]"), + singletonList(new ExplainResponseNode( + "FilterOperator", + ImmutableMap.of("conditions", "and(=(balance, 10000), >(age, 30))"), + singletonList(tableScan.explain()))))), + explain.apply(plan)); + } + + @Test + void can_explain_aggregations() { + List aggExprs = ImmutableList.of(ref("balance", DOUBLE)); + List aggList = ImmutableList.of( + named("avg(balance)", dsl.avg(aggExprs.toArray(new Expression[0])))); + List groupByList = ImmutableList.of( + named("state", ref("state", STRING))); + + PhysicalPlan plan = agg(new FakeTableScan(), aggList, groupByList); + assertEquals( + new ExplainResponse( + new ExplainResponseNode( + "AggregationOperator", + ImmutableMap.of( + "aggregators", "[avg(balance)]", + "groupBy", "[state]"), + singletonList(tableScan.explain()))), + explain.apply(plan)); + } + + @Test + void can_explain_rare_top_n() { + Expression field = ref("state", STRING); + + PhysicalPlan plan = rareTopN(tableScan, TOP, emptyList(), field); + assertEquals( + new ExplainResponse( + new ExplainResponseNode( + "RareTopNOperator", + ImmutableMap.of( + "commandType", TOP, + "noOfResults", 10, + "fields", "[state]", + "groupBy", "[]"), + singletonList(tableScan.explain()))), + explain.apply(plan)); + } + + @Test + void can_explain_head() { + Boolean keepLast = false; + Expression whileExpr = dsl.and( + dsl.equal(ref("balance", INTEGER), literal(10000)), + dsl.greater(ref("age", INTEGER), literal(30))); + Integer number = 5; + + PhysicalPlan plan = head(tableScan, keepLast, whileExpr, number); + + assertEquals( + new ExplainResponse( + new ExplainResponseNode( + "HeadOperator", + ImmutableMap.of( + "keepLast", false, + "whileExpr", "and(=(balance, 10000), >(age, 30))", + "number", 5), + singletonList(tableScan.explain()))), + explain.apply(plan)); + } + + @Test + void can_explain_other_operators() { + ReferenceExpression[] removeList = {ref("state", STRING)}; + Map renameMapping = ImmutableMap.of( + ref("state", STRING), ref("s", STRING)); + Pair evalExprs = ImmutablePair.of( + ref("age", INTEGER), dsl.add(ref("age", INTEGER), literal(2))); + Expression[] dedupeList = {ref("age", INTEGER)}; + Pair sortList = ImmutablePair.of( + PPL_ASC, ref("age", INTEGER)); + List values = ImmutableList.of(literal("WA"), literal(30)); + + PhysicalPlan plan = + remove( + rename( + eval( + dedupe( + sort( + values(values), + 1000, + sortList), + dedupeList), + evalExprs), + renameMapping), + removeList); + + assertEquals( + new ExplainResponse( + new ExplainResponseNode( + "RemoveOperator", + ImmutableMap.of("removeList", "[state]"), + singletonList(new ExplainResponseNode( + "RenameOperator", + ImmutableMap.of("mapping", ImmutableMap.of("state", "s")), + singletonList(new ExplainResponseNode( + "EvalOperator", + ImmutableMap.of("expressions", ImmutableMap.of("age", "+(age, 2)")), + singletonList(new ExplainResponseNode( + "DedupeOperator", + ImmutableMap.of( + "dedupeList", "[age]", + "allowedDuplication", 1, + "keepEmpty", false, + "consecutive", false), + singletonList(new ExplainResponseNode( + "SortOperator", + ImmutableMap.of( + "count", 1000, + "sortList", ImmutableMap.of( + "age", ImmutableMap.of( + "sortOrder", "ASC", + "nullOrder", "NULL_FIRST"))), + singletonList(new ExplainResponseNode( + "ValuesOperator", + ImmutableMap.of("values", ImmutableList.of(values)), + emptyList()))))))))))) + ), + explain.apply(plan) + ); + } + + private static class FakeTableScan extends TableScanOperator { + @Override + public boolean hasNext() { + return false; + } + + @Override + public ExprValue next() { + return null; + } + + @Override + public String toString() { + return "Fake DSL request"; + } + + /** Used to ignore table scan which is duplicate but required for each operator test. */ + public ExplainResponseNode explain() { + return new ExplainResponseNode( + "FakeTableScan", + ImmutableMap.of("request", "Fake DSL request"), + emptyList()); + } + } + +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitorTest.java index 58a4eff006..3ee0e80b19 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/ExpressionNodeVisitorTest.java @@ -25,7 +25,9 @@ import static org.junit.jupiter.api.Assertions.assertNull; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AvgAggregator; import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.junit.jupiter.api.DisplayNameGeneration; @@ -45,6 +47,8 @@ void should_return_null_by_default() { assertNull(named("bool", literal(true)).accept(visitor, null)); assertNull(dsl.abs(literal(-10)).accept(visitor, null)); assertNull(dsl.sum(literal(10)).accept(visitor, null)); + assertNull(named("avg", new AvgAggregator(Collections.singletonList(ref("age", INTEGER)), + INTEGER)).accept(visitor, null)); } @Test diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunctionTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunctionTest.java index 1f6e35ba16..b0ce5088d9 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunctionTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/datetime/DateTimeFunctionTest.java @@ -23,6 +23,8 @@ import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DATE; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DATETIME; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTERVAL; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.LONG; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.TIME; import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.TIMESTAMP; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -65,13 +67,18 @@ public void setup() { public void dayOfMonth() { when(nullRef.type()).thenReturn(DATE); when(missingRef.type()).thenReturn(DATE); - - FunctionExpression expression = dsl.dayofmonth(DSL.literal(new ExprDateValue("2020-07-07"))); - assertEquals(INTEGER, expression.type()); - assertEquals("dayofmonth(DATE '2020-07-07')", expression.toString()); - assertEquals(integerValue(7), eval(expression)); assertEquals(nullValue(), eval(dsl.dayofmonth(nullRef))); assertEquals(missingValue(), eval(dsl.dayofmonth(missingRef))); + + FunctionExpression expression = dsl.dayofmonth(DSL.literal(new ExprDateValue("2020-07-08"))); + assertEquals(INTEGER, expression.type()); + assertEquals("dayofmonth(DATE '2020-07-08')", expression.toString()); + assertEquals(integerValue(8), eval(expression)); + + expression = dsl.dayofmonth(DSL.literal("2020-07-08")); + assertEquals(INTEGER, expression.type()); + assertEquals("dayofmonth(\"2020-07-08\")", expression.toString()); + assertEquals(integerValue(8), eval(expression)); } @Test @@ -128,6 +135,51 @@ public void timestamp() { assertEquals("timestamp(TIMESTAMP '2020-08-17 01:01:01')", expr.toString()); } + @Test + public void adddate() { + FunctionExpression expr = dsl.adddate(dsl.date(DSL.literal("2020-08-26")), DSL.literal(7)); + assertEquals(DATE, expr.type()); + assertEquals(new ExprDateValue("2020-09-02"), expr.valueOf(env)); + assertEquals("adddate(date(\"2020-08-26\"), 7)", expr.toString()); + + expr = dsl.adddate(dsl.timestamp(DSL.literal("2020-08-26 12:05:00")), DSL.literal(7)); + assertEquals(DATETIME, expr.type()); + assertEquals(new ExprDatetimeValue("2020-09-02 12:05:00"), expr.valueOf(env)); + assertEquals("adddate(timestamp(\"2020-08-26 12:05:00\"), 7)", expr.toString()); + + expr = dsl.adddate( + dsl.date(DSL.literal("2020-08-26")), dsl.interval(DSL.literal(1), DSL.literal("hour"))); + assertEquals(DATETIME, expr.type()); + assertEquals(new ExprDatetimeValue("2020-08-26 01:00:00"), expr.valueOf(env)); + assertEquals("adddate(date(\"2020-08-26\"), interval(1, \"hour\"))", expr.toString()); + + when(nullRef.type()).thenReturn(DATE); + assertEquals(nullValue(), eval(dsl.adddate(nullRef, DSL.literal(1L)))); + assertEquals(nullValue(), + eval(dsl.adddate(nullRef, dsl.interval(DSL.literal(1), DSL.literal("month"))))); + + when(missingRef.type()).thenReturn(DATE); + assertEquals(missingValue(), eval(dsl.adddate(missingRef, DSL.literal(1L)))); + assertEquals(missingValue(), + eval(dsl.adddate(missingRef, dsl.interval(DSL.literal(1), DSL.literal("month"))))); + + when(nullRef.type()).thenReturn(LONG); + when(missingRef.type()).thenReturn(LONG); + assertEquals(nullValue(), eval(dsl.adddate(dsl.date(DSL.literal("2020-08-26")), nullRef))); + assertEquals(missingValue(), + eval(dsl.adddate(dsl.date(DSL.literal("2020-08-26")), missingRef))); + + when(nullRef.type()).thenReturn(INTERVAL); + when(missingRef.type()).thenReturn(INTERVAL); + assertEquals(nullValue(), eval(dsl.adddate(dsl.date(DSL.literal("2020-08-26")), nullRef))); + assertEquals(missingValue(), + eval(dsl.adddate(dsl.date(DSL.literal("2020-08-26")), missingRef))); + + when(nullRef.type()).thenReturn(DATE); + when(missingRef.type()).thenReturn(INTERVAL); + assertEquals(missingValue(), eval(dsl.adddate(nullRef, missingRef))); + } + private ExprValue eval(Expression expression) { return expression.valueOf(env); } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementorTest.java index 06a8033764..2db202c8df 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/DefaultImplementorTest.java @@ -24,6 +24,7 @@ import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.aggregation; import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.eval; import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.filter; +import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.head; import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.project; import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.rareTopN; import static com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL.remove; @@ -38,11 +39,12 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprBooleanValue; import com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; -import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AvgAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation; @@ -67,10 +69,12 @@ public void visitShouldReturnDefaultPhysicalOperator() { ReferenceExpression exclude = ref("name", STRING); ReferenceExpression dedupeField = ref("name", STRING); Expression filterExpr = literal(ExprBooleanValue.of(true)); - List groupByExprs = Arrays.asList(ref("age", INTEGER)); + List groupByExprs = Arrays.asList(DSL.named("age", ref("age", INTEGER))); + List aggExprs = Arrays.asList(ref("age", INTEGER)); ReferenceExpression rareTopNField = ref("age", INTEGER); - List aggregators = - Arrays.asList(new AvgAggregator(groupByExprs, ExprCoreType.DOUBLE)); + List topByExprs = Arrays.asList(ref("age", INTEGER)); + List aggregators = + Arrays.asList(DSL.named("avg(age)", new AvgAggregator(aggExprs, ExprCoreType.DOUBLE))); Map mappings = ImmutableMap.of(ref("name", STRING), ref("lastname", STRING)); Pair newEvalField = @@ -78,27 +82,34 @@ public void visitShouldReturnDefaultPhysicalOperator() { Integer sortCount = 100; Pair sortField = ImmutablePair.of(Sort.SortOption.PPL_ASC, ref("name1", STRING)); + Boolean keeplast = true; + Expression whileExpr = literal(ExprBooleanValue.of(true)); + Integer number = 5; LogicalPlan plan = project( LogicalPlanDSL.dedupe( - rareTopN( - sort( - eval( - remove( - rename( - aggregation( - filter(values(emptyList()), filterExpr), - aggregators, - groupByExprs), - mappings), - exclude), - newEvalField), - sortCount, - sortField), - CommandType.TOP, - groupByExprs, - rareTopNField), + head( + rareTopN( + sort( + eval( + remove( + rename( + aggregation( + filter(values(emptyList()), filterExpr), + aggregators, + groupByExprs), + mappings), + exclude), + newEvalField), + sortCount, + sortField), + CommandType.TOP, + topByExprs, + rareTopNField), + keeplast, + whileExpr, + number), dedupeField), include); @@ -107,29 +118,32 @@ public void visitShouldReturnDefaultPhysicalOperator() { assertEquals( PhysicalPlanDSL.project( PhysicalPlanDSL.dedupe( - PhysicalPlanDSL.rareTopN( - PhysicalPlanDSL.sort( - PhysicalPlanDSL.eval( - PhysicalPlanDSL.remove( - PhysicalPlanDSL.rename( - PhysicalPlanDSL.agg( - PhysicalPlanDSL.filter( - PhysicalPlanDSL.values(emptyList()), - filterExpr), - aggregators, - groupByExprs), - mappings), - exclude), - newEvalField), - sortCount, - sortField), - CommandType.TOP, - groupByExprs, - rareTopNField), + PhysicalPlanDSL.head( + PhysicalPlanDSL.rareTopN( + PhysicalPlanDSL.sort( + PhysicalPlanDSL.eval( + PhysicalPlanDSL.remove( + PhysicalPlanDSL.rename( + PhysicalPlanDSL.agg( + PhysicalPlanDSL.filter( + PhysicalPlanDSL.values(emptyList()), + filterExpr), + aggregators, + groupByExprs), + mappings), + exclude), + newEvalField), + sortCount, + sortField), + CommandType.TOP, + topByExprs, + rareTopNField), + keeplast, + whileExpr, + number), dedupeField), include), actual); - } @Test diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/PlannerTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/PlannerTest.java index 33e14cf902..be783c7d42 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/PlannerTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/PlannerTest.java @@ -71,7 +71,7 @@ public void planner_test() { scan, dsl.equal(DSL.ref("response", INTEGER), DSL.literal(10)) ), - ImmutableList.of(dsl.avg(DSL.ref("response", INTEGER))), + ImmutableList.of(DSL.named("avg(response)", dsl.avg(DSL.ref("response", INTEGER)))), ImmutableList.of() ), ImmutableMap.of(DSL.ref("ivalue", INTEGER), DSL.ref("avg(response)", DOUBLE)) @@ -82,7 +82,7 @@ public void planner_test() { LogicalPlanDSL.relation("schema"), dsl.equal(DSL.ref("response", INTEGER), DSL.literal(10)) ), - ImmutableList.of(dsl.avg(DSL.ref("response", INTEGER))), + ImmutableList.of(DSL.named("avg(response)", dsl.avg(DSL.ref("response", INTEGER)))), ImmutableList.of() ), ImmutableMap.of(DSL.ref("ivalue", INTEGER), DSL.ref("avg(response)", DOUBLE)) diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitorTest.java index 397a79c8b3..bbc82c9f7c 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitorTest.java @@ -21,6 +21,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN.CommandType; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort.SortOption; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; @@ -51,17 +52,19 @@ public void logicalPlanShouldTraversable() { LogicalPlan logicalPlan = LogicalPlanDSL.rename( LogicalPlanDSL.aggregation( - LogicalPlanDSL.rareTopN( - LogicalPlanDSL.filter(LogicalPlanDSL.relation("schema"), expression), - CommandType.TOP, - ImmutableList.of(expression), - expression), - ImmutableList.of(aggregator), - ImmutableList.of(expression)), + LogicalPlanDSL.head( + LogicalPlanDSL.rareTopN( + LogicalPlanDSL.filter(LogicalPlanDSL.relation("schema"), expression), + CommandType.TOP, + ImmutableList.of(expression), + expression), + false, expression, 10), + ImmutableList.of(DSL.named("avg", aggregator)), + ImmutableList.of(DSL.named("group", expression))), ImmutableMap.of(ref, ref)); Integer result = logicalPlan.accept(new NodesCount(), null); - assertEquals(5, result); + assertEquals(6, result); } @Test @@ -74,9 +77,14 @@ public void testAbstractPlanNodeVisitorShouldReturnNull() { assertNull(filter.accept(new LogicalPlanNodeVisitor() { }, null)); + LogicalPlan head = LogicalPlanDSL.head(relation, false, expression, 10); + assertNull(head.accept(new LogicalPlanNodeVisitor() { + }, null)); + LogicalPlan aggregation = LogicalPlanDSL.aggregation( - filter, ImmutableList.of(aggregator), ImmutableList.of(expression)); + filter, ImmutableList.of(DSL.named("avg", aggregator)), ImmutableList.of(DSL.named( + "group", expression))); assertNull(aggregation.accept(new LogicalPlanNodeVisitor() { }, null)); @@ -111,7 +119,6 @@ public void testAbstractPlanNodeVisitorShouldReturnNull() { } private static class NodesCount extends LogicalPlanNodeVisitor { - @Override public Integer visitRelation(LogicalRelation plan, Object context) { return 1; @@ -125,6 +132,14 @@ public Integer visitFilter(LogicalFilter plan, Object context) { .collect(Collectors.summingInt(Integer::intValue)); } + @Override + public Integer visitHead(LogicalHead plan, Object context) { + return 1 + + plan.getChild().stream() + .map(child -> child.accept(this, context)) + .collect(Collectors.summingInt(Integer::intValue)); + } + @Override public Integer visitAggregation(LogicalAggregation plan, Object context) { return 1 diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperatorTest.java index fbfca109c9..a2dfd53390 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperatorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperatorTest.java @@ -34,8 +34,9 @@ class AggregationOperatorTest extends PhysicalPlanTestBase { @Test public void avg_with_one_groups() { PhysicalPlan plan = new AggregationOperator(new TestScan(), - Collections.singletonList(dsl.avg(DSL.ref("response", INTEGER))), - Collections.singletonList(DSL.ref("action", STRING))); + Collections + .singletonList(DSL.named("avg(response)", dsl.avg(DSL.ref("response", INTEGER)))), + Collections.singletonList(DSL.named("action", DSL.ref("action", STRING)))); List result = execute(plan); assertEquals(2, result.size()); assertThat(result, containsInAnyOrder( @@ -47,8 +48,10 @@ public void avg_with_one_groups() { @Test public void avg_with_two_groups() { PhysicalPlan plan = new AggregationOperator(new TestScan(), - Collections.singletonList(dsl.avg(DSL.ref("response", INTEGER))), - Arrays.asList(DSL.ref("action", STRING), DSL.ref("ip", STRING))); + Collections + .singletonList(DSL.named("avg(response)", dsl.avg(DSL.ref("response", INTEGER)))), + Arrays.asList(DSL.named("action", DSL.ref("action", STRING)), + DSL.named("ip", DSL.ref("ip", STRING)))); List result = execute(plan); assertEquals(3, result.size()); assertThat(result, containsInAnyOrder( @@ -64,8 +67,9 @@ public void avg_with_two_groups() { @Test public void sum_with_one_groups() { PhysicalPlan plan = new AggregationOperator(new TestScan(), - Collections.singletonList(dsl.sum(DSL.ref("response", INTEGER))), - Collections.singletonList(DSL.ref("action", STRING))); + Collections + .singletonList(DSL.named("sum(response)", dsl.sum(DSL.ref("response", INTEGER)))), + Collections.singletonList(DSL.named("action", DSL.ref("action", STRING)))); List result = execute(plan); assertEquals(2, result.size()); assertThat(result, containsInAnyOrder( diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/HeadOperatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/HeadOperatorTest.java new file mode 100644 index 0000000000..a510529cea --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/HeadOperatorTest.java @@ -0,0 +1,152 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.planner.physical; + +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.LITERAL_MISSING; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.LITERAL_NULL; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.when; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.google.common.collect.ImmutableMap; +import java.util.LinkedHashMap; +import java.util.List; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@ExtendWith(MockitoExtension.class) +class HeadOperatorTest extends PhysicalPlanTestBase { + + @Mock + private PhysicalPlan inputPlan; + + private final int defaultResultCount = 10; + + @Test + public void headTest() { + HeadOperator plan = new HeadOperator(new CountTestScan()); + List result = execute(plan); + assertEquals(defaultResultCount, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("id", 1, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 2, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 3, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 4, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 5, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 6, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 7, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 8, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 9, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 10, "testString", "asdf")) + )); + } + + @Test + public void headTest_Number() { + HeadOperator plan = new HeadOperator(new CountTestScan(), + false, DSL.literal(true), 2); + List result = execute(plan); + assertEquals(2, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("id", 1, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 2, "testString", "asdf")) + )); + } + + @Test + public void headTest_InputEnd() { + HeadOperator plan = new HeadOperator(new CountTestScan(), + false, DSL.literal(true), 12); + List result = execute(plan); + assertEquals(11, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("id", 1, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 2, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 3, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 4, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 5, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 6, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 7, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 8, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 9, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 10, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 11, "testString", "asdf")) + )); + } + + @Test + public void headTest_keepLastTrue() { + HeadOperator plan = new HeadOperator(new CountTestScan(), + true, dsl.less(DSL.ref("id", INTEGER), DSL.literal(5)), defaultResultCount); + List result = execute(plan); + assertEquals(5, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("id", 1, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 2, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 3, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 4, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 5, "testString", "asdf")) + )); + } + + @Test + public void headTest_keepLastFalse() { + HeadOperator plan = new HeadOperator(new CountTestScan(), + false, dsl.less(DSL.ref("id", INTEGER), DSL.literal(5)), defaultResultCount); + List result = execute(plan); + assertEquals(4, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("id", 1, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 2, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 3, "testString", "asdf")), + ExprValueUtils.tupleValue(ImmutableMap.of("id", 4, "testString", "asdf")) + )); + } + + @Test + public void nullValueShouldBeenIgnored() { + LinkedHashMap value = new LinkedHashMap<>(); + value.put("id", LITERAL_NULL); + when(inputPlan.hasNext()).thenReturn(true, false); + when(inputPlan.next()).thenReturn(new ExprTupleValue(value)); + + HeadOperator plan = new HeadOperator(inputPlan, + false, dsl.less(DSL.ref("id", INTEGER), DSL.literal(5)), defaultResultCount); + List result = execute(plan); + assertEquals(0, result.size()); + } + + @Test + public void headTest_missingValueShouldBeenIgnored() { + LinkedHashMap value = new LinkedHashMap<>(); + value.put("id", LITERAL_MISSING); + when(inputPlan.hasNext()).thenReturn(true, false); + when(inputPlan.next()).thenReturn(new ExprTupleValue(value)); + + HeadOperator plan = new HeadOperator(inputPlan, + false, dsl.less(DSL.ref("id", INTEGER), DSL.literal(5)), defaultResultCount); + List result = execute(plan); + assertEquals(0, result.size()); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitorTest.java index 151afdf779..f4f10aec27 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitorTest.java @@ -42,7 +42,6 @@ */ @ExtendWith(MockitoExtension.class) class PhysicalPlanNodeVisitorTest extends PhysicalPlanTestBase { - @Mock PhysicalPlan plan; @Mock @@ -55,14 +54,19 @@ public void print_physical_plan() { PhysicalPlanDSL.project( PhysicalPlanDSL.rename( PhysicalPlanDSL.agg( - PhysicalPlanDSL.rareTopN( - PhysicalPlanDSL.filter( - new TestScan(), - dsl.equal(DSL.ref("response", INTEGER), DSL.literal(10))), - CommandType.TOP, - ImmutableList.of(), - DSL.ref("response", INTEGER)), - ImmutableList.of(dsl.avg(DSL.ref("response", INTEGER))), + PhysicalPlanDSL.head( + PhysicalPlanDSL.rareTopN( + PhysicalPlanDSL.filter( + new TestScan(), + dsl.equal(DSL.ref("response", INTEGER), DSL.literal(10))), + CommandType.TOP, + ImmutableList.of(), + DSL.ref("response", INTEGER)), + false, + DSL.literal(false), + 10), + ImmutableList + .of(DSL.named("avg(response)", dsl.avg(DSL.ref("response", INTEGER)))), ImmutableList.of()), ImmutableMap.of(DSL.ref("ivalue", INTEGER), DSL.ref("avg(response)", DOUBLE))), named("ref", ref)), @@ -74,8 +78,9 @@ public void print_physical_plan() { + "\tProject->\n" + "\t\tRename->\n" + "\t\t\tAggregation->\n" - + "\t\t\t\tRareTopN->\n" - + "\t\t\t\t\tFilter->", + + "\t\t\t\tHead->\n" + + "\t\t\t\t\tRareTopN->\n" + + "\t\t\t\t\t\tFilter->", printer.print(plan)); } @@ -87,9 +92,15 @@ public void test_PhysicalPlanVisitor_should_return_null() { assertNull(filter.accept(new PhysicalPlanNodeVisitor() { }, null)); + PhysicalPlan head = PhysicalPlanDSL.head( + new TestScan(), false, dsl.equal(DSL.ref("response", INTEGER), DSL.literal(10)), 10); + assertNull(head.accept(new PhysicalPlanNodeVisitor() { + }, null)); + PhysicalPlan aggregation = PhysicalPlanDSL.agg( - filter, ImmutableList.of(dsl.avg(DSL.ref("response", INTEGER))), ImmutableList.of()); + filter, ImmutableList.of(DSL.named("avg(response)", + dsl.avg(DSL.ref("response", INTEGER)))), ImmutableList.of()); assertNull(aggregation.accept(new PhysicalPlanNodeVisitor() { }, null)); @@ -141,6 +152,11 @@ public String visitFilter(FilterOperator node, Integer tabs) { return name(node, "Filter->", tabs); } + @Override + public String visitHead(HeadOperator node, Integer tabs) { + return name(node, "Head->", tabs); + } + @Override public String visitAggregation(AggregationOperator node, Integer tabs) { return name(node, "Aggregation->", tabs); diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanTestBase.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanTestBase.java index 9bc4fdc38b..5589b37664 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanTestBase.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanTestBase.java @@ -43,7 +43,32 @@ public class PhysicalPlanTestBase { @Autowired protected DSL dsl; - private static final List inputs = new ImmutableList.Builder() + protected static final List countTestInputs = new ImmutableList.Builder() + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 1, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 2, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 3, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 4, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 5, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 6, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 7, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 8, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 9, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 10, "testString", "asdf"))) + .add(ExprValueUtils.tupleValue(ImmutableMap + .of("id", 11, "testString", "asdf"))) + .build(); + + protected static final List inputs = new ImmutableList.Builder() .add(ExprValueUtils.tupleValue(ImmutableMap .of("ip", "209.160.24.63", "action", "GET", "response", 200, "referer", "www.amazon.com"))) @@ -118,4 +143,32 @@ public ExprValue next() { return iterator.next(); } } + + protected static class CountTestScan extends PhysicalPlan { + private final Iterator iterator; + + public CountTestScan() { + iterator = countTestInputs.iterator(); + } + + @Override + public R accept(PhysicalPlanNodeVisitor visitor, C context) { + return null; + } + + @Override + public List getChild() { + return ImmutableList.of(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public ExprValue next() { + return iterator.next(); + } + } } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperatorTest.java index a8bf626a37..a2b7be1c62 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperatorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RareTopNOperatorTest.java @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.planner.physical; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java index ddccf4bbbf..d43bc3a83d 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java @@ -43,8 +43,9 @@ public class RenameOperatorTest extends PhysicalPlanTestBase { public void avg_aggregation_rename() { PhysicalPlan plan = new RenameOperator( new AggregationOperator(new TestScan(), - Collections.singletonList(dsl.avg(DSL.ref("response", INTEGER))), - Collections.singletonList(DSL.ref("action", STRING))), + Collections + .singletonList(DSL.named("avg(response)", dsl.avg(DSL.ref("response", INTEGER)))), + Collections.singletonList(DSL.named("action", DSL.ref("action", STRING)))), ImmutableMap.of(DSL.ref("avg(response)", DOUBLE), DSL.ref("avg", DOUBLE)) ); List result = execute(plan); diff --git a/docs/experiment/ppl/RFC_ Pipe Processing Language.pdf b/docs/experiment/ppl/RFC_ Pipe Processing Language.pdf new file mode 100644 index 0000000000..33f6d0b07a Binary files /dev/null and b/docs/experiment/ppl/RFC_ Pipe Processing Language.pdf differ diff --git a/docs/experiment/ppl/cmd/head.rst b/docs/experiment/ppl/cmd/head.rst new file mode 100644 index 0000000000..d88dcc1800 --- /dev/null +++ b/docs/experiment/ppl/cmd/head.rst @@ -0,0 +1,102 @@ +============= +head +============= + +.. rubric:: Table of contents + +.. contents:: + :local: + :depth: 2 + + +Description +============ +| The ``head`` command returns the first N number of specified results in search order. + + +Syntax +============ +head [keeplast = (true | false)] [while "("")"] [N] + +* keeplast: optional. use in conjunction with the while argument to determine whether the last result in the result set is retained. The last result returned is the result that caused the to evaluate to false or NULL. Set keeplast=true to retain the last result in the result set. Set keeplast=false to discard the last result. **Default:** true +* while: optional. expression that evaluates to either true or false. statistical functions can not be used in the expression. **Default:** false +* N: optional. number of results to return. **Default:** 10 + +Example 1: Get first 10 results +=========================================== + +The example show first 10 results from accounts index. + +PPL query:: + + od> source=accounts | fields firstname, age | head; + fetched rows / total rows = 10/10 + +---------------+-----------+ + | firstname | age | + |---------------+-----------| + | Amber | 32 | + | Hattie | 36 | + | Nanette | 28 | + | Dale | 33 | + | Elinor | 36 | + | Virginia | 39 | + | Dillard | 34 | + | Mcgee | 39 | + | Aurelia | 37 | + | Fulton | 23 | + +---------------+-----------+ + +Example 2: Get first N results +=========================================== + +The example show first N results from accounts index. + +PPL query:: + + od> source=accounts | fields firstname, age | head 3; + fetched rows / total rows = 3/3 + +---------------+-----------+ + | firstname | age | + |---------------+-----------| + | Amber | 32 | + | Hattie | 36 | + | Nanette | 28 | + +---------------+-----------+ + +Example 3: Get first N results with while condition +=========================================================== + +The example show first N results from accounts index with while condition. + +PPL query:: + + od> source=accounts | fields firstname, age | sort age | head while(age < 21) 7; + fetched rows / total rows = 4/4 + +---------------+-----------+ + | firstname | age | + |---------------+-----------| + | Claudia | 20 | + | Copeland | 20 | + | Cornelia | 20 | + | Schultz | 20 | + | Simpson | 21 | + +---------------+-----------+ + +Example 4: Get first N results with while condition and last result which failed condition +========================================================================================== + +The example show first N results with while condition and last result which failed condition. + +PPL query:: + + od> source=accounts | fields firstname, age | sort age | head keeplast=false while(age < 21) 7; + fetched rows / total rows = 4/4 + +---------------+-----------+ + | firstname | age | + |---------------+-----------| + | Claudia | 20 | + | Copeland | 20 | + | Cornelia | 20 | + | Schultz | 20 | + +---------------+-----------+ + diff --git a/docs/experiment/ppl/cmd/stats.rst b/docs/experiment/ppl/cmd/stats.rst index ed4f45aada..3bdf67c818 100644 --- a/docs/experiment/ppl/cmd/stats.rst +++ b/docs/experiment/ppl/cmd/stats.rst @@ -13,6 +13,18 @@ Description ============ | Using ``stats`` command to calculate the aggregation from search result. +The following table catalogs the aggregation functions and also indicates how each one handles NULL/MISSING values is handled: + ++----------+-------------+-------------+ +| Function | NULL | MISSING | ++----------+-------------+-------------+ +| COUNT | Not counted | Not counted | ++----------+-------------+-------------+ +| SUM | Ignore | Ignore | ++----------+-------------+-------------+ +| AVG | Ignore | Ignore | ++----------+-------------+-------------+ + Syntax ============ diff --git a/docs/experiment/ppl/index.rst b/docs/experiment/ppl/index.rst index fb2964ccd7..0602c86bde 100644 --- a/docs/experiment/ppl/index.rst +++ b/docs/experiment/ppl/index.rst @@ -43,6 +43,8 @@ OpenDistro PPL Reference Manual - `where command `_ + - `head command `_ + - `rare command `_ - `top command `_ diff --git a/docs/user/dql/functions.rst b/docs/user/dql/functions.rst index fbb261464e..c6b1004dc4 100644 --- a/docs/user/dql/functions.rst +++ b/docs/user/dql/functions.rst @@ -1,4 +1,3 @@ - ============= SQL Functions ============= @@ -7,7 +6,7 @@ SQL Functions .. contents:: :local: - :depth: 1 + :depth: 2 Introduction ============ @@ -16,22 +15,38 @@ There is support for a wide variety of SQL functions. We are intend to generate Most of the specifications can be self explained just as a regular function with data type as argument. The only notation that needs elaboration is generic type ``T`` which binds to an actual type and can be used as return type. For example, ``ABS(NUMBER T) -> T`` means function ``ABS`` accepts an numerical argument of type ``T`` which could be any sub-type of ``NUMBER`` type and returns the actual type of ``T`` as return type. The actual type binds to generic type at runtime dynamically. + +Type Conversion +=============== + +CAST +---- + +Description +>>>>>>>>>>> + +Specification is undefined and type check is skipped for now + + +Mathematical Functions +====================== + ABS -=== +--- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. ABS(NUMBER T) -> T ACOS -==== +---- Description ------------ +>>>>>>>>>>> Usage: acos(x) calculate the arc cosine of x. Returns NULL if x is not in the range -1 to 1. @@ -51,32 +66,21 @@ Example:: ADD -=== +--- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. ADD(NUMBER T, NUMBER) -> T -ASCII -===== - -Description ------------ - -Specifications: - -1. ASCII(STRING T) -> INTEGER - - ASIN -==== +---- Description ------------ +>>>>>>>>>>> Usage: asin(x) calculate the arc sine of x. Returns NULL if x is not in the range -1 to 1. @@ -96,10 +100,10 @@ Example:: ATAN -==== +---- Description ------------ +>>>>>>>>>>> Usage: atan(x) calculates the arc tangent of x. atan(y, x) calculates the arc tangent of y / x, except that the signs of both arguments are used to determine the quadrant of the result. @@ -119,10 +123,10 @@ Example:: ATAN2 -===== +----- Description ------------ +>>>>>>>>>>> Usage: atan2(y, x) calculates the arc tangent of y / x, except that the signs of both arguments are used to determine the quadrant of the result. @@ -141,85 +145,33 @@ Example:: +--------------------+ -CAST -==== - -Description ------------ - -Specification is undefined and type check is skipped for now - CBRT -==== +---- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. CBRT(NUMBER T) -> T CEIL -==== +---- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. CEIL(NUMBER T) -> T -CONCAT -====== - -Description ------------ - -Usage: CONCAT(str1, str2) returns str1 and str strings concatenated together. - -Argument type: STRING, STRING - -Return Type: STRING - -Example:: - - od> SELECT CONCAT('hello', 'world') - fetched rows / total rows = 1/1 - +----------------------------+ - | CONCAT('hello', 'world') | - |----------------------------| - | helloworld | - +----------------------------+ - -CONCAT_WS -========= - -Description ------------ - -Usage: CONCAT_WS(sep, str1, str2) returns str1 concatenated with str2 using sep as a separator between them. - -Argument type: STRING, STRING, STRING - -Return Type: STRING - -Example:: - - od> SELECT CONCAT_WS(',', 'hello', 'world') - fetched rows / total rows = 1/1 - +------------------------------------+ - | CONCAT_WS(',', 'hello', 'world') | - |------------------------------------| - | hello,world | - +------------------------------------+ - CONV -==== +---- Description ------------ +>>>>>>>>>>> Usage: CONV(x, a, b) converts the number x from a base to b base. @@ -238,10 +190,10 @@ Example:: +----------------------+----------------------+-------------------+---------------------+ COS -=== +--- Description ------------ +>>>>>>>>>>> Usage: cos(x) calculate the cosine of x, where x is given in radians. @@ -261,21 +213,21 @@ Example:: COSH -==== +---- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. COSH(NUMBER T) -> DOUBLE COT -=== +--- Description ------------ +>>>>>>>>>>> Usage: cot(x) calculate the cotangent of x. Returns out-of-range error if x equals to 0. @@ -295,10 +247,10 @@ Example:: CRC32 -===== +----- Description ------------ +>>>>>>>>>>> Usage: Calculates a cyclic redundancy check value and returns a 32-bit unsigned value. @@ -317,56 +269,11 @@ Example:: +------------------+ -CURDATE -======= - -Description ------------ - -Specifications: - -1. CURDATE() -> DATE - - -DATE -==== - -Description ------------ - -Specifications: - -1. DATE(DATE) -> DATE - - -DATE_FORMAT -=========== - -Description ------------ - -Specifications: - -1. DATE_FORMAT(DATE, STRING) -> STRING -2. DATE_FORMAT(DATE, STRING, STRING) -> STRING - - -DAYOFMONTH -========== - -Description ------------ - -Specifications: - -1. DAYOFMONTH(DATE) -> INTEGER - - DEGREES -======= +------- Description ------------ +>>>>>>>>>>> Usage: degrees(x) converts x from radians to degrees. @@ -386,21 +293,21 @@ Example:: DIVIDE -====== +------ Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. DIVIDE(NUMBER T, NUMBER) -> T E -= +- Description ------------ +>>>>>>>>>>> Usage: E() returns the Euler's number @@ -418,38 +325,39 @@ Example:: EXP -=== +--- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. EXP(NUMBER T) -> T EXPM1 -===== +----- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. EXPM1(NUMBER T) -> T FLOOR -===== +----- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. FLOOR(NUMBER T) -> T +<<<<<<< HEAD IF == @@ -494,129 +402,54 @@ Specifications: 1. LEFT(STRING T, INTEGER) -> T -LENGTH -====== - -Description ------------ - -Usage: length(str) returns length of string measured in bytes. - -Argument type: STRING - -Return Type: INTEGER - -Example:: - - od> SELECT LENGTH('helloworld') - fetched rows / total rows = 1/1 - +------------------------+ - | LENGTH('helloworld') | - |------------------------| - | 10 | - +------------------------+ +======= +>>>>>>> 485208e069e8e916f39991cf9a79903dc5d0fa78 LN -== +-- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. LN(NUMBER T) -> DOUBLE -LOCATE -====== - -Description ------------ - -Specifications: - -1. LOCATE(STRING, STRING, INTEGER) -> INTEGER -2. LOCATE(STRING, STRING) -> INTEGER - - LOG -=== +--- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. LOG(NUMBER T) -> DOUBLE 2. LOG(NUMBER T, NUMBER) -> DOUBLE LOG2 -==== +---- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. LOG2(NUMBER T) -> DOUBLE LOG10 -===== +----- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. LOG10(NUMBER T) -> DOUBLE -LOWER -===== - -Description ------------ - -Usage: lower(string) converts the string to lowercase. - -Argument type: STRING - -Return Type: STRING - -Example:: - - od> SELECT LOWER('helloworld'), LOWER('HELLOWORLD') - fetched rows / total rows = 1/1 - +-----------------------+-----------------------+ - | LOWER('helloworld') | LOWER('HELLOWORLD') | - |-----------------------+-----------------------| - | helloworld | helloworld | - +-----------------------+-----------------------+ - -LTRIM -===== - -Description ------------ - -Usage: ltrim(str) trims leading space characters from the string. - -Argument type: STRING - -Return Type: STRING - -Example:: - - od> SELECT LTRIM(' hello'), LTRIM('hello ') - fetched rows / total rows = 1/1 - +---------------------+---------------------+ - | LTRIM(' hello') | LTRIM('hello ') | - |---------------------+---------------------| - | hello | hello | - +---------------------+---------------------+ - MAKETIME ======== @@ -628,11 +461,13 @@ Specifications: 1. MAKETIME(INTEGER, INTEGER, INTEGER) -> DATE -MOD ======= +>>>>>>> 485208e069e8e916f39991cf9a79903dc5d0fa78 +MOD +--- Description ------------ +>>>>>>>>>>> Usage: MOD(n, m) calculates the remainder of the number n divided by m. @@ -651,55 +486,22 @@ Example:: +-------------+---------------+ -MONTH -===== +MULTIPLY +-------- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: -1. MONTH(DATE) -> INTEGER - - -MONTHNAME -========= - -Description ------------ - -Specifications: - -1. MONTHNAME(DATE) -> STRING - - -MULTIPLY -======== - -Description ------------ - -Specifications: - -1. MULTIPLY(NUMBER T, NUMBER) -> NUMBER - - -NOW -=== - -Description ------------ - -Specifications: - -1. NOW() -> DATE +1. MULTIPLY(NUMBER T, NUMBER) -> NUMBER PI -== +-- Description ------------ +>>>>>>>>>>> Usage: PI() returns the constant pi @@ -717,10 +519,10 @@ Example:: POW -=== +--- Description ------------ +>>>>>>>>>>> Usage: POW(x, y) calculates the value of x raised to the power of y. Bad inputs return NULL result. @@ -728,6 +530,8 @@ Argument type: INTEGER/LONG/FLOAT/DOUBLE Return type: DOUBLE +Synonyms: `POWER`_ + Example:: od> SELECT POW(3, 2), POW(-3, 2), POW(3, -2) @@ -740,10 +544,10 @@ Example:: POWER -===== +----- Description ------------ +>>>>>>>>>>> Usage: POWER(x, y) calculates the value of x raised to the power of y. Bad inputs return NULL result. @@ -751,6 +555,8 @@ Argument type: INTEGER/LONG/FLOAT/DOUBLE Return type: DOUBLE +Synonyms: `POW`_ + Example:: od> SELECT POWER(3, 2), POWER(-3, 2), POWER(3, -2) @@ -763,10 +569,10 @@ Example:: RADIANS -======= +------- Description ------------ +>>>>>>>>>>> Usage: radians(x) converts x from degrees to radians. @@ -786,10 +592,10 @@ Example:: RAND -==== +---- Description ------------ +>>>>>>>>>>> Usage: RAND()/RAND(N) returns a random floating-point value in the range 0 <= value < 1.0. If integer N is specified, the seed is initialized prior to execution. One implication of this behavior is with identical argument N, rand(N) returns the same value each time, and thus produces a repeatable sequence of column values. @@ -808,44 +614,22 @@ Example:: +------------+ -REPLACE -======= - -Description ------------ - -Specifications: - -1. REPLACE(STRING T, STRING, STRING) -> T - - -RIGHT -===== - -Description ------------ - -Specifications: - -1. RIGHT(STRING T, INTEGER) -> T - - RINT -==== +---- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. RINT(NUMBER T) -> T ROUND -===== +----- Description ------------ +>>>>>>>>>>> Usage: ROUND(x, d) rounds the argument x to d decimal places, d defaults to 0 if not specified @@ -867,33 +651,11 @@ Example:: +----------------+-------------------+--------------------+----------------+ -RTRIM -===== - -Description ------------ - -Usage: rtrim(str) trims trailing space characters from the string. - -Argument type: STRING - -Return Type: STRING - -Example:: - - od> SELECT RTRIM(' hello'), RTRIM('hello ') - fetched rows / total rows = 1/1 - +---------------------+---------------------+ - | RTRIM(' hello') | RTRIM('hello ') | - |---------------------+---------------------| - | hello | hello | - +---------------------+---------------------+ - SIGN -==== +---- Description ------------ +>>>>>>>>>>> Usage: Returns the sign of the argument as -1, 0, or 1, depending on whether the number is negative, zero, or positive @@ -913,21 +675,21 @@ Example:: SIGNUM -====== +------ Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. SIGNUM(NUMBER T) -> T SIN -=== +--- Description ------------ +>>>>>>>>>>> Usage: sin(x) calculate the sine of x, where x is given in radians. @@ -947,21 +709,21 @@ Example:: SINH -==== +---- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. SINH(NUMBER T) -> DOUBLE SQRT -==== +---- Description ------------ +>>>>>>>>>>> Usage: Calculates the square root of a non-negative number @@ -984,10 +746,10 @@ Example:: STRCMP -========= +------ Description ------------ +>>>>>>>>>>> Usage: strcmp(str1, str2) returns 0 if strings are same, -1 if first arg < second arg according to current sort order, and 1 otherwise. @@ -1005,46 +767,24 @@ Example:: | -1 | 0 | +----------------------------+----------------------------+ -SUBSTRING -========= - -Description ------------ -Usage: substring(str, start) or substring(str, start, length) returns substring using start and length. With no length, entire string from start is returned. - -Argument type: STRING, INTEGER, INTEGER - -Return Type: STRING - -Synonyms: SUBSTR - -Example:: - - od> SELECT SUBSTRING('helloworld', 5), SUBSTRING('helloworld', 5, 3) - fetched rows / total rows = 1/1 - +------------------------------+---------------------------------+ - | SUBSTRING('helloworld', 5) | SUBSTRING('helloworld', 5, 3) | - |------------------------------+---------------------------------| - | oworld | owo | - +------------------------------+---------------------------------+ SUBTRACT -======== +-------- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: 1. SUBTRACT(NUMBER T, NUMBER) -> T TAN -=== +--- Description ------------ +>>>>>>>>>>> Usage: tan(x) calculate the tangent of x, where x is given in radians. @@ -1063,24 +803,343 @@ Example:: +----------+ -TIMESTAMP -========= +TRUNCATE +-------- + +Description +>>>>>>>>>>> + +Usage: TRUNCATE(x, d) returns the number x, truncated to d decimal place + +Argument type: INTEGER/LONG/FLOAT/DOUBLE + +Return type map: + +INTEGER/LONG -> LONG +FLOAT/DOUBLE -> DOUBLE + +Example:: + + fetched rows / total rows = 1/1 + +----------------------+-----------------------+-------------------+ + | TRUNCATE(56.78, 1) | TRUNCATE(56.78, -1) | TRUNCATE(56, 1) | + |----------------------+-----------------------+-------------------| + | 56.7 | 50 | 56 | + +----------------------+-----------------------+-------------------+ + + +Date and Time Functions +======================= + +CURDATE +------- Description +>>>>>>>>>>> + +Specifications: + +1. CURDATE() -> DATE + +DATE +---- + +Description +>>>>>>>>>>> + +Usage: date(expr) constructs a date type with the input string expr as a date. If the argument is of date/datetime/timestamp, it extracts the date value part from the expression. + +Argument type: STRING/DATE/DATETIME/TIMESTAMP + +Return type: DATE + +Example:: + + >od SELECT DATE('2020-08-26'), DATE(TIMESTAMP('2020-08-26 13:49:00')) + fetched rows / total rows = 1/1 + +----------------------+------------------------------------------+ + | DATE('2020-08-26') | DATE(TIMESTAMP('2020-08-26 13:49:00')) | + |----------------------+------------------------------------------| + | DATE '2020-08-26' | DATE '2020-08-26' | + +----------------------+------------------------------------------+ + + +ADDDATE +------- + +Description +>>>>>>>>>>> + +Usage: adddate(date, INTERVAL expr unit) adds the time interval of second argument to date; adddate(date, days) adds the second argument as integer number of days to date. + +Argument type: DATE/DATETIME/TIMESTAMP, INTERVAL/LONG + +Return type map: + +(DATE/DATETIME/TIMESTAMP, INTERVAL) -> DATETIME +(DATE, LONG) -> DATE +(DATETIME/TIMESTAMP, LONG) -> DATETIME + +Synonyms: `DATE_ADD`_ + +Example:: + + >od SELECT ADDDATE(DATE('2020-08-26'), INTERVAL 1 HOUR), ADDDATE(DATE('2020-08-26'), 1) + fetched rows / total rows = 1/1 + +------------------------------------------------+----------------------------------+ + | ADDDATE(DATE('2020-08-26'), INTERVAL 1 HOUR) | ADDDATE(DATE('2020-08-26'), 1) | + |------------------------------------------------+----------------------------------| + | DATETIME '2020-08-26 01:00:00' | DATE '2020-08-26' | + +------------------------------------------------+----------------------------------+ + + +DATE_ADD +-------- + +todo + + +DATE_FORMAT ----------- -Specifications: +Description +>>>>>>>>>>> -1. TIMESTAMP(DATE) -> DATE +Specifications: +1. DATE_FORMAT(DATE, STRING) -> STRING +2. DATE_FORMAT(DATE, STRING, STRING) -> STRING -TRIM -==== + +DAYOFMONTH +---------- + +Description +>>>>>>>>>>> + +Specifications: + +1. DAYOFMONTH(DATE) -> INTEGER + + +MAKETIME +-------- + +Description +>>>>>>>>>>> + +Specifications: + +1. MAKETIME(INTEGER, INTEGER, INTEGER) -> DATE + + +MONTH +----- + +Description +>>>>>>>>>>> + +Specifications: + +1. MONTH(DATE) -> INTEGER + + +MONTHNAME +--------- + +Description +>>>>>>>>>>> + +Specifications: + +1. MONTHNAME(DATE) -> STRING + + +NOW +--- + +Description +>>>>>>>>>>> + +Specifications: + +1. NOW() -> DATE + + +TIME +---- + +Description +>>>>>>>>>>> + +Usage: time(expr) constructs a time type with the input string expr as a time. If the argument is of date/datetime/time/timestamp, it extracts the time value part from the expression. + +Argument type: STRING/DATE/DATETIME/TIME/TIMESTAMP + +Return type: TIME + +Example:: + + >od SELECT TIME('13:49:00'), TIME(TIMESTAMP('2020-08-26 13:49:00')) + fetched rows / total rows = 1/1 + +--------------------+------------------------------------------+ + | TIME('13:49:00') | TIME(TIMESTAMP('2020-08-26 13:49:00')) | + |--------------------+------------------------------------------| + | TIME '13:49:00' | TIME '13:49:00' | + +--------------------+------------------------------------------+ + + +TIMESTAMP +--------- + +Description +>>>>>>>>>>> + +Usage: timestamp(expr) construct a timestamp type with the input string expr as an timestamp. If the argument is of date/datetime/timestamp type, cast expr to timestamp type with default timezone UTC. + +Argument type: STRING/DATE/DATETIME/TIMESTAMP + +Return type: TIMESTAMP + +Example:: + + >od SELECT TIMESTAMP('2020-08-26 13:49:00') + fetched rows / total rows = 1/1 + +------------------------------------+ + | TIMESTAMP('2020-08-26 13:49:00') | + |------------------------------------| + | TIMESTAMP '2020-08-26 13:49:00 | + +------------------------------------+ + + +YEAR +---- + +Description +>>>>>>>>>>> + +Specifications: + +1. YEAR(DATE) -> INTEGER + + + +String Functions +================ + +ASCII +----- + +Description +>>>>>>>>>>> + +Specifications: + +1. ASCII(STRING T) -> INTEGER + + +CONCAT +------ + +Description +>>>>>>>>>>> + +Usage: CONCAT(str1, str2) returns str1 and str strings concatenated together. + +Argument type: STRING, STRING + +Return Type: STRING + +Example:: + + od> SELECT CONCAT('hello', 'world') + fetched rows / total rows = 1/1 + +----------------------------+ + | CONCAT('hello', 'world') | + |----------------------------| + | helloworld | + +----------------------------+ + + +CONCAT_WS +--------- + +Description +>>>>>>>>>>> + +Usage: CONCAT_WS(sep, str1, str2) returns str1 concatenated with str2 using sep as a separator between them. + +Argument type: STRING, STRING, STRING + +Return Type: STRING + +Example:: + + od> SELECT CONCAT_WS(',', 'hello', 'world') + fetched rows / total rows = 1/1 + +------------------------------------+ + | CONCAT_WS(',', 'hello', 'world') | + |------------------------------------| + | hello,world | + +------------------------------------+ + + +LEFT +---- + +Description +>>>>>>>>>>> + +Specifications: + +1. LEFT(STRING T, INTEGER) -> T + +LENGTH +------ + +Description +>>>>>>>>>>> + +Specifications: + +1. LENGTH(STRING) -> INTEGER + +Usage: length(str) returns length of string measured in bytes. + +Argument type: STRING + +Return Type: INTEGER + +Example:: + + od> SELECT LENGTH('helloworld') + fetched rows / total rows = 1/1 + +------------------------+ + | LENGTH('helloworld') | + |------------------------| + | 10 | + +------------------------+ + + +LOCATE +------ + +Description +>>>>>>>>>>> + +Specifications: + +1. LOCATE(STRING, STRING, INTEGER) -> INTEGER +2. LOCATE(STRING, STRING) -> INTEGER + + +LOWER +===== Description ----------- -Usage: trim(str) trims leading and trailing space characters from the string. +Usage: lower(string) converts the string to lowercase. Argument type: STRING @@ -1088,44 +1147,132 @@ Return Type: STRING Example:: - od> SELECT TRIM(' hello'), TRIM('hello ') + od> SELECT LOWER('helloworld'), LOWER('HELLOWORLD') fetched rows / total rows = 1/1 - +--------------------+--------------------+ - | TRIM(' hello') | TRIM('hello ') | - |--------------------+--------------------| - | hello | hello | - +--------------------+--------------------+ + +-----------------------+-----------------------+ + | LOWER('helloworld') | LOWER('HELLOWORLD') | + |-----------------------+-----------------------| + | helloworld | helloworld | + +-----------------------+-----------------------+ -TRUNCATE -======== + +LTRIM +===== Description ----------- -Usage: TRUNCATE(x, d) returns the number x, truncated to d decimal place +Usage: ltrim(str) trims leading space characters from the string. -Argument type: INTEGER/LONG/FLOAT/DOUBLE +Argument type: STRING -Return type map: +Return Type: STRING -INTEGER/LONG -> LONG -FLOAT/DOUBLE -> DOUBLE +Example:: + + od> SELECT LTRIM(' hello'), LTRIM('hello ') + fetched rows / total rows = 1/1 + +---------------------+---------------------+ + | LTRIM(' hello') | LTRIM('hello ') | + |---------------------+---------------------| + | hello | hello | + +---------------------+---------------------+ + + +REPLACE +------- + +Description +>>>>>>>>>>> + +Specifications: + +1. REPLACE(STRING T, STRING, STRING) -> T + + +RIGHT +----- + +Description +>>>>>>>>>>> + +Specifications: + +1. RIGHT(STRING T, INTEGER) -> T + + +RTRIM +----- + +Description +>>>>>>>>>>> + +Usage: rtrim(str) trims trailing space characters from the string. + +Argument type: STRING + +Return Type: STRING Example:: + od> SELECT RTRIM(' hello'), RTRIM('hello ') fetched rows / total rows = 1/1 - +----------------------+-----------------------+-------------------+ - | TRUNCATE(56.78, 1) | TRUNCATE(56.78, -1) | TRUNCATE(56, 1) | - |----------------------+-----------------------+-------------------| - | 56.7 | 50 | 56 | - +----------------------+-----------------------+-------------------+ + +---------------------+---------------------+ + | RTRIM(' hello') | RTRIM('hello ') | + |---------------------+---------------------| + | hello | hello | + +---------------------+---------------------+ + + +SUBSTRING +--------- + +Description +>>>>>>>>>>> + +Usage: substring(str, start) or substring(str, start, length) returns substring using start and length. With no length, entire string from start is returned. + +Argument type: STRING, INTEGER, INTEGER + +Return Type: STRING + +Synonyms: SUBSTR + +Example:: + + od> SELECT SUBSTRING('helloworld', 5), SUBSTRING('helloworld', 5, 3) + fetched rows / total rows = 1/1 + +------------------------------+---------------------------------+ + | SUBSTRING('helloworld', 5) | SUBSTRING('helloworld', 5, 3) | + |------------------------------+---------------------------------| + | oworld | owo | + +------------------------------+---------------------------------+ + + +TRIM +---- + +Description +>>>>>>>>>>> + +Return Type: STRING + +Example:: + + od> SELECT TRIM(' hello'), TRIM('hello ') + fetched rows / total rows = 1/1 + +--------------------+--------------------+ + | TRIM(' hello') | TRIM('hello ') | + |--------------------+--------------------| + | hello | hello | + +--------------------+--------------------+ UPPER -===== +----- Description ------------ +>>>>>>>>>>> Usage: upper(string) converts the string to uppercase. @@ -1143,14 +1290,38 @@ Example:: | HELLOWORLD | HELLOWORLD | +-----------------------+-----------------------+ -YEAR -==== + +Conditional Functions +===================== + +IF +-- Description ------------ +>>>>>>>>>>> -Specifications: +Specifications: -1. YEAR(DATE) -> INTEGER +1. IF(BOOLEAN, ES_TYPE, ES_TYPE) -> ES_TYPE + + +IFNULL +------ +Description +>>>>>>>>>>> + +Specifications: + +1. IFNULL(ES_TYPE, ES_TYPE) -> ES_TYPE + +ISNULL +------ + +Description +>>>>>>>>>>> + +Specifications: + +1. ISNULL(ES_TYPE) -> INTEGER diff --git a/elasticsearch/build.gradle b/elasticsearch/build.gradle index 7aeebc5ab6..e18a3a9a7e 100644 --- a/elasticsearch/build.gradle +++ b/elasticsearch/build.gradle @@ -19,8 +19,8 @@ dependencies { testImplementation('org.junit.jupiter:junit-jupiter:5.6.2') testCompile group: 'org.hamcrest', name: 'hamcrest-library', version: '2.1' - testCompile group: 'org.mockito', name: 'mockito-core', version: '3.3.3' - testCompile group: 'org.mockito', name: 'mockito-junit-jupiter', version: '3.3.3' + testCompile group: 'org.mockito', name: 'mockito-core', version: '3.5.0' + testCompile group: 'org.mockito', name: 'mockito-junit-jupiter', version: '3.5.0' testCompile group: 'org.elasticsearch.client', name: 'elasticsearch-rest-high-level-client', version: "${es_version}" } diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactory.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactory.java index 5b2fe8c07f..95cf4bc670 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactory.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactory.java @@ -56,14 +56,17 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import lombok.RequiredArgsConstructor; +import java.util.function.Function; +import lombok.AllArgsConstructor; +import lombok.Setter; import org.elasticsearch.common.time.DateFormatters; /** Construct ExprValue from Elasticsearch response. */ -@RequiredArgsConstructor +@AllArgsConstructor public class ElasticsearchExprValueFactory { /** The Mapping of Field and ExprType. */ - private final Map typeMapping; + @Setter + private Map typeMapping; private static final DateTimeFormatter DATE_TIME_FORMATTER = new DateTimeFormatterBuilder() @@ -150,13 +153,13 @@ public ExprValue construct(String field, Object value) { ExprType type = type(field); if (type.equals(INTEGER)) { - return constructInteger((Integer) value); + return constructInteger(parseNumberValue(value, Integer::valueOf, Number::intValue)); } else if (type.equals(LONG)) { - return constructLong((Long) value); + return constructLong(parseNumberValue(value, Long::valueOf, Number::longValue)); } else if (type.equals(FLOAT)) { - return constructFloat((Float) value); + return constructFloat(parseNumberValue(value, Float::valueOf, Number::floatValue)); } else if (type.equals(DOUBLE)) { - return constructDouble((Double) value); + return constructDouble(parseNumberValue(value, Double::valueOf, Number::doubleValue)); } else if (type.equals(STRING)) { return constructString((String) value); } else if (type.equals(BOOLEAN)) { @@ -180,6 +183,18 @@ public ExprValue construct(String field, Object value) { } } + /** + * Elastisearch could return value String value even the type is Number. + */ + private T parseNumberValue(Object value, Function stringTFunction, + Function numberTFunction) { + if (value instanceof String) { + return stringTFunction.apply((String) value); + } else { + return numberTFunction.apply((Number) value); + } + } + private ExprType type(String field) { if (typeMapping.containsKey(field)) { return typeMapping.get(field); diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngine.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngine.java index 97163f7e5b..3f4cd2bf37 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngine.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngine.java @@ -20,8 +20,12 @@ import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.client.ElasticsearchClient; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.executor.protector.ExecutionProtector; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.ElasticsearchIndexScan; import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine; +import com.amazon.opendistroforelasticsearch.sql.executor.Explain; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; +import com.amazon.opendistroforelasticsearch.sql.storage.TableScanOperator; +import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.List; import lombok.RequiredArgsConstructor; @@ -56,4 +60,27 @@ public void execute(PhysicalPlan physicalPlan, ResponseListener l } }); } + + @Override + public void explain(PhysicalPlan plan, ResponseListener listener) { + client.schedule(() -> { + try { + Explain esExplain = new Explain() { + @Override + public ExplainResponseNode visitTableScan(TableScanOperator node, Object context) { + return explain(node, context, explainNode -> { + ElasticsearchIndexScan indexScan = (ElasticsearchIndexScan) node; + explainNode.setDescription(ImmutableMap.of( + "request", indexScan.getRequest().toString())); + }); + } + }; + + listener.onResponse(esExplain.apply(plan)); + } catch (Exception e) { + listener.onFailure(e); + } + }); + } + } diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/protector/ElasticsearchExecutionProtector.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/protector/ElasticsearchExecutionProtector.java index cef0164ca0..54630987d1 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/protector/ElasticsearchExecutionProtector.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/protector/ElasticsearchExecutionProtector.java @@ -22,6 +22,7 @@ import com.amazon.opendistroforelasticsearch.sql.planner.physical.DedupeOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.EvalOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.FilterOperator; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.HeadOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.physical.ProjectOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.RareTopNOperator; @@ -97,6 +98,16 @@ public PhysicalPlan visitDedupe(DedupeOperator node, Object context) { node.getAllowedDuplication(), node.getKeepEmpty(), node.getConsecutive()); } + @Override + public PhysicalPlan visitHead(HeadOperator node, Object context) { + return new HeadOperator( + visitInput(node.getInput(), context), + node.getKeepLast(), + node.getWhileExpr(), + node.getNumber() + ); + } + @Override public PhysicalPlan visitSort(SortOperator node, Object context) { return new SortOperator(visitInput(node.getInput(), context), node.getCount(), diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequest.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequest.java index a599b63ec8..fc66e8a091 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequest.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequest.java @@ -17,10 +17,12 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.request; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.response.ElasticsearchResponse; import com.google.common.annotations.VisibleForTesting; import java.util.function.Consumer; import java.util.function.Function; +import lombok.Builder; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.ToString; @@ -57,6 +59,14 @@ public class ElasticsearchQueryRequest implements ElasticsearchRequest { */ private final SearchSourceBuilder sourceBuilder; + + /** + * ElasticsearchExprValueFactory. + */ + @EqualsAndHashCode.Exclude + @ToString.Exclude + private final ElasticsearchExprValueFactory exprValueFactory; + /** * Indicate the search already done. */ @@ -65,23 +75,24 @@ public class ElasticsearchQueryRequest implements ElasticsearchRequest { /** * Constructor of ElasticsearchQueryRequest. */ - public ElasticsearchQueryRequest(String indexName, int size) { + public ElasticsearchQueryRequest(String indexName, int size, + ElasticsearchExprValueFactory factory) { this.indexName = indexName; this.sourceBuilder = new SearchSourceBuilder(); sourceBuilder.from(0); sourceBuilder.size(size); sourceBuilder.timeout(DEFAULT_QUERY_TIMEOUT); - + this.exprValueFactory = factory; } @Override public ElasticsearchResponse search(Function searchAction, Function scrollAction) { if (searchDone) { - return new ElasticsearchResponse(SearchHits.empty()); + return new ElasticsearchResponse(SearchHits.empty(), exprValueFactory); } else { searchDone = true; - return new ElasticsearchResponse(searchAction.apply(searchRequest())); + return new ElasticsearchResponse(searchAction.apply(searchRequest()), exprValueFactory); } } diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchRequest.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchRequest.java index 60f851b100..4b38e8a0d9 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchRequest.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchRequest.java @@ -17,6 +17,7 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.request; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.response.ElasticsearchResponse; import java.util.function.Consumer; import java.util.function.Function; @@ -53,4 +54,10 @@ ElasticsearchResponse search(Function searchActio * @return SearchSourceBuilder. */ SearchSourceBuilder getSourceBuilder(); + + /** + * Get the ElasticsearchExprValueFactory. + * @return ElasticsearchExprValueFactory. + */ + ElasticsearchExprValueFactory getExprValueFactory(); } diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequest.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequest.java index 87cb9ae2fd..bcbdf440c8 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequest.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequest.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.request; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.response.ElasticsearchResponse; import java.util.Objects; import java.util.function.Consumer; @@ -49,6 +50,11 @@ public class ElasticsearchScrollRequest implements ElasticsearchRequest { /** Index name. */ private final String indexName; + /** Index name. */ + @EqualsAndHashCode.Exclude + @ToString.Exclude + private final ElasticsearchExprValueFactory exprValueFactory; + /** * Scroll id which is set after first request issued. Because ElasticsearchClient is shared by * multi-thread so this state has to be maintained here. @@ -70,7 +76,7 @@ public ElasticsearchResponse search(Function sear } setScrollId(esResponse.getScrollId()); - return new ElasticsearchResponse(esResponse); + return new ElasticsearchResponse(esResponse, exprValueFactory); } @Override diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchAggregationResponseParser.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchAggregationResponseParser.java new file mode 100644 index 0000000000..c37739b129 --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchAggregationResponseParser.java @@ -0,0 +1,92 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.response; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.experimental.UtilityClass; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; +import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; + +/** + * AggregationResponseParser. + */ +@UtilityClass +public class ElasticsearchAggregationResponseParser { + + /** + * Parse Aggregations as a list of field and value map. + * + * @param aggregations aggregations + * @return a list of field and value map + */ + public static List> parse(Aggregations aggregations) { + List aggregationList = aggregations.asList(); + ImmutableList.Builder> builder = new ImmutableList.Builder<>(); + + for (Aggregation aggregation : aggregationList) { + if (aggregation instanceof CompositeAggregation) { + for (CompositeAggregation.Bucket bucket : + ((CompositeAggregation) aggregation).getBuckets()) { + builder.add(parse(bucket)); + } + } else { + builder.add(parseInternal(aggregation)); + } + + } + return builder.build(); + } + + private static Map parse(CompositeAggregation.Bucket bucket) { + Map resultMap = new HashMap<>(); + // The NodeClient return InternalComposite + + // build pair + resultMap.putAll(bucket.getKey()); + + // build pair + for (Aggregation aggregation : bucket.getAggregations()) { + resultMap.putAll(parseInternal(aggregation)); + } + + return resultMap; + } + + private static Map parseInternal(Aggregation aggregation) { + Map resultMap = new HashMap<>(); + if (aggregation instanceof NumericMetricsAggregation.SingleValue) { + resultMap.put( + aggregation.getName(), + handleNanValue(((NumericMetricsAggregation.SingleValue) aggregation).value())); + } else { + throw new IllegalStateException("unsupported aggregation type " + aggregation.getType()); + } + return resultMap; + } + + @VisibleForTesting + protected static Object handleNanValue(double value) { + return Double.isNaN(value) ? null : value; + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponse.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponse.java index 11f88b7029..5aeb0f3a55 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponse.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponse.java @@ -16,27 +16,59 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.response; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; import java.util.Iterator; +import java.util.Map; import lombok.EqualsAndHashCode; import lombok.ToString; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.aggregations.Aggregations; -/** Elasticsearch search response. */ +/** + * Elasticsearch search response. + */ @EqualsAndHashCode @ToString -public class ElasticsearchResponse implements Iterable { +public class ElasticsearchResponse implements Iterable { - /** Search query result (non-aggregation). */ + /** + * Search query result (non-aggregation). + */ private final SearchHits hits; - public ElasticsearchResponse(SearchResponse esResponse) { - this.hits = esResponse.getHits(); // TODO: aggregation result is separate and not in SearchHit[] + /** + * Search aggregation result. + */ + private final Aggregations aggregations; + + /** + * ElasticsearchExprValueFactory used to build ExprValue from search result. + */ + @EqualsAndHashCode.Exclude + private final ElasticsearchExprValueFactory exprValueFactory; + + /** + * Constructor of ElasticsearchResponse. + */ + public ElasticsearchResponse(SearchResponse esResponse, + ElasticsearchExprValueFactory exprValueFactory) { + this.hits = esResponse.getHits(); + this.aggregations = esResponse.getAggregations(); + this.exprValueFactory = exprValueFactory; } - public ElasticsearchResponse(SearchHits hits) { + /** + * Constructor of ElasticsearchResponse with SearchHits. + */ + public ElasticsearchResponse(SearchHits hits, ElasticsearchExprValueFactory exprValueFactory) { this.hits = hits; + this.aggregations = null; + this.exprValueFactory = exprValueFactory; } /** @@ -46,7 +78,11 @@ public ElasticsearchResponse(SearchHits hits) { * @return true for empty */ public boolean isEmpty() { - return (hits.getHits() == null) || (hits.getHits().length == 0); + return (hits.getHits() == null) || (hits.getHits().length == 0) && aggregations == null; + } + + public boolean isAggregationResponse() { + return aggregations != null; } /** @@ -54,8 +90,18 @@ public boolean isEmpty() { * * @return search hit iterator */ - @Override - public Iterator iterator() { - return hits.iterator(); + public Iterator iterator() { + if (isAggregationResponse()) { + return ElasticsearchAggregationResponseParser.parse(aggregations).stream().map(entry -> { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + for (Map.Entry value : entry.entrySet()) { + builder.put(value.getKey(), exprValueFactory.construct(value.getKey(), value.getValue())); + } + return (ExprValue) ExprTupleValue.fromExprValueMap(builder.build()); + }).iterator(); + } else { + return Arrays.stream(hits.getHits()) + .map(hit -> (ExprValue) exprValueFactory.construct(hit.getSourceAsString())).iterator(); + } } } diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndex.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndex.java index ab06267c46..df737f3f3f 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndex.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndex.java @@ -21,12 +21,13 @@ import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.client.ElasticsearchClient; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType; -import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprIpValue; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.mapping.IndexMapping; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.AggregationQueryBuilder; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.filter.FilterQueryBuilder; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.DefaultExpressionSerializer; import com.amazon.opendistroforelasticsearch.sql.planner.DefaultImplementor; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalAggregation; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalFilter; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation; @@ -34,9 +35,11 @@ import com.amazon.opendistroforelasticsearch.sql.storage.Table; import com.google.common.collect.ImmutableMap; import java.util.HashMap; +import java.util.List; import java.util.Map; import lombok.RequiredArgsConstructor; import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.search.aggregations.AggregationBuilder; /** Elasticsearch table (index) implementation. */ @RequiredArgsConstructor @@ -115,6 +118,33 @@ public PhysicalPlan visitFilter(LogicalFilter node, ElasticsearchIndexScan conte return visitChild(node, context); } + @Override + public PhysicalPlan visitAggregation(LogicalAggregation node, + ElasticsearchIndexScan context) { + // Todo, aggregation in the following pattern can be push down + // aggregation -> relation + // aggregation -> filter -> relation + if ((node.getChild().get(0) instanceof LogicalRelation) + || (node.getChild().get(0) instanceof LogicalFilter && node.getChild().get(0) + .getChild().get(0) instanceof LogicalRelation)) { + AggregationQueryBuilder builder = + new AggregationQueryBuilder(new DefaultExpressionSerializer()); + + List aggregationBuilder = + builder.buildAggregationBuilder(node.getAggregatorList(), + node.getGroupByList()); + + context.pushDownAggregation(aggregationBuilder); + context.pushTypeMapping( + builder.buildTypeMapping(node.getAggregatorList(), + node.getGroupByList())); + + return visitChild(node, context); + } else { + return super.visitAggregation(node, context); + } + } + @Override public PhysicalPlan visitRelation(LogicalRelation node, ElasticsearchIndexScan context) { return indexScan; diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScan.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScan.java index ccc31291f9..bac4bc82ed 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScan.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScan.java @@ -21,6 +21,7 @@ import com.amazon.opendistroforelasticsearch.sql.common.setting.Settings; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.client.ElasticsearchClient; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.request.ElasticsearchQueryRequest; @@ -31,12 +32,14 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Map; import lombok.EqualsAndHashCode; +import lombok.Getter; import lombok.ToString; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; /** @@ -49,15 +52,14 @@ public class ElasticsearchIndexScan extends TableScanOperator { /** Elasticsearch client. */ private final ElasticsearchClient client; - private final ElasticsearchExprValueFactory exprValueFactory; - /** Search request. */ @EqualsAndHashCode.Include + @Getter @ToString.Include private final ElasticsearchRequest request; /** Search response for current batch. */ - private Iterator hits; + private Iterator iterator; /** * Todo. @@ -67,8 +69,7 @@ public ElasticsearchIndexScan(ElasticsearchClient client, ElasticsearchExprValueFactory exprValueFactory) { this.client = client; this.request = new ElasticsearchQueryRequest(indexName, - settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)); - this.exprValueFactory = exprValueFactory; + settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT), exprValueFactory); } @Override @@ -82,17 +83,17 @@ public void open() { responses.add(response); response = client.search(request); } - hits = Iterables.concat(responses.toArray(new ElasticsearchResponse[0])).iterator(); + iterator = Iterables.concat(responses.toArray(new ElasticsearchResponse[0])).iterator(); } @Override public boolean hasNext() { - return hits.hasNext(); + return iterator.hasNext(); } @Override public ExprValue next() { - return exprValueFactory.construct(hits.next().getSourceAsString()); + return iterator.next(); } /** @@ -119,6 +120,20 @@ public void pushDown(QueryBuilder query) { } } + /** + * Push down aggregation to DSL request. + * @param aggregationBuilderList aggregation query. + */ + public void pushDownAggregation(List aggregationBuilderList) { + SearchSourceBuilder source = request.getSourceBuilder(); + aggregationBuilderList.forEach(aggregationBuilder -> source.aggregation(aggregationBuilder)); + source.size(0); + } + + public void pushTypeMapping(Map typeMapping) { + request.getExprValueFactory().setTypeMapping(typeMapping); + } + @Override public void close() { super.close(); diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngine.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngine.java index abebafc294..533ee62616 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngine.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngine.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.ExpressionAggregationScriptFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.filter.ExpressionFilterScriptFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; @@ -24,6 +25,7 @@ import java.util.Set; import java.util.function.Function; import lombok.RequiredArgsConstructor; +import org.elasticsearch.script.AggregationScript; import org.elasticsearch.script.FilterScript; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptEngine; @@ -44,10 +46,10 @@ public class ExpressionScriptEngine implements ScriptEngine { * All supported script contexts and function to create factory from expression. */ private static final Map, Function> CONTEXTS = - ImmutableMap.of( - FilterScript.CONTEXT, - ExpressionFilterScriptFactory::new - ); + new ImmutableMap.Builder, Function>() + .put(FilterScript.CONTEXT, ExpressionFilterScriptFactory::new) + .put(AggregationScript.CONTEXT, ExpressionAggregationScriptFactory::new) + .build(); /** * Expression serializer that (de-)serializes expression. diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ScriptUtils.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ScriptUtils.java new file mode 100644 index 0000000000..1fe91e6f47 --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ScriptUtils.java @@ -0,0 +1,41 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script; + +import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType.ES_TEXT_KEYWORD; + +import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; +import lombok.experimental.UtilityClass; + +/** + * Script Utils. + */ +@UtilityClass +public class ScriptUtils { + + /** + * Text field doesn't have doc value (exception thrown even when you call "get") + * Limitation: assume inner field name is always "keyword". + */ + public static String convertTextToKeyword(String fieldName, ExprType fieldType) { + if (fieldType == ES_TEXT_KEYWORD) { + return fieldName + ".keyword"; + } + return fieldName; + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/AggregationQueryBuilder.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/AggregationQueryBuilder.java new file mode 100644 index 0000000000..3dd0db717f --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/AggregationQueryBuilder.java @@ -0,0 +1,92 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl.BucketAggregationBuilder; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl.MetricAggregationBuilder; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import lombok.RequiredArgsConstructor; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregationBuilders; + +/** + * Build the AggregationBuilder from the list of {@link NamedAggregator} + * and list of {@link NamedExpression}. + */ +@RequiredArgsConstructor +public class AggregationQueryBuilder extends ExpressionNodeVisitor { + + /** + * How many composite buckets should be returned. + */ + public static final int AGGREGATION_BUCKET_SIZE = 1000; + + /** + * Bucket Aggregation builder. + */ + private final BucketAggregationBuilder bucketBuilder; + + /** + * Metric Aggregation builder. + */ + private final MetricAggregationBuilder metricBuilder; + + public AggregationQueryBuilder( + ExpressionSerializer serializer) { + this.bucketBuilder = new BucketAggregationBuilder(serializer); + this.metricBuilder = new MetricAggregationBuilder(serializer); + } + + /** + * Build AggregationBuilder. + */ + public List buildAggregationBuilder(List namedAggregatorList, + List groupByList) { + if (groupByList.isEmpty()) { + // no bucket + return ImmutableList + .copyOf(metricBuilder.build(namedAggregatorList).getAggregatorFactories()); + } else { + return Collections.singletonList(AggregationBuilders.composite("composite_buckets", + bucketBuilder.build(groupByList)) + .subAggregations(metricBuilder.build(namedAggregatorList)) + .size(AGGREGATION_BUCKET_SIZE)); + } + } + + /** + * Build ElasticsearchExprValueFactory. + */ + public Map buildTypeMapping( + List namedAggregatorList, + List groupByList) { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + namedAggregatorList.forEach(agg -> builder.put(agg.getName(), agg.type())); + groupByList.forEach(group -> builder.put(group.getName(), group.type())); + return builder.build(); + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScript.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScript.java new file mode 100644 index 0000000000..64f7538e0d --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScript.java @@ -0,0 +1,69 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprNullValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.core.ExpressionScript; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; +import java.util.Map; +import lombok.EqualsAndHashCode; +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.script.AggregationScript; +import org.elasticsearch.search.lookup.SearchLookup; + +/** + * Aggregation expression script that executed on each document. + */ +@EqualsAndHashCode(callSuper = false) +public class ExpressionAggregationScript extends AggregationScript { + + /** + * Expression Script. + */ + private final ExpressionScript expressionScript; + + /** + * Constructor of ExpressionAggregationScript. + */ + public ExpressionAggregationScript( + Expression expression, + SearchLookup lookup, + LeafReaderContext context, + Map params) { + super(params, lookup, context); + this.expressionScript = new ExpressionScript(expression); + } + + @Override + public Object execute() { + return expressionScript.execute(this::getDoc, this::evaluateExpression); + } + + private ExprValue evaluateExpression(Expression expression, Environment valueEnv) { + ExprValue result = expression.valueOf(valueEnv); + + // The missing value is treated as null value in doc_value, so we can't distinguish with them. + if (result.isNull()) { + return ExprNullValue.of(); + } + return result; + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptFactory.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptFactory.java new file mode 100644 index 0000000000..ec6528716b --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptFactory.java @@ -0,0 +1,48 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import java.util.Map; +import lombok.EqualsAndHashCode; +import org.elasticsearch.script.AggregationScript; +import org.elasticsearch.search.lookup.SearchLookup; + +/** + * Aggregation Expression script factory that generates leaf factory. + */ +@EqualsAndHashCode +public class ExpressionAggregationScriptFactory implements AggregationScript.Factory { + + private final Expression expression; + + public ExpressionAggregationScriptFactory(Expression expression) { + this.expression = expression; + } + + @Override + public boolean isResultDeterministic() { + // This implies the results are cacheable + return true; + } + + @Override + public AggregationScript.LeafFactory newFactory(Map params, SearchLookup lookup) { + return new ExpressionAggregationScriptLeafFactory(expression, params, lookup); + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptLeafFactory.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptLeafFactory.java new file mode 100644 index 0000000000..63c54fb8a7 --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptLeafFactory.java @@ -0,0 +1,66 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import java.io.IOException; +import java.util.Map; +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.script.AggregationScript; +import org.elasticsearch.search.lookup.SearchLookup; + +/** + * Expression script leaf factory that produces script executor for each leaf. + */ +public class ExpressionAggregationScriptLeafFactory implements AggregationScript.LeafFactory { + + /** + * Expression to execute. + */ + private final Expression expression; + + /** + * Expression to execute. + */ + private final Map params; + + /** + * Expression to execute. + */ + private final SearchLookup lookup; + + /** + * Constructor of ExpressionAggregationScriptLeafFactory. + */ + public ExpressionAggregationScriptLeafFactory( + Expression expression, Map params, SearchLookup lookup) { + this.expression = expression; + this.params = params; + this.lookup = lookup; + } + + @Override + public AggregationScript newInstance(LeafReaderContext ctx) { + return new ExpressionAggregationScript(expression, lookup, ctx, params); + } + + @Override + public boolean needs_score() { + return false; + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/AggregationBuilderHelper.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/AggregationBuilderHelper.java new file mode 100644 index 0000000000..0d441e6e1b --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/AggregationBuilderHelper.java @@ -0,0 +1,63 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl; + +import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.ExpressionScriptEngine.EXPRESSION_LANG_NAME; +import static java.util.Collections.emptyMap; +import static org.elasticsearch.script.Script.DEFAULT_SCRIPT_TYPE; + +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.ScriptUtils; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.FunctionExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import java.util.function.Function; +import lombok.RequiredArgsConstructor; +import org.elasticsearch.script.Script; + +/** + * Abstract Aggregation Builder. + * + * @param type of the actual AggregationBuilder to be built. + */ +@RequiredArgsConstructor +public class AggregationBuilderHelper { + + private final ExpressionSerializer serializer; + + /** + * Build AggregationBuilder from Expression. + * + * @param expression Expression + * @return AggregationBuilder + */ + public T build(Expression expression, Function fieldBuilder, + Function scriptBuilder) { + if (expression instanceof ReferenceExpression) { + String fieldName = ((ReferenceExpression) expression).getAttr(); + return fieldBuilder.apply(ScriptUtils.convertTextToKeyword(fieldName, expression.type())); + } else if (expression instanceof FunctionExpression) { + return scriptBuilder.apply(new Script( + DEFAULT_SCRIPT_TYPE, EXPRESSION_LANG_NAME, serializer.serialize(expression), + emptyMap())); + } else { + throw new IllegalStateException(String.format("metric aggregation doesn't support " + + "expression %s", expression)); + } + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/BucketAggregationBuilder.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/BucketAggregationBuilder.java new file mode 100644 index 0000000000..38aabcdeea --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/BucketAggregationBuilder.java @@ -0,0 +1,54 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl; + +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import com.google.common.collect.ImmutableList; +import java.util.List; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.TermsValuesSourceBuilder; + +/** + * Bucket Aggregation Builder. + */ +public class BucketAggregationBuilder { + + private final AggregationBuilderHelper> helper; + + public BucketAggregationBuilder( + ExpressionSerializer serializer) { + this.helper = new AggregationBuilderHelper<>(serializer); + } + + /** + * Build the list of CompositeValuesSourceBuilder. + */ + public List> build(List expressions) { + ImmutableList.Builder> resultBuilder = + new ImmutableList.Builder<>(); + for (NamedExpression expression : expressions) { + TermsValuesSourceBuilder valuesSourceBuilder = + new TermsValuesSourceBuilder(expression.getName()).missingBucket(true); + resultBuilder + .add(helper.build(expression.getDelegated(), valuesSourceBuilder::field, + valuesSourceBuilder::script)); + } + return resultBuilder.build(); + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/MetricAggregationBuilder.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/MetricAggregationBuilder.java new file mode 100644 index 0000000000..1481161602 --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/MetricAggregationBuilder.java @@ -0,0 +1,80 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl; + +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; +import java.util.List; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; + +/** + * Build the Metric Aggregation from {@link NamedAggregator}. + */ +public class MetricAggregationBuilder + extends ExpressionNodeVisitor { + + private final AggregationBuilderHelper> helper; + + public MetricAggregationBuilder( + ExpressionSerializer serializer) { + this.helper = new AggregationBuilderHelper<>(serializer); + } + + /** + * Build AggregatorFactories.Builder from {@link NamedAggregator}. + * + * @param aggregatorList aggregator list + * @return AggregatorFactories.Builder + */ + public AggregatorFactories.Builder build(List aggregatorList) { + AggregatorFactories.Builder builder = new AggregatorFactories.Builder(); + for (NamedAggregator aggregator : aggregatorList) { + builder.addAggregator(aggregator.accept(this, null)); + } + return builder; + } + + @Override + public AggregationBuilder visitNamedAggregator(NamedAggregator node, + Object context) { + Expression expression = node.getArguments().get(0); + String name = node.getName(); + + switch (node.getFunctionName().getFunctionName()) { + case "avg": + return make(AggregationBuilders.avg(name), expression); + case "sum": + return make(AggregationBuilders.sum(name), expression); + case "count": + return make(AggregationBuilders.count(name), expression); + default: + throw new IllegalStateException( + String.format("unsupported aggregator %s", node.getFunctionName().getFunctionName())); + } + } + + private ValuesSourceAggregationBuilder make(ValuesSourceAggregationBuilder builder, + Expression expression) { + return helper.build(expression, builder::field, builder::script); + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/core/ExpressionScript.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/core/ExpressionScript.java new file mode 100644 index 0000000000..813f93e031 --- /dev/null +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/core/ExpressionScript.java @@ -0,0 +1,177 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.core; + +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.FLOAT; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType.ES_TEXT_KEYWORD; +import static java.util.stream.Collectors.reducing; +import static java.util.stream.Collectors.toMap; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprMissingValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.ScriptUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.time.chrono.ChronoZonedDateTime; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Supplier; +import lombok.EqualsAndHashCode; +import org.elasticsearch.index.fielddata.ScriptDocValues; + +/** + * Expression script executor that executes the expression on each document + * and determine if the document is supposed to be filtered out or not. + */ +@EqualsAndHashCode(callSuper = false) +public class ExpressionScript { + + /** + * Expression to execute. + */ + private final Expression expression; + + /** + * ElasticsearchExprValueFactory. + */ + @EqualsAndHashCode.Exclude + private final ElasticsearchExprValueFactory valueFactory; + + /** + * Reference Fields. + */ + @EqualsAndHashCode.Exclude + private final Set fields; + + /** + * Expression constructor. + */ + public ExpressionScript(Expression expression) { + this.expression = expression; + this.fields = AccessController.doPrivileged((PrivilegedAction>) () -> + extractFields(expression)); + this.valueFactory = + AccessController.doPrivileged( + (PrivilegedAction) () -> buildValueFactory(fields)); + } + + /** + * Evaluate on the doc generate by the doc provider. + * @param docProvider doc provider. + * @param evaluator evaluator + * @return + */ + public Object execute(Supplier>> docProvider, + BiFunction, ExprValue> evaluator) { + return AccessController.doPrivileged((PrivilegedAction) () -> { + Environment valueEnv = + buildValueEnv(fields, valueFactory, docProvider); + ExprValue result = evaluator.apply(expression, valueEnv); + return result.value(); + }); + } + + private Set extractFields(Expression expr) { + Set fields = new HashSet<>(); + expr.accept(new ExpressionNodeVisitor>() { + @Override + public Object visitReference(ReferenceExpression node, Set context) { + context.add(node); + return null; + } + }, fields); + return fields; + } + + private ElasticsearchExprValueFactory buildValueFactory(Set fields) { + Map typeEnv = fields.stream() + .collect(toMap( + ReferenceExpression::getAttr, + ReferenceExpression::type)); + return new ElasticsearchExprValueFactory(typeEnv); + } + + private Environment buildValueEnv( + Set fields, ElasticsearchExprValueFactory valueFactory, + Supplier>> docProvider) { + + Map valueEnv = new HashMap<>(); + for (ReferenceExpression field : fields) { + String fieldName = field.getAttr(); + ExprValue exprValue = valueFactory.construct(fieldName, getDocValue(field, docProvider)); + valueEnv.put(field, exprValue); + } + // Encapsulate map data structure into anonymous Environment class + return valueEnv::get; + } + + private Object getDocValue(ReferenceExpression field, + Supplier>> docProvider) { + String fieldName = getDocValueName(field); + ScriptDocValues docValue = docProvider.get().get(fieldName); + if (docValue == null || docValue.isEmpty()) { + return null; + } + + Object value = docValue.get(0); + if (value instanceof ChronoZonedDateTime) { + return ((ChronoZonedDateTime) value).toInstant(); + } + return castNumberToFieldType(value, field.type()); + } + + /** + * Text field doesn't have doc value (exception thrown even when you call "get") + * Limitation: assume inner field name is always "keyword". + */ + private String getDocValueName(ReferenceExpression field) { + String fieldName = field.getAttr(); + return ScriptUtils.convertTextToKeyword(fieldName, field.type()); + } + + /** + * DocValue only support long and double so cast to integer and float if needed. + * The doc value must be Long and Double for expr type Long/Integer and Double/Float respectively. + * Otherwise there must be bugs in our engine that causes the mismatch. + */ + private Object castNumberToFieldType(Object value, ExprType type) { + if (value == null) { + return value; + } + + if (type == INTEGER) { + return ((Long) value).intValue(); + } else if (type == FLOAT) { + return ((Double) value).floatValue(); + } else { + return value; + } + } +} diff --git a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/filter/ExpressionFilterScript.java b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/filter/ExpressionFilterScript.java index 08cb490080..e64cf161bd 100644 --- a/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/filter/ExpressionFilterScript.java +++ b/elasticsearch/src/main/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/filter/ExpressionFilterScript.java @@ -16,31 +16,15 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.filter; -import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.FLOAT; -import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; -import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType.ES_TEXT_KEYWORD; -import static java.util.stream.Collectors.toMap; - import com.amazon.opendistroforelasticsearch.sql.data.model.ExprBooleanValue; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType; -import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; -import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.core.ExpressionScript; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionNodeVisitor; -import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; -import java.security.AccessController; -import java.security.PrivilegedAction; -import java.time.chrono.ChronoZonedDateTime; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import lombok.EqualsAndHashCode; import org.apache.lucene.index.LeafReaderContext; -import org.elasticsearch.SpecialPermission; -import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.script.FilterScript; import org.elasticsearch.search.lookup.SearchLookup; @@ -52,118 +36,26 @@ class ExpressionFilterScript extends FilterScript { /** - * Expression to execute. - */ - private final Expression expression; - - /** - * ElasticsearchExprValueFactory. + * Expression Script. */ - @EqualsAndHashCode.Exclude - private final ElasticsearchExprValueFactory valueFactory; - - /** - * Reference Fields. - */ - @EqualsAndHashCode.Exclude - private final Set fields; + private final ExpressionScript expressionScript; public ExpressionFilterScript(Expression expression, SearchLookup lookup, LeafReaderContext context, Map params) { super(params, lookup, context); - this.expression = expression; - this.fields = AccessController.doPrivileged((PrivilegedAction>) () -> - extractFields(expression)); - this.valueFactory = - AccessController.doPrivileged( - (PrivilegedAction) () -> buildValueFactory(fields)); + this.expressionScript = new ExpressionScript(expression); } @Override public boolean execute() { - return AccessController.doPrivileged((PrivilegedAction) () -> { - Environment valueEnv = buildValueEnv(fields, valueFactory); - ExprValue result = evaluateExpression(valueEnv); - return (Boolean) result.value(); - }); - } - - private Set extractFields(Expression expr) { - Set fields = new HashSet<>(); - expr.accept(new ExpressionNodeVisitor>() { - @Override - public Object visitReference(ReferenceExpression node, Set context) { - context.add(node); - return null; - } - }, fields); - return fields; - } - - private ElasticsearchExprValueFactory buildValueFactory(Set fields) { - Map typeEnv = fields.stream() - .collect(toMap( - ReferenceExpression::getAttr, - ReferenceExpression::type)); - return new ElasticsearchExprValueFactory(typeEnv); - } - - private Environment buildValueEnv( - Set fields, ElasticsearchExprValueFactory valueFactory) { - - Map valueEnv = new HashMap<>(); - for (ReferenceExpression field : fields) { - String fieldName = field.getAttr(); - ExprValue exprValue = valueFactory.construct(fieldName, getDocValue(field)); - valueEnv.put(field, exprValue); - } - return valueEnv::get; // Encapsulate map data structure into anonymous Environment class + return (Boolean) expressionScript.execute(this::getDoc, this::evaluateExpression); } - private Object getDocValue(ReferenceExpression field) { - String fieldName = getDocValueName(field); - ScriptDocValues docValue = getDoc().get(fieldName); - if (docValue == null || docValue.isEmpty()) { - return null; - } - - Object value = docValue.get(0); - if (value instanceof ChronoZonedDateTime) { - return ((ChronoZonedDateTime) value).toInstant(); - } - return castNumberToFieldType(value, field.type()); - } - - /** - * Text field doesn't have doc value (exception thrown even when you call "get") - * Limitation: assume inner field name is always "keyword". - */ - private String getDocValueName(ReferenceExpression field) { - String fieldName = field.getAttr(); - if (field.type() == ES_TEXT_KEYWORD) { - fieldName += ".keyword"; - } - return fieldName; - } - - /** - * DocValue only support long and double so cast to integer and float if needed. - * The doc value must be Long and Double for expr type Long/Integer and Double/Float respectively. - * Otherwise there must be bugs in our engine that causes the mismatch. - */ - private Object castNumberToFieldType(Object value, ExprType type) { - if (type == INTEGER) { - return ((Long) value).intValue(); - } else if (type == FLOAT) { - return ((Double) value).floatValue(); - } else { - return value; - } - } - private ExprValue evaluateExpression(Environment valueEnv) { + private ExprValue evaluateExpression(Expression expression, + Environment valueEnv) { ExprValue result = expression.valueOf(valueEnv); if (result.isNull() || result.isMissing()) { return ExprBooleanValue.of(false); diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchNodeClientTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchNodeClientTest.java index e53fa1a08e..265bfa7c42 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchNodeClientTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchNodeClientTest.java @@ -28,10 +28,15 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprIntegerValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.mapping.IndexMapping; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.request.ElasticsearchScrollRequest; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.response.ElasticsearchResponse; import com.google.common.base.Charsets; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSortedMap; import com.google.common.io.Resources; import java.io.IOException; @@ -73,6 +78,15 @@ class ElasticsearchNodeClientTest { @Mock(answer = RETURNS_DEEP_STUBS) private NodeClient nodeClient; + @Mock + private ElasticsearchExprValueFactory factory; + + @Mock + private SearchHit searchHit; + + private ExprTupleValue exprTupleValue = ExprTupleValue.fromExprValueMap(ImmutableMap.of("id", + new ExprIntegerValue(1))); + @Test public void getIndexMappings() throws IOException { URL url = Resources.getResource(TEST_MAPPING_FILE); @@ -153,9 +167,11 @@ public void search() { when(searchResponse.getHits()) .thenReturn( new SearchHits( - new SearchHit[] {new SearchHit(1)}, + new SearchHit[] {searchHit}, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0F)); + when(searchHit.getSourceAsString()).thenReturn("{\"id\", 1}"); + when(factory.construct(any())).thenReturn(exprTupleValue); // Mock second scroll request followed SearchResponse scrollResponse = mock(SearchResponse.class); @@ -164,13 +180,13 @@ public void search() { when(scrollResponse.getHits()).thenReturn(SearchHits.empty()); // Verify response for first scroll request - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); ElasticsearchResponse response1 = client.search(request); assertFalse(response1.isEmpty()); - Iterator hits = response1.iterator(); + Iterator hits = response1.iterator(); assertTrue(hits.hasNext()); - assertEquals(new SearchHit(1), hits.next()); + assertEquals(exprTupleValue, hits.next()); assertFalse(hits.hasNext()); // Verify response for second scroll request @@ -208,7 +224,7 @@ void cleanup() { ElasticsearchNodeClient client = new ElasticsearchNodeClient(mock(ClusterService.class), nodeClient); - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); request.setScrollId("scroll123"); client.cleanup(request); assertFalse(request.isScrollStarted()); @@ -224,7 +240,7 @@ void cleanupWithoutScrollId() { ElasticsearchNodeClient client = new ElasticsearchNodeClient(mock(ClusterService.class), nodeClient); - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); client.cleanup(request); verify(nodeClient, never()).prepareClearScroll(); } diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchRestClientTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchRestClientTest.java index a91a2e861c..66b30f74a7 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchRestClientTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/client/ElasticsearchRestClientTest.java @@ -27,6 +27,10 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprIntegerValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.mapping.IndexMapping; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.request.ElasticsearchScrollRequest; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.response.ElasticsearchResponse; @@ -67,6 +71,15 @@ class ElasticsearchRestClientTest { private ElasticsearchRestClient client; + @Mock + private ElasticsearchExprValueFactory factory; + + @Mock + private SearchHit searchHit; + + private ExprTupleValue exprTupleValue = ExprTupleValue.fromExprValueMap(ImmutableMap.of("id", + new ExprIntegerValue(1))); + @BeforeEach void setUp() { client = new ElasticsearchRestClient(restClient); @@ -126,9 +139,11 @@ void search() throws IOException { when(searchResponse.getHits()) .thenReturn( new SearchHits( - new SearchHit[] {new SearchHit(1)}, + new SearchHit[] {searchHit}, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0F)); + when(searchHit.getSourceAsString()).thenReturn("{\"id\", 1}"); + when(factory.construct(any())).thenReturn(exprTupleValue); // Mock second scroll request followed SearchResponse scrollResponse = mock(SearchResponse.class); @@ -137,13 +152,13 @@ void search() throws IOException { when(scrollResponse.getHits()).thenReturn(SearchHits.empty()); // Verify response for first scroll request - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); ElasticsearchResponse response1 = client.search(request); assertFalse(response1.isEmpty()); - Iterator hits = response1.iterator(); + Iterator hits = response1.iterator(); assertTrue(hits.hasNext()); - assertEquals(new SearchHit(1), hits.next()); + assertEquals(exprTupleValue, hits.next()); assertFalse(hits.hasNext()); // Verify response for second scroll request @@ -155,7 +170,8 @@ void search() throws IOException { void searchWithIOException() throws IOException { when(restClient.search(any(), any())).thenThrow(new IOException()); assertThrows( - IllegalStateException.class, () -> client.search(new ElasticsearchScrollRequest("test"))); + IllegalStateException.class, + () -> client.search(new ElasticsearchScrollRequest("test", factory))); } @Test @@ -175,7 +191,7 @@ void scrollWithIOException() throws IOException { when(restClient.scroll(any(), any())).thenThrow(new IOException()); // First request run successfully - ElasticsearchScrollRequest scrollRequest = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest scrollRequest = new ElasticsearchScrollRequest("test", factory); client.search(scrollRequest); assertThrows( IllegalStateException.class, () -> client.search(scrollRequest)); @@ -193,7 +209,7 @@ void schedule() { @Test void cleanup() throws IOException { - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); request.setScrollId("scroll123"); client.cleanup(request); verify(restClient).clearScroll(any(), any()); @@ -202,7 +218,7 @@ void cleanup() throws IOException { @Test void cleanupWithoutScrollId() throws IOException { - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); client.cleanup(request); verify(restClient, never()).clearScroll(any(), any()); } @@ -211,7 +227,7 @@ void cleanupWithoutScrollId() throws IOException { void cleanupWithIOException() throws IOException { when(restClient.clearScroll(any(), any())).thenThrow(new IOException()); - ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test", factory); request.setScrollId("scroll123"); assertThrows(IllegalStateException.class, () -> client.cleanup(request)); } diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactoryTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactoryTest.java index c668a8d897..ac9b302442 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactoryTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/data/value/ElasticsearchExprValueFactoryTest.java @@ -91,6 +91,11 @@ public void constructInteger() { assertEquals(integerValue(1), constructFromObject("intV", 1)); } + @Test + public void constructIntegerValueInStringValue() { + assertEquals(integerValue(1), constructFromObject("intV", "1")); + } + @Test public void constructLong() { assertEquals(longValue(1L), tupleValue("{\"longV\":1}").get("longV")); diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngineTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngineTest.java index c91ecd062d..828c8ea14a 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngineTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionEngineTest.java @@ -16,10 +16,12 @@ package com.amazon.opendistroforelasticsearch.sql.elasticsearch.executor; +import static com.amazon.opendistroforelasticsearch.sql.common.setting.Settings.Key.QUERY_SIZE_LIMIT; import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.tupleValue; import static com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.QueryResponse; import static com.google.common.collect.ImmutableMap.of; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; @@ -29,10 +31,14 @@ import static org.mockito.Mockito.when; import com.amazon.opendistroforelasticsearch.sql.common.response.ResponseListener; +import com.amazon.opendistroforelasticsearch.sql.common.setting.Settings; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.client.ElasticsearchClient; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.executor.protector.ElasticsearchExecutionProtector; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.ElasticsearchIndexScan; import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.storage.TableScanOperator; import java.util.ArrayList; @@ -124,6 +130,52 @@ public void onFailure(Exception e) { verify(plan).close(); } + @Test + void explainSuccessfully() { + ElasticsearchExecutionEngine executor = new ElasticsearchExecutionEngine(client, protector); + Settings settings = mock(Settings.class); + when(settings.getSettingValue(QUERY_SIZE_LIMIT)).thenReturn(100); + PhysicalPlan plan = new ElasticsearchIndexScan(mock(ElasticsearchClient.class), + settings, "test", mock(ElasticsearchExprValueFactory.class)); + + AtomicReference result = new AtomicReference<>(); + executor.explain(plan, new ResponseListener() { + @Override + public void onResponse(ExplainResponse response) { + result.set(response); + } + + @Override + public void onFailure(Exception e) { + fail(e); + } + }); + + assertNotNull(result.get()); + } + + @Test + void explainWithFailure() { + ElasticsearchExecutionEngine executor = new ElasticsearchExecutionEngine(client, protector); + PhysicalPlan plan = mock(PhysicalPlan.class); + when(plan.accept(any(), any())).thenThrow(IllegalStateException.class); + + AtomicReference result = new AtomicReference<>(); + executor.explain(plan, new ResponseListener() { + @Override + public void onResponse(ExplainResponse response) { + fail("Should fail as expected"); + } + + @Override + public void onFailure(Exception e) { + result.set(e); + } + }); + + assertNotNull(result.get()); + } + @RequiredArgsConstructor private static class FakePhysicalPlan extends TableScanOperator { private final Iterator it; diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionProtectorTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionProtectorTest.java index 71389a4aed..de11d4a812 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionProtectorTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/executor/ElasticsearchExecutionProtectorTest.java @@ -40,8 +40,8 @@ import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; -import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AvgAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.monitor.ResourceMonitor; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL; @@ -88,9 +88,15 @@ public void testProtectIndexScan() { ReferenceExpression exclude = ref("name", STRING); ReferenceExpression dedupeField = ref("name", STRING); ReferenceExpression topField = ref("name", STRING); + List topExprs = Arrays.asList(ref("age", INTEGER)); Expression filterExpr = literal(ExprBooleanValue.of(true)); - List groupByExprs = Arrays.asList(ref("age", INTEGER)); - List aggregators = Arrays.asList(new AvgAggregator(groupByExprs, DOUBLE)); + Expression whileExpr = literal(ExprBooleanValue.of(true)); + Boolean keepLast = false; + Integer headNumber = 5; + List groupByExprs = Arrays.asList(named("age", ref("age", INTEGER))); + List aggregators = + Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), + DOUBLE))); Map mappings = ImmutableMap.of(ref("name", STRING), ref("lastname", STRING)); Pair newEvalField = @@ -108,11 +114,15 @@ public void testProtectIndexScan() { PhysicalPlanDSL.remove( PhysicalPlanDSL.rename( PhysicalPlanDSL.agg( - filter( - resourceMonitor( + PhysicalPlanDSL.head( + filter( + resourceMonitor( new ElasticsearchIndexScan( client, settings, indexName, exprValueFactory)), - filterExpr), + filterExpr), + keepLast, + whileExpr, + headNumber), aggregators, groupByExprs), mappings), @@ -121,7 +131,7 @@ public void testProtectIndexScan() { sortCount, sortField), CommandType.TOP, - groupByExprs, + topExprs, topField), dedupeField), include), @@ -134,10 +144,15 @@ public void testProtectIndexScan() { PhysicalPlanDSL.remove( PhysicalPlanDSL.rename( PhysicalPlanDSL.agg( - filter( - new ElasticsearchIndexScan( - client, settings, indexName, exprValueFactory), - filterExpr), + PhysicalPlanDSL.head( + filter( + new ElasticsearchIndexScan( + client, settings, indexName, + exprValueFactory), + filterExpr), + keepLast, + whileExpr, + headNumber), aggregators, groupByExprs), mappings), @@ -146,7 +161,7 @@ public void testProtectIndexScan() { sortCount, sortField), CommandType.TOP, - groupByExprs, + topExprs, topField), dedupeField), include))); diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequestTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequestTest.java index de6ab040f3..47b13f2a10 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequestTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchQueryRequestTest.java @@ -21,13 +21,12 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.response.ElasticsearchResponse; import java.util.function.Consumer; import java.util.function.Function; @@ -64,13 +63,17 @@ public class ElasticsearchQueryRequestTest { @Mock private SearchHit searchHit; - private final ElasticsearchQueryRequest request = new ElasticsearchQueryRequest("test", 200); + @Mock + private ElasticsearchExprValueFactory factory; + + private final ElasticsearchQueryRequest request = + new ElasticsearchQueryRequest("test", 200, factory); @Test void search() { when(searchAction.apply(any())).thenReturn(searchResponse); when(searchResponse.getHits()).thenReturn(searchHits); - when(searchHits.getHits()).thenReturn(new SearchHit[]{searchHit}); + when(searchHits.getHits()).thenReturn(new SearchHit[] {searchHit}); ElasticsearchResponse searchResponse = request.search(searchAction, scrollAction); assertFalse(searchResponse.isEmpty()); diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequestTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequestTest.java index b836c48f7b..cf78655a87 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequestTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/request/ElasticsearchScrollRequestTest.java @@ -20,15 +20,24 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +@ExtendWith(MockitoExtension.class) class ElasticsearchScrollRequestTest { - private final ElasticsearchScrollRequest request = new ElasticsearchScrollRequest("test"); + @Mock + private ElasticsearchExprValueFactory factory; + + private final ElasticsearchScrollRequest request = + new ElasticsearchScrollRequest("test", factory); @Test void searchRequest() { diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/AggregationResponseUtils.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/AggregationResponseUtils.java new file mode 100644 index 0000000000..f75b618d23 --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/AggregationResponseUtils.java @@ -0,0 +1,102 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.response; + +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ContextParser; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContent; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.ParsedComposite; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.ParsedDateHistogram; +import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms; +import org.elasticsearch.search.aggregations.bucket.terms.LongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.ParsedDoubleTerms; +import org.elasticsearch.search.aggregations.bucket.terms.ParsedLongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.ParsedStringTerms; +import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.ParsedAvg; +import org.elasticsearch.search.aggregations.metrics.ParsedMax; +import org.elasticsearch.search.aggregations.metrics.ParsedMin; +import org.elasticsearch.search.aggregations.metrics.ParsedSum; +import org.elasticsearch.search.aggregations.metrics.ParsedValueCount; +import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.pipeline.ParsedPercentilesBucket; +import org.elasticsearch.search.aggregations.pipeline.PercentilesBucketPipelineAggregationBuilder; + +public class AggregationResponseUtils { + private static final List entryList = + new ImmutableMap.Builder>().put( + MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c)) + .put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c)) + .put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c)) + .put(AvgAggregationBuilder.NAME, (p, c) -> ParsedAvg.fromXContent(p, (String) c)) + .put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c)) + .put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c)) + .put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c)) + .put(ValueCountAggregationBuilder.NAME, + (p, c) -> ParsedValueCount.fromXContent(p, (String) c)) + .put(PercentilesBucketPipelineAggregationBuilder.NAME, + (p, c) -> ParsedPercentilesBucket.fromXContent(p, (String) c)) + .put(DateHistogramAggregationBuilder.NAME, + (p, c) -> ParsedDateHistogram.fromXContent(p, (String) c)) + .put(CompositeAggregationBuilder.NAME, + (p, c) -> ParsedComposite.fromXContent(p, (String) c)) + .build() + .entrySet() + .stream() + .map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, + new ParseField(entry.getKey()), + entry.getValue())) + .collect(Collectors.toList()); + private static final NamedXContentRegistry namedXContentRegistry = + new NamedXContentRegistry(entryList); + private static final XContent xContent = XContentFactory.xContent(XContentType.JSON); + + /** + * Populate {@link Aggregations} from JSON string. + * + * @param json json string + * @return {@link Aggregations} + */ + public static Aggregations fromJson(String json) { + try { + XContentParser contentParser = + xContent.createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, json); + contentParser.nextToken(); + return Aggregations.fromXContent(contentParser); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchAggregationResponseParserTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchAggregationResponseParserTest.java new file mode 100644 index 0000000000..f873f0cb06 --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchAggregationResponseParserTest.java @@ -0,0 +1,160 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.response; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import com.google.common.collect.ImmutableMap; +import java.util.List; +import java.util.Map; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +class ElasticsearchAggregationResponseParserTest { + + /** + * SELECT MAX(age) as max FROM accounts. + */ + @Test + void no_bucket_one_metric_should_pass() { + String response = "{\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + "}"; + assertThat(parse(response), contains(entry("max", 40d))); + } + + /** + * SELECT MAX(age) as max, MIN(age) as min FROM accounts. + */ + @Test + void no_bucket_two_metric_should_pass() { + String response = "{\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " },\n" + + " \"min#min\": {\n" + + " \"value\": 20\n" + + " }\n" + + "}"; + assertThat(parse(response), + containsInAnyOrder(entry("max", 40d), entry("min", 20d))); + } + + @Test + void one_bucket_one_metric_should_pass() { + String response = "{\n" + + " \"composite#composite_buckets\": {\n" + + " \"after_key\": {\n" + + " \"type\": \"sale\"\n" + + " },\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": {\n" + + " \"type\": \"cost\"\n" + + " },\n" + + " \"doc_count\": 2,\n" + + " \"avg#avg\": {\n" + + " \"value\": 20\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": {\n" + + " \"type\": \"sale\"\n" + + " },\n" + + " \"doc_count\": 2,\n" + + " \"avg#avg\": {\n" + + " \"value\": 105\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + assertThat(parse(response), + containsInAnyOrder(ImmutableMap.of("type", "cost", "avg", 20d), + ImmutableMap.of("type", "sale", "avg", 105d))); + } + + @Test + void two_bucket_one_metric_should_pass() { + String response = "{\n" + + " \"composite#composite_buckets\": {\n" + + " \"after_key\": {\n" + + " \"type\": \"sale\",\n" + + " \"region\": \"us\"\n" + + " },\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": {\n" + + " \"type\": \"cost\",\n" + + " \"region\": \"us\"\n" + + " },\n" + + " \"avg#avg\": {\n" + + " \"value\": 20\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": {\n" + + " \"type\": \"sale\",\n" + + " \"region\": \"uk\"\n" + + " },\n" + + " \"avg#avg\": {\n" + + " \"value\": 130\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + assertThat(parse(response), + containsInAnyOrder(ImmutableMap.of("type", "cost", "region", "us", "avg", 20d), + ImmutableMap.of("type", "sale", "region", "uk", "avg", 130d))); + } + + @Test + void unsupported_aggregation_should_fail() { + String response = "{\n" + + " \"date_histogram#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + "}"; + IllegalStateException exception = + assertThrows(IllegalStateException.class, () -> parse(response)); + assertEquals("unsupported aggregation type date_histogram", exception.getMessage()); + } + + @Test + void nan_value_should_return_null() { + assertNull(ElasticsearchAggregationResponseParser.handleNanValue(Double.NaN)); + } + + public List> parse(String json) { + return ElasticsearchAggregationResponseParser.parse(AggregationResponseUtils.fromJson(json)); + } + + public Map entry(String name, Object value) { + return ImmutableMap.of(name, value); + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponseTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponseTest.java index 68ea1af73d..2746a64885 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponseTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/response/ElasticsearchResponseTest.java @@ -20,60 +20,138 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.when; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprIntegerValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.value.ElasticsearchExprValueFactory; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; -import org.junit.jupiter.api.BeforeEach; +import org.elasticsearch.search.aggregations.Aggregations; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; @ExtendWith(MockitoExtension.class) class ElasticsearchResponseTest { - @Mock private SearchResponse esResponse; + @Mock + private SearchResponse esResponse; - @BeforeEach - void setUp() { + @Mock + private ElasticsearchExprValueFactory factory; + + @Mock + private SearchHit searchHit1; + + @Mock + private SearchHit searchHit2; + + @Mock + private Aggregations aggregations; + + private ExprTupleValue exprTupleValue1 = ExprTupleValue.fromExprValueMap(ImmutableMap.of("id1", + new ExprIntegerValue(1))); + + private ExprTupleValue exprTupleValue2 = ExprTupleValue.fromExprValueMap(ImmutableMap.of("id2", + new ExprIntegerValue(2))); + + @Test + void isEmpty() { when(esResponse.getHits()) .thenReturn( new SearchHits( - new SearchHit[] {new SearchHit(1), new SearchHit(2)}, + new SearchHit[] {searchHit1, searchHit2}, new TotalHits(2L, TotalHits.Relation.EQUAL_TO), 1.0F)); - } - @Test - void isEmpty() { - ElasticsearchResponse response1 = new ElasticsearchResponse(esResponse); + ElasticsearchResponse response1 = new ElasticsearchResponse(esResponse, factory); assertFalse(response1.isEmpty()); when(esResponse.getHits()).thenReturn(SearchHits.empty()); - ElasticsearchResponse response2 = new ElasticsearchResponse(esResponse); + ElasticsearchResponse response2 = new ElasticsearchResponse(esResponse, factory); assertTrue(response2.isEmpty()); when(esResponse.getHits()) .thenReturn(new SearchHits(null, new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0)); - ElasticsearchResponse response3 = new ElasticsearchResponse(esResponse); + ElasticsearchResponse response3 = new ElasticsearchResponse(esResponse, factory); assertTrue(response3.isEmpty()); } @Test void iterator() { + when(esResponse.getHits()) + .thenReturn( + new SearchHits( + new SearchHit[] {searchHit1, searchHit2}, + new TotalHits(2L, TotalHits.Relation.EQUAL_TO), + 1.0F)); + + when(searchHit1.getSourceAsString()).thenReturn("{\"id1\", 1}"); + when(searchHit2.getSourceAsString()).thenReturn("{\"id1\", 2}"); + when(factory.construct(any())).thenReturn(exprTupleValue1).thenReturn(exprTupleValue2); + int i = 0; - for (SearchHit hit : new ElasticsearchResponse(esResponse)) { + for (ExprValue hit : new ElasticsearchResponse(esResponse, factory)) { if (i == 0) { - assertEquals(new SearchHit(1), hit); + assertEquals(exprTupleValue1, hit); } else if (i == 1) { - assertEquals(new SearchHit(2), hit); + assertEquals(exprTupleValue2, hit); } else { fail("More search hits returned than expected"); } i++; } } + + @Test + void response_is_aggregation_when_aggregation_not_empty() { + when(esResponse.getAggregations()).thenReturn(aggregations); + + ElasticsearchResponse response = new ElasticsearchResponse(esResponse, factory); + assertTrue(response.isAggregationResponse()); + } + + @Test + void response_isnot_aggregation_when_aggregation_is_empty() { + when(esResponse.getAggregations()).thenReturn(null); + + ElasticsearchResponse response = new ElasticsearchResponse(esResponse, factory); + assertFalse(response.isAggregationResponse()); + } + + @Test + void aggregation_iterator() { + try ( + MockedStatic mockedStatic = Mockito + .mockStatic(ElasticsearchAggregationResponseParser.class)) { + when(ElasticsearchAggregationResponseParser.parse(any())) + .thenReturn(Arrays.asList(ImmutableMap.of("id1", 1), ImmutableMap.of("id2", 2))); + when(esResponse.getAggregations()).thenReturn(aggregations); + when(factory.construct(anyString(), any())).thenReturn(new ExprIntegerValue(1)) + .thenReturn(new ExprIntegerValue(2)); + + int i = 0; + for (ExprValue hit : new ElasticsearchResponse(esResponse, factory)) { + if (i == 0) { + assertEquals(exprTupleValue1, hit); + } else if (i == 1) { + assertEquals(exprTupleValue2, hit); + } else { + fail("More search hits returned than expected"); + } + i++; + } + } + } } diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScanTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScanTest.java index 5a41dfd2fe..aa9790068a 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScanTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexScanTest.java @@ -81,21 +81,21 @@ void queryEmptyResult() { @Test void queryAllResults() { mockResponse( - new SearchHit[]{employee(1, "John", "IT"), employee(2, "Smith", "HR")}, - new SearchHit[]{employee(3, "Allen", "IT")}); + new ExprValue[]{employee(1, "John", "IT"), employee(2, "Smith", "HR")}, + new ExprValue[]{employee(3, "Allen", "IT")}); try (ElasticsearchIndexScan indexScan = new ElasticsearchIndexScan(client, settings, "employees", exprValueFactory)) { indexScan.open(); assertTrue(indexScan.hasNext()); - assertEquals(tupleValue(employee(1, "John", "IT")), indexScan.next()); + assertEquals(employee(1, "John", "IT"), indexScan.next()); assertTrue(indexScan.hasNext()); - assertEquals(tupleValue(employee(2, "Smith", "HR")), indexScan.next()); + assertEquals(employee(2, "Smith", "HR"), indexScan.next()); assertTrue(indexScan.hasNext()); - assertEquals(tupleValue(employee(3, "Allen", "IT")), indexScan.next()); + assertEquals(employee(3, "Allen", "IT"), indexScan.next()); assertFalse(indexScan.hasNext()); } @@ -128,6 +128,7 @@ private static class PushDownAssertion { private final ElasticsearchClient client; private final ElasticsearchIndexScan indexScan; private final ElasticsearchResponse response; + private final ElasticsearchExprValueFactory factory; public PushDownAssertion(ElasticsearchClient client, ElasticsearchExprValueFactory valueFactory, @@ -135,6 +136,7 @@ public PushDownAssertion(ElasticsearchClient client, this.client = client; this.indexScan = new ElasticsearchIndexScan(client, settings, "test", valueFactory); this.response = mock(ElasticsearchResponse.class); + this.factory = valueFactory; when(response.isEmpty()).thenReturn(true); } @@ -144,7 +146,7 @@ PushDownAssertion pushDown(QueryBuilder query) { } PushDownAssertion shouldQuery(QueryBuilder expected) { - ElasticsearchRequest request = new ElasticsearchQueryRequest("test", 200); + ElasticsearchRequest request = new ElasticsearchQueryRequest("test", 200, factory); request.getSourceBuilder() .query(expected) .sort(DOC_FIELD_NAME, ASC); @@ -155,7 +157,7 @@ PushDownAssertion shouldQuery(QueryBuilder expected) { } - private void mockResponse(SearchHit[]... searchHitBatches) { + private void mockResponse(ExprValue[]... searchHitBatches) { when(client.search(any())) .thenAnswer( new Answer() { @@ -167,7 +169,7 @@ public ElasticsearchResponse answer(InvocationOnMock invocation) { int totalBatch = searchHitBatches.length; if (batchNum < totalBatch) { when(response.isEmpty()).thenReturn(false); - SearchHit[] searchHit = searchHitBatches[batchNum]; + ExprValue[] searchHit = searchHitBatches[batchNum]; when(response.iterator()).thenReturn(Arrays.asList(searchHit).iterator()); } else if (batchNum == totalBatch) { when(response.isEmpty()).thenReturn(true); @@ -181,11 +183,11 @@ public ElasticsearchResponse answer(InvocationOnMock invocation) { }); } - protected SearchHit employee(int docId, String name, String department) { + protected ExprValue employee(int docId, String name, String department) { SearchHit hit = new SearchHit(docId); hit.sourceRef( new BytesArray("{\"name\":\"" + name + "\",\"department\":\"" + department + "\"}")); - return hit; + return tupleValue(hit); } private ExprValue tupleValue(SearchHit hit) { diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexTest.java index 709ba51086..f204d6fba9 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/ElasticsearchIndexTest.java @@ -51,11 +51,12 @@ import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; -import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AvgAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.AggregationOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.FilterOperator; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanDSL; @@ -145,8 +146,10 @@ void implementOtherLogicalOperators() { ReferenceExpression exclude = ref("name", STRING); ReferenceExpression dedupeField = ref("name", STRING); Expression filterExpr = literal(ExprBooleanValue.of(true)); - List groupByExprs = Arrays.asList(ref("age", INTEGER)); - List aggregators = Arrays.asList(new AvgAggregator(groupByExprs, DOUBLE)); + List groupByExprs = Arrays.asList(named("age", ref("age", INTEGER))); + List aggregators = + Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), + DOUBLE))); Map mappings = ImmutableMap.of(ref("name", STRING), ref("lastname", STRING)); Pair newEvalField = @@ -162,10 +165,7 @@ void implementOtherLogicalOperators() { eval( remove( rename( - aggregation( - relation(indexName), - aggregators, - groupByExprs), + relation(indexName), mappings), exclude), newEvalField), @@ -182,11 +182,8 @@ void implementOtherLogicalOperators() { PhysicalPlanDSL.eval( PhysicalPlanDSL.remove( PhysicalPlanDSL.rename( - PhysicalPlanDSL.agg( - new ElasticsearchIndexScan( - client, settings, indexName, exprValueFactory), - aggregators, - groupByExprs), + new ElasticsearchIndexScan(client, settings, indexName, + exprValueFactory), mappings), exclude), newEvalField), @@ -225,21 +222,80 @@ void shouldNotPushDownFilterFarFromRelation() { ReferenceExpression field = ref("name", STRING); Expression filterExpr = dsl.equal(field, literal("John")); - List groupByExprs = Arrays.asList(ref("age", INTEGER)); - List aggregators = Arrays.asList(new AvgAggregator(groupByExprs, DOUBLE)); + List groupByExprs = Arrays.asList(named("age", ref("age", INTEGER))); + List aggregators = + Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), + DOUBLE))); String indexName = "test"; ElasticsearchIndex index = new ElasticsearchIndex(client, settings, indexName); PhysicalPlan plan = index.implement( - filter( - aggregation( - relation(indexName), - aggregators, - groupByExprs - ), - filterExpr)); + filter( + aggregation( + relation(indexName), + aggregators, + groupByExprs + ), + filterExpr)); assertTrue(plan instanceof FilterOperator); } + @Test + void shouldPushDownAggregation() { + when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); + + ReferenceExpression field = ref("name", STRING); + Expression filterExpr = dsl.equal(field, literal("John")); + List groupByExprs = Arrays.asList(named("age", ref("age", INTEGER))); + List aggregators = + Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), + DOUBLE))); + + String indexName = "test"; + ElasticsearchIndex index = new ElasticsearchIndex(client, settings, indexName); + PhysicalPlan plan = index.implement( + filter( + aggregation( + relation(indexName), + aggregators, + groupByExprs + ), + filterExpr)); + + assertTrue(plan.getChild().get(0) instanceof ElasticsearchIndexScan); + + plan = index.implement( + aggregation( + filter( + relation(indexName), + filterExpr), + aggregators, + groupByExprs)); + assertTrue(plan instanceof ElasticsearchIndexScan); + } + + @Test + void shouldNotPushDownAggregationFarFromRelation() { + when(settings.getSettingValue(Settings.Key.QUERY_SIZE_LIMIT)).thenReturn(200); + + ReferenceExpression field = ref("name", STRING); + Expression filterExpr = dsl.equal(field, literal("John")); + List groupByExprs = Arrays.asList(named("age", ref("age", INTEGER))); + List aggregators = + Arrays.asList(named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), + DOUBLE))); + + String indexName = "test"; + ElasticsearchIndex index = new ElasticsearchIndex(client, settings, indexName); + + PhysicalPlan plan = index.implement( + aggregation( + filter(filter( + relation(indexName), + filterExpr), filterExpr), + aggregators, + groupByExprs)); + assertTrue(plan instanceof AggregationOperator); + } } diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngineTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngineTest.java index 9be9087527..eda964f6dd 100644 --- a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngineTest.java +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/ExpressionScriptEngineTest.java @@ -28,6 +28,7 @@ import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; import com.amazon.opendistroforelasticsearch.sql.expression.DSL; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import org.elasticsearch.script.AggregationScript; import org.elasticsearch.script.FilterScript; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptEngine; @@ -65,7 +66,7 @@ void can_initialize_filter_script_factory_by_compiled_script() { when(serializer.deserialize("test code")).thenReturn(expression); assertThat(scriptEngine.getSupportedContexts(), - contains(FilterScript.CONTEXT)); + contains(FilterScript.CONTEXT, AggregationScript.CONTEXT)); Object actualFactory = scriptEngine.compile( "test", "test code", FilterScript.CONTEXT, emptyMap()); diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/AggregationQueryBuilderTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/AggregationQueryBuilderTest.java new file mode 100644 index 0000000000..537fe2f200 --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/AggregationQueryBuilderTest.java @@ -0,0 +1,263 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.DOUBLE; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.STRING; +import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType.ES_TEXT_KEYWORD; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.named; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.ref; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; + +import com.amazon.opendistroforelasticsearch.sql.data.type.ExprType; +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AvgAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import lombok.SneakyThrows; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +@ExtendWith(MockitoExtension.class) +class AggregationQueryBuilderTest { + + private final DSL dsl = new ExpressionConfig().dsl(new ExpressionConfig().functionRepository()); + + @Mock + private ExpressionSerializer serializer; + + private AggregationQueryBuilder queryBuilder; + + @BeforeEach + void set_up() { + queryBuilder = new AggregationQueryBuilder(serializer); + } + + @Test + void should_build_composite_aggregation_for_field_reference() { + assertEquals( + "{\n" + + " \"composite_buckets\" : {\n" + + " \"composite\" : {\n" + + " \"size\" : 1000,\n" + + " \"sources\" : [ {\n" + + " \"name\" : {\n" + + " \"terms\" : {\n" + + " \"field\" : \"name\",\n" + + " \"missing_bucket\" : true,\n" + + " \"order\" : \"asc\"\n" + + " }\n" + + " }\n" + + " } ]\n" + + " },\n" + + " \"aggregations\" : {\n" + + " \"avg(age)\" : {\n" + + " \"avg\" : {\n" + + " \"field\" : \"age\"\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))), + Arrays.asList(named("name", ref("name", STRING))))); + } + + @Test + void should_build_type_mapping_for_field_reference() { + assertThat( + buildTypeMapping(Arrays.asList( + named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))), + Arrays.asList(named("name", ref("name", STRING)))), + containsInAnyOrder( + map("avg(age)", INTEGER), + map("name", STRING) + )); + } + + @Test + void should_build_composite_aggregation_for_field_reference_of_keyword() { + assertEquals( + "{\n" + + " \"composite_buckets\" : {\n" + + " \"composite\" : {\n" + + " \"size\" : 1000,\n" + + " \"sources\" : [ {\n" + + " \"name\" : {\n" + + " \"terms\" : {\n" + + " \"field\" : \"name.keyword\",\n" + + " \"missing_bucket\" : true,\n" + + " \"order\" : \"asc\"\n" + + " }\n" + + " }\n" + + " } ]\n" + + " },\n" + + " \"aggregations\" : {\n" + + " \"avg(age)\" : {\n" + + " \"avg\" : {\n" + + " \"field\" : \"age\"\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))), + Arrays.asList(named("name", ref("name", ES_TEXT_KEYWORD))))); + } + + @Test + void should_build_type_mapping_for_field_reference_of_keyword() { + assertThat( + buildTypeMapping(Arrays.asList( + named("avg(age)", new AvgAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))), + Arrays.asList(named("name", ref("name", ES_TEXT_KEYWORD)))), + containsInAnyOrder( + map("avg(age)", INTEGER), + map("name", ES_TEXT_KEYWORD) + )); + } + + @Test + void should_build_composite_aggregation_for_expression() { + doAnswer(invocation -> { + Expression expr = invocation.getArgument(0); + return expr.toString(); + }).when(serializer).serialize(any()); + assertEquals( + "{\n" + + " \"composite_buckets\" : {\n" + + " \"composite\" : {\n" + + " \"size\" : 1000,\n" + + " \"sources\" : [ {\n" + + " \"age\" : {\n" + + " \"terms\" : {\n" + + " \"script\" : {\n" + + " \"source\" : \"asin(age)\",\n" + + " \"lang\" : \"opendistro_expression\"\n" + + " },\n" + + " \"missing_bucket\" : true,\n" + + " \"order\" : \"asc\"\n" + + " }\n" + + " }\n" + + " } ]\n" + + " },\n" + + " \"aggregations\" : {\n" + + " \"avg(balance)\" : {\n" + + " \"avg\" : {\n" + + " \"script\" : {\n" + + " \"source\" : \"abs(balance)\",\n" + + " \"lang\" : \"opendistro_expression\"\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("avg(balance)", new AvgAggregator( + Arrays.asList(dsl.abs(ref("balance", INTEGER))), INTEGER))), + Arrays.asList(named("age", dsl.asin(ref("age", INTEGER)))))); + } + + @Test + void should_build_type_mapping_for_expression() { + assertThat( + buildTypeMapping(Arrays.asList( + named("avg(balance)", new AvgAggregator( + Arrays.asList(dsl.abs(ref("balance", INTEGER))), INTEGER))), + Arrays.asList(named("age", dsl.asin(ref("age", INTEGER))))), + containsInAnyOrder( + map("avg(balance)", INTEGER), + map("age", DOUBLE) + )); + } + + @Test + void should_build_aggregation_without_bucket() { + assertEquals( + "{\n" + + " \"avg(balance)\" : {\n" + + " \"avg\" : {\n" + + " \"field\" : \"balance\"\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("avg(balance)", new AvgAggregator( + Arrays.asList(ref("balance", INTEGER)), INTEGER))), + Collections.emptyList())); + } + + @Test + void should_build_type_mapping_without_bucket() { + assertThat( + buildTypeMapping(Arrays.asList( + named("avg(balance)", new AvgAggregator( + Arrays.asList(ref("balance", INTEGER)), INTEGER))), + Collections.emptyList()), + containsInAnyOrder( + map("avg(balance)", INTEGER) + )); + } + + @SneakyThrows + private String buildQuery(List namedAggregatorList, + List groupByList) { + ObjectMapper objectMapper = new ObjectMapper(); + return objectMapper.readTree( + queryBuilder.buildAggregationBuilder(namedAggregatorList, groupByList).get(0).toString()) + .toPrettyString(); + } + + private Set> buildTypeMapping( + List namedAggregatorList, + List groupByList) { + return queryBuilder.buildTypeMapping(namedAggregatorList, groupByList).entrySet(); + } + + private Map.Entry map(String name, ExprType type) { + return new AbstractMap.SimpleEntry(name, type); + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptFactoryTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptFactoryTest.java new file mode 100644 index 0000000000..afd5691ace --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptFactoryTest.java @@ -0,0 +1,80 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; + +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.script.AggregationScript; +import org.elasticsearch.search.lookup.LeafSearchLookup; +import org.elasticsearch.search.lookup.SearchLookup; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +@ExtendWith(MockitoExtension.class) +class ExpressionAggregationScriptFactoryTest { + + @Mock + private SearchLookup searchLookup; + + @Mock + private LeafSearchLookup leafSearchLookup; + + @Mock + private LeafReaderContext leafReaderContext; + + private final Expression expression = DSL.literal(true); + + private final Map params = Collections.emptyMap(); + + private final AggregationScript.Factory factory = + new ExpressionAggregationScriptFactory(expression); + + @Test + void should_return_deterministic_result() { + assertTrue(factory.isResultDeterministic()); + } + + @Test + void can_initialize_expression_filter_script() throws IOException { + when(searchLookup.getLeafSearchLookup(leafReaderContext)).thenReturn(leafSearchLookup); + + AggregationScript.LeafFactory leafFactory = factory.newFactory(params, searchLookup); + assertFalse(leafFactory.needs_score()); + + AggregationScript actualScript = leafFactory.newInstance(leafReaderContext); + + assertEquals( + new ExpressionAggregationScript(expression, searchLookup, leafReaderContext, params), + actualScript + ); + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptTest.java new file mode 100644 index 0000000000..0b8bb41187 --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/ExpressionAggregationScriptTest.java @@ -0,0 +1,182 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation; + +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.STRING; +import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType.ES_TEXT_KEYWORD; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.literal; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.ref; +import static java.util.Collections.emptyMap; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; +import com.google.common.collect.ImmutableMap; +import java.util.Map; +import lombok.RequiredArgsConstructor; +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.index.fielddata.ScriptDocValues; +import org.elasticsearch.search.lookup.LeafDocLookup; +import org.elasticsearch.search.lookup.LeafSearchLookup; +import org.elasticsearch.search.lookup.SearchLookup; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +@ExtendWith(MockitoExtension.class) +class ExpressionAggregationScriptTest { + + private final DSL dsl = new ExpressionConfig().dsl(new ExpressionConfig().functionRepository()); + + @Mock + private SearchLookup lookup; + + @Mock + private LeafSearchLookup leafLookup; + + @Mock + private LeafReaderContext context; + + @Test + void can_execute_expression_with_integer_field() { + assertThat() + .docValues("age", 30L) // DocValue only supports long + .evaluate( + dsl.abs(ref("age", INTEGER))) + .shouldMatch(30); + } + + @Test + void can_execute_expression_with_integer_field_with_boolean_result() { + assertThat() + .docValues("age", 30L) // DocValue only supports long + .evaluate( + dsl.greater(ref("age", INTEGER), literal(20))) + .shouldMatch(true); + } + + @Test + void can_execute_expression_with_text_keyword_field() { + assertThat() + .docValues("name.keyword", "John") + .evaluate( + dsl.equal(ref("name", ES_TEXT_KEYWORD), literal("John"))) + .shouldMatch(true); + } + + @Test + void can_execute_expression_with_null_field() { + assertThat() + .docValues("age", null) + .evaluate(ref("age", INTEGER)) + .shouldMatch(null); + } + + @Test + void can_execute_expression_with_missing_field() { + assertThat() + .docValues("age", 30) + .evaluate(ref("name", STRING)) + .shouldMatch(null); + } + + private ExprScriptAssertion assertThat() { + return new ExprScriptAssertion(lookup, leafLookup, context); + } + + @RequiredArgsConstructor + private static class ExprScriptAssertion { + private final SearchLookup lookup; + private final LeafSearchLookup leafLookup; + private final LeafReaderContext context; + private Object actual; + + ExprScriptAssertion docValues() { + return this; + } + + ExprScriptAssertion docValues(String name, Object value) { + LeafDocLookup leafDocLookup = mockLeafDocLookup( + ImmutableMap.of(name, new FakeScriptDocValues<>(value))); + + when(lookup.getLeafSearchLookup(any())).thenReturn(leafLookup); + when(leafLookup.doc()).thenReturn(leafDocLookup); + return this; + } + + ExprScriptAssertion docValues(String name1, Object value1, + String name2, Object value2) { + LeafDocLookup leafDocLookup = mockLeafDocLookup( + ImmutableMap.of( + name1, new FakeScriptDocValues<>(value1), + name2, new FakeScriptDocValues<>(value2))); + + when(lookup.getLeafSearchLookup(any())).thenReturn(leafLookup); + when(leafLookup.doc()).thenReturn(leafDocLookup); + return this; + } + + ExprScriptAssertion evaluate(Expression expr) { + ExpressionAggregationScript script = + new ExpressionAggregationScript(expr, lookup, context, emptyMap()); + actual = script.execute(); + return this; + } + + void shouldMatch(Object expected) { + assertEquals(expected, actual); + } + + private LeafDocLookup mockLeafDocLookup(Map> docValueByNames) { + LeafDocLookup leafDocLookup = mock(LeafDocLookup.class); + when(leafDocLookup.get(anyString())) + .thenAnswer(invocation -> docValueByNames.get(invocation.getArgument(0))); + return leafDocLookup; + } + } + + @RequiredArgsConstructor + private static class FakeScriptDocValues extends ScriptDocValues { + private final T value; + + @Override + public void setNextDocId(int docId) { + throw new UnsupportedOperationException("Fake script doc values doesn't implement this yet"); + } + + @Override + public T get(int index) { + return value; + } + + @Override + public int size() { + return 1; + } + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/BucketAggregationBuilderTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/BucketAggregationBuilderTest.java new file mode 100644 index 0000000000..85c8537a22 --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/BucketAggregationBuilderTest.java @@ -0,0 +1,99 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl; + +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.elasticsearch.data.type.ElasticsearchDataType.ES_TEXT_KEYWORD; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.named; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.ref; +import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.NamedExpression; +import java.util.Arrays; +import java.util.List; +import lombok.SneakyThrows; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +@ExtendWith(MockitoExtension.class) +class BucketAggregationBuilderTest { + + @Mock + private ExpressionSerializer serializer; + + private BucketAggregationBuilder aggregationBuilder; + + @BeforeEach + void set_up() { + aggregationBuilder = new BucketAggregationBuilder(serializer); + } + + @Test + void should_build_bucket_with_field() { + assertEquals( + "{\n" + + " \"terms\" : {\n" + + " \"field\" : \"age\",\n" + + " \"missing_bucket\" : true,\n" + + " \"order\" : \"asc\"\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("age", ref("age", INTEGER))))); + } + + @Test + void should_build_bucket_with_keyword_field() { + assertEquals( + "{\n" + + " \"terms\" : {\n" + + " \"field\" : \"name.keyword\",\n" + + " \"missing_bucket\" : true,\n" + + " \"order\" : \"asc\"\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("name", ref("name", ES_TEXT_KEYWORD))))); + } + + @SneakyThrows + private String buildQuery(List groupByExpressions) { + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON).prettyPrint(); + builder.startObject(); + CompositeValuesSourceBuilder sourceBuilder = + aggregationBuilder.build(groupByExpressions).get(0); + sourceBuilder.toXContent(builder, EMPTY_PARAMS); + builder.endObject(); + return BytesReference.bytes(builder).utf8ToString(); + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/MetricAggregationBuilderTest.java b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/MetricAggregationBuilderTest.java new file mode 100644 index 0000000000..094b74235c --- /dev/null +++ b/elasticsearch/src/test/java/com/amazon/opendistroforelasticsearch/sql/elasticsearch/storage/script/aggregation/dsl/MetricAggregationBuilderTest.java @@ -0,0 +1,139 @@ +/* + * + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.script.aggregation.dsl; + +import static com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType.INTEGER; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.named; +import static com.amazon.opendistroforelasticsearch.sql.expression.DSL.ref; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.when; + +import com.amazon.opendistroforelasticsearch.sql.elasticsearch.storage.serialization.ExpressionSerializer; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AvgAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.CountAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.NamedAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.SumAggregator; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Arrays; +import java.util.List; +import lombok.SneakyThrows; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayNameGeneration; +import org.junit.jupiter.api.DisplayNameGenerator; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) +@ExtendWith(MockitoExtension.class) +class MetricAggregationBuilderTest { + + @Mock + private ExpressionSerializer serializer; + + @Mock + private NamedAggregator aggregator; + + private MetricAggregationBuilder aggregationBuilder; + + @BeforeEach + void set_up() { + aggregationBuilder = new MetricAggregationBuilder(serializer); + } + + @Test + void should_build_avg_aggregation() { + assertEquals( + "{\n" + + " \"avg(age)\" : {\n" + + " \"avg\" : {\n" + + " \"field\" : \"age\"\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("avg(age)", + new AvgAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))))); + } + + @Test + void should_build_sum_aggregation() { + assertEquals( + "{\n" + + " \"sum(age)\" : {\n" + + " \"sum\" : {\n" + + " \"field\" : \"age\"\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("sum(age)", + new SumAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))))); + } + + @Test + void should_build_count_aggregation() { + assertEquals( + "{\n" + + " \"count(age)\" : {\n" + + " \"value_count\" : {\n" + + " \"field\" : \"age\"\n" + + " }\n" + + " }\n" + + "}", + buildQuery( + Arrays.asList( + named("count(age)", + new CountAggregator(Arrays.asList(ref("age", INTEGER)), INTEGER))))); + } + + @Test + void should_throw_exception_for_unsupported_aggregator() { + when(aggregator.getFunctionName()).thenReturn(new FunctionName("max")); + when(aggregator.getArguments()).thenReturn(Arrays.asList(ref("age", INTEGER))); + + IllegalStateException exception = + assertThrows(IllegalStateException.class, () -> buildQuery(Arrays.asList(named("count(age)", + aggregator)))); + assertEquals("unsupported aggregator max", exception.getMessage()); + } + + @Test + void should_throw_exception_for_unsupported_exception() { + IllegalStateException exception = + assertThrows(IllegalStateException.class, () -> buildQuery(Arrays.asList( + named("count(age)", + new CountAggregator(Arrays.asList(named("age", ref("age", INTEGER))), INTEGER))))); + assertEquals( + "metric aggregation doesn't support expression age", + exception.getMessage()); + } + + @SneakyThrows + private String buildQuery(List namedAggregatorList) { + ObjectMapper objectMapper = new ObjectMapper(); + return objectMapper.readTree( + aggregationBuilder.build(namedAggregatorList).toString()) + .toPrettyString(); + } +} \ No newline at end of file diff --git a/integ-test/build.gradle b/integ-test/build.gradle index 622497dcf2..7b5f621dbb 100644 --- a/integ-test/build.gradle +++ b/integ-test/build.gradle @@ -115,6 +115,12 @@ task integTestWithNewEngine(type: RestIntegTestTask) { exclude 'com/amazon/opendistroforelasticsearch/sql/doctest/**/*IT.class' exclude 'com/amazon/opendistroforelasticsearch/sql/correctness/**' + // Explain IT is dependent on internal implementation of old engine so it's not necessary + // to run these with new engine and not necessary to make this consistent with old engine. + exclude 'com/amazon/opendistroforelasticsearch/sql/legacy/ExplainIT.class' + exclude 'com/amazon/opendistroforelasticsearch/sql/legacy/PrettyFormatterIT.class' + exclude 'com/amazon/opendistroforelasticsearch/sql/legacy/TermQueryExplainIT.class' + // Skip old semantic analyzer IT because analyzer in new engine has different behavior exclude 'com/amazon/opendistroforelasticsearch/sql/legacy/QueryAnalysisIT.class' } diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/runner/connection/JDBCConnection.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/runner/connection/JDBCConnection.java index 2aff6b982c..bc0ca84626 100644 --- a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/runner/connection/JDBCConnection.java +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/runner/connection/JDBCConnection.java @@ -29,6 +29,7 @@ import java.sql.Statement; import java.util.Arrays; import java.util.List; +import org.json.JSONException; import org.json.JSONObject; /** @@ -135,8 +136,7 @@ private String parseColumnNameAndTypesInSchemaJson(String schema) { JSONObject json = (JSONObject) new JSONObject(schema).query("/mappings/properties"); return json.keySet().stream(). map(colName -> colName + " " + mapToJDBCType(json.getJSONObject(colName).getString("type"))) - . - collect(joining(",")); + .collect(joining(",")); } private String getValueList(Object[] fieldValues) { diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/testset/TestDataSet.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/testset/TestDataSet.java index 42036a2f2a..8d3cfc1d6a 100644 --- a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/testset/TestDataSet.java +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/correctness/testset/TestDataSet.java @@ -118,6 +118,8 @@ private Object convertStringToObject(String type, String str) { case "text": case "keyword": case "date": + case "time": + case "timestamp": return str; case "integer": return Integer.valueOf(str); diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/ExplainIT.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/ExplainIT.java new file mode 100644 index 0000000000..70729fe23a --- /dev/null +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/ExplainIT.java @@ -0,0 +1,51 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.ppl; + +import com.google.common.io.Resources; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Paths; +import org.junit.jupiter.api.Test; + +public class ExplainIT extends PPLIntegTestCase { + + @Override + public void init() throws IOException { + loadIndex(Index.ACCOUNT); + } + + @Test + public void testExplain() throws Exception { + URI uri = Resources.getResource("expectedOutput/ppl/explain_output.json").toURI(); + String expected = new String(Files.readAllBytes(Paths.get(uri))); + assertEquals( + expected, + explainQueryToString( + "source=elasticsearch-sql_test_index_account" + + "| where age > 30 " + + "| stats avg(age) AS avg_age by state, city " + + "| sort state " + + "| fields - city " + + "| eval age2 = avg_age + 2 " + + "| dedup age2 " + + "| fields age2") + ); + } + +} diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/HeadCommandIT.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/HeadCommandIT.java new file mode 100644 index 0000000000..48217127c8 --- /dev/null +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/HeadCommandIT.java @@ -0,0 +1,87 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.ppl; + +import org.json.JSONObject; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static com.amazon.opendistroforelasticsearch.sql.legacy.TestsConstants.TEST_INDEX_ACCOUNT; +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.rows; +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.verifyDataRows; + +public class HeadCommandIT extends PPLIntegTestCase { + + @Override + public void init() throws IOException { + loadIndex(Index.ACCOUNT); + } + + @Test + public void testHead() throws IOException { + JSONObject result = + executeQuery(String.format("source=%s | fields firstname, age | head", TEST_INDEX_ACCOUNT)); + verifyDataRows(result, + rows("Amber", 32), + rows("Hattie", 36), + rows("Nanette", 28), + rows("Dale", 33), + rows("Elinor", 36), + rows("Virginia", 39), + rows("Dillard", 34), + rows("Mcgee", 39), + rows("Aurelia", 37), + rows("Fulton", 23)); + } + + @Test + public void testHeadWithNumber() throws IOException { + JSONObject result = + executeQuery(String.format("source=%s | fields firstname, age | head 3", TEST_INDEX_ACCOUNT)); + verifyDataRows(result, + rows("Amber", 32), + rows("Hattie", 36), + rows("Nanette", 28)); + } + + @Test + public void testHeadWithWhile() throws IOException { + JSONObject result = + executeQuery(String + .format("source=%s | fields firstname, age | sort age | head while(age < 21) 7", + TEST_INDEX_ACCOUNT)); + verifyDataRows(result, + rows("Claudia", 20), + rows("Copeland", 20), + rows("Cornelia", 20), + rows("Schultz", 20), + rows("Simpson", 21)); + } + + @Test + public void testHeadWithKeeplast() throws IOException { + JSONObject result = + executeQuery(String.format( + "source=%s | fields firstname, age | sort age | head keeplast=false while(age < 21) 7", + TEST_INDEX_ACCOUNT)); + verifyDataRows(result, + rows("Claudia", 20), + rows("Copeland", 20), + rows("Cornelia", 20), + rows("Schultz", 20)); + } +} diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLIntegTestCase.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLIntegTestCase.java index 40c2c604d0..6ac46a8168 100644 --- a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLIntegTestCase.java +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLIntegTestCase.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.ppl; import static com.amazon.opendistroforelasticsearch.sql.legacy.TestUtils.getResponseBody; +import static com.amazon.opendistroforelasticsearch.sql.plugin.rest.RestPPLQueryAction.EXPLAIN_API_ENDPOINT; import static com.amazon.opendistroforelasticsearch.sql.plugin.rest.RestPPLQueryAction.QUERY_API_ENDPOINT; import com.amazon.opendistroforelasticsearch.sql.legacy.SQLIntegTestCase; @@ -38,13 +39,19 @@ protected JSONObject executeQuery(String query) throws IOException { } protected String executeQueryToString(String query) throws IOException { - Response response = client().performRequest(buildRequest(query)); + Response response = client().performRequest(buildRequest(query, QUERY_API_ENDPOINT)); Assert.assertEquals(200, response.getStatusLine().getStatusCode()); return getResponseBody(response, true); } - protected Request buildRequest(String query) { - Request request = new Request("POST", QUERY_API_ENDPOINT); + protected String explainQueryToString(String query) throws IOException { + Response response = client().performRequest(buildRequest(query, EXPLAIN_API_ENDPOINT)); + Assert.assertEquals(200, response.getStatusLine().getStatusCode()); + return getResponseBody(response, true); + } + + protected Request buildRequest(String query, String endpoint) { + Request request = new Request("POST", endpoint); request.setJsonEntity(String.format(Locale.ROOT, "{\n" + " \"query\": \"%s\"\n" + "}", query)); RequestOptions.Builder restOptionsBuilder = RequestOptions.DEFAULT.toBuilder(); diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StandaloneIT.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StandaloneIT.java index e76c0da6a1..03c04b81c6 100644 --- a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StandaloneIT.java +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StandaloneIT.java @@ -114,7 +114,7 @@ public void testSourceFieldQuery() throws IOException { private String executeByStandaloneQueryEngine(String query) { AtomicReference actual = new AtomicReference<>(); pplService.execute( - new PPLQueryRequest(query, null), + new PPLQueryRequest(query, null, null), new ResponseListener() { @Override diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StatsCommandIT.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StatsCommandIT.java index 74808a1436..3d9f5e9f4d 100644 --- a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StatsCommandIT.java +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/StatsCommandIT.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.ppl; import static com.amazon.opendistroforelasticsearch.sql.legacy.TestsConstants.TEST_INDEX_ACCOUNT; +import static com.amazon.opendistroforelasticsearch.sql.legacy.TestsConstants.TEST_INDEX_BANK_WITH_NULL_VALUES; import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.rows; import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.schema; import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.verifyDataRows; @@ -30,7 +31,7 @@ public class StatsCommandIT extends PPLIntegTestCase { @Override public void init() throws IOException { loadIndex(Index.ACCOUNT); - setQuerySizeLimit(2000); + loadIndex(Index.BANK_WITH_NULL_VALUES); } @Test @@ -71,10 +72,28 @@ public void testStatsNested() throws IOException { public void testStatsWhere() throws IOException { JSONObject response = executeQuery(String.format( - "source=%s | stats sum(balance) as a by gender | where a > 13000000", TEST_INDEX_ACCOUNT)); + "source=%s | stats sum(balance) as a by state | where a > 780000", + TEST_INDEX_ACCOUNT)); verifySchema(response, schema("a", null, "long"), - schema("gender", null, "string")); - verifyDataRows(response, rows(13082527, "M")); + schema("state", null, "string")); + verifyDataRows(response, rows(782199, "TX")); } + @Test + public void testGroupByNullValue() throws IOException { + JSONObject response = + executeQuery(String.format( + "source=%s | stats avg(balance) as a by age", + TEST_INDEX_BANK_WITH_NULL_VALUES)); + verifySchema(response, schema("a", null, "double"), + schema("age", null, "integer")); + verifyDataRows(response, + rows(null, null), + rows(32838D, 28), + rows(39225D, 32), + rows(4180D, 33), + rows(48086D, 34), + rows(null, 36) + ); + } } diff --git a/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/sql/DateTimeFunctionIT.java b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/sql/DateTimeFunctionIT.java new file mode 100644 index 0000000000..62f2d6d639 --- /dev/null +++ b/integ-test/src/test/java/com/amazon/opendistroforelasticsearch/sql/sql/DateTimeFunctionIT.java @@ -0,0 +1,67 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.sql.sql; + +import static com.amazon.opendistroforelasticsearch.sql.legacy.plugin.RestSqlAction.QUERY_API_ENDPOINT; +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.rows; +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.schema; +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.verifyDataRows; +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.verifySchema; +import static com.amazon.opendistroforelasticsearch.sql.util.TestUtils.getResponseBody; + +import com.amazon.opendistroforelasticsearch.sql.legacy.SQLIntegTestCase; +import com.amazon.opendistroforelasticsearch.sql.util.TestUtils; +import java.io.IOException; +import java.util.Locale; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.json.JSONObject; +import org.junit.jupiter.api.Test; + +public class DateTimeFunctionIT extends SQLIntegTestCase { + + @Override + public void init() throws Exception { + super.init(); + TestUtils.enableNewQueryEngine(client()); + } + + @Test + public void add_date() throws IOException { + JSONObject result = + executeQuery("select adddate(timestamp('2020-09-16 17:30:00'), interval 1 day)"); + verifySchema(result, + schema("adddate(timestamp('2020-09-16 17:30:00'), interval 1 day)", null, "datetime")); + verifyDataRows(result, rows("2020-09-17 17:30:00")); + + result = executeQuery("select adddate(date('2020-09-16'), 1)"); + verifySchema(result, schema("adddate(date('2020-09-16'), 1)", null, "date")); + verifyDataRows(result, rows("2020-09-17")); + } + + protected JSONObject executeQuery(String query) throws IOException { + Request request = new Request("POST", QUERY_API_ENDPOINT); + request.setJsonEntity(String.format(Locale.ROOT, "{\n" + " \"query\": \"%s\"\n" + "}", query)); + + RequestOptions.Builder restOptionsBuilder = RequestOptions.DEFAULT.toBuilder(); + restOptionsBuilder.addHeader("Content-Type", "application/json"); + request.setOptions(restOptionsBuilder); + + Response response = client().performRequest(request); + return new JSONObject(getResponseBody(response)); + } +} diff --git a/integ-test/src/test/resources/correctness/expressions/date_and_time_functions.txt b/integ-test/src/test/resources/correctness/expressions/date_and_time_functions.txt new file mode 100644 index 0000000000..e69de29bb2 diff --git a/integ-test/src/test/resources/correctness/expressions/functions.txt b/integ-test/src/test/resources/correctness/expressions/mathematical_functions.txt similarity index 96% rename from integ-test/src/test/resources/correctness/expressions/functions.txt rename to integ-test/src/test/resources/correctness/expressions/mathematical_functions.txt index fc6120d0ce..c893e625da 100644 --- a/integ-test/src/test/resources/correctness/expressions/functions.txt +++ b/integ-test/src/test/resources/correctness/expressions/mathematical_functions.txt @@ -73,3 +73,4 @@ sin(-1.57) tan(0) tan(1.57) tan(-1.57) +dayofmonth('2020-08-26') as dom diff --git a/integ-test/src/test/resources/expectedOutput/ppl/explain_output.json b/integ-test/src/test/resources/expectedOutput/ppl/explain_output.json new file mode 100644 index 0000000000..7e133bb24e --- /dev/null +++ b/integ-test/src/test/resources/expectedOutput/ppl/explain_output.json @@ -0,0 +1,60 @@ +{ + "root": { + "name": "ProjectOperator", + "description": { + "fields": "[age2]" + }, + "children": [ + { + "name": "DedupeOperator", + "description": { + "dedupeList": "[age2]", + "allowedDuplication": 1, + "keepEmpty": false, + "consecutive": false + }, + "children": [ + { + "name": "EvalOperator", + "description": { + "expressions": { + "age2": "+(avg_age, 2)" + } + }, + "children": [ + { + "name": "RemoveOperator", + "description": { + "removeList": "[city]" + }, + "children": [ + { + "name": "SortOperator", + "description": { + "count": 1000, + "sortList": { + "state": { + "sortOrder": "ASC", + "nullOrder": "NULL_FIRST" + } + } + }, + "children": [ + { + "name": "ElasticsearchIndexScan", + "description": { + "request": "ElasticsearchQueryRequest(indexName\u003delasticsearch-sql_test_index_account, sourceBuilder\u003d{\"from\":0,\"size\":0,\"timeout\":\"1m\",\"query\":{\"range\":{\"age\":{\"from\":30,\"to\":null,\"include_lower\":false,\"include_upper\":true,\"boost\":1.0}}},\"sort\":[{\"_doc\":{\"order\":\"asc\"}}],\"aggregations\":{\"composite_buckets\":{\"composite\":{\"size\":1000,\"sources\":[{\"state\":{\"terms\":{\"field\":\"state.keyword\",\"missing_bucket\":true,\"order\":\"asc\"}}},{\"city\":{\"terms\":{\"field\":\"city.keyword\",\"missing_bucket\":true,\"order\":\"asc\"}}}]},\"aggregations\":{\"avg_age\":{\"avg\":{\"field\":\"age\"}}}}}}, searchDone\u003dfalse)" + }, + "children": [] + } + ] + } + ] + } + ] + } + ] + } + ] + } +} diff --git a/legacy/src/main/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryAction.java b/legacy/src/main/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryAction.java index 8683600d60..935c9c849f 100644 --- a/legacy/src/main/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryAction.java +++ b/legacy/src/main/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryAction.java @@ -25,9 +25,10 @@ import com.amazon.opendistroforelasticsearch.sql.common.response.ResponseListener; import com.amazon.opendistroforelasticsearch.sql.common.setting.Settings; import com.amazon.opendistroforelasticsearch.sql.elasticsearch.security.SecurityAccess; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.protocol.response.QueryResult; +import com.amazon.opendistroforelasticsearch.sql.protocol.response.format.JsonResponseFormatter; import com.amazon.opendistroforelasticsearch.sql.protocol.response.format.SimpleJsonResponseFormatter; import com.amazon.opendistroforelasticsearch.sql.sql.SQLService; import com.amazon.opendistroforelasticsearch.sql.sql.config.SQLServiceConfig; @@ -107,7 +108,11 @@ public RestChannelConsumer prepareRequest(SQLQueryRequest request, NodeClient no } catch (SyntaxCheckException e) { return NOT_SUPPORTED_YET; } - return channel -> sqlService.execute(plan, createListener(channel)); + + if (request.isExplainRequest()) { + return channel -> sqlService.explain(plan, createExplainResponseListener(channel)); + } + return channel -> sqlService.execute(plan, createQueryResponseListener(channel)); } private SQLService createSQLService(NodeClient client) { @@ -123,25 +128,41 @@ private SQLService createSQLService(NodeClient client) { }); } + private ResponseListener createExplainResponseListener(RestChannel channel) { + return new ResponseListener() { + @Override + public void onResponse(ExplainResponse response) { + sendResponse(channel, OK, new JsonResponseFormatter(PRETTY) { + @Override + protected Object buildJsonObject(ExplainResponse response) { + return response; + } + }.format(response)); + } + + @Override + public void onFailure(Exception e) { + LOG.error("Error happened during explain", e); + sendResponse(channel, INTERNAL_SERVER_ERROR, + "Failed to explain the query due to error: " + e.getMessage()); + } + }; + } + // TODO: duplicate code here as in RestPPLQueryAction - private ResponseListener createListener(RestChannel channel) { + private ResponseListener createQueryResponseListener(RestChannel channel) { SimpleJsonResponseFormatter formatter = new SimpleJsonResponseFormatter(PRETTY); return new ResponseListener() { @Override public void onResponse(QueryResponse response) { - sendResponse(OK, formatter.format(new QueryResult(response.getSchema(), - response.getResults()))); + sendResponse(channel, OK, + formatter.format(new QueryResult(response.getSchema(), response.getResults()))); } @Override public void onFailure(Exception e) { LOG.error("Error happened during query handling", e); - sendResponse(INTERNAL_SERVER_ERROR, formatter.format(e)); - } - - private void sendResponse(RestStatus status, String content) { - channel.sendResponse(new BytesRestResponse( - status, "application/json; charset=UTF-8", content)); + sendResponse(channel, INTERNAL_SERVER_ERROR, formatter.format(e)); } }; } @@ -154,4 +175,9 @@ private T doPrivileged(PrivilegedExceptionAction action) { } } + private void sendResponse(RestChannel channel, RestStatus status, String content) { + channel.sendResponse(new BytesRestResponse( + status, "application/json; charset=UTF-8", content)); + } + } diff --git a/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryActionTest.java b/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryActionTest.java index 50549e5884..2d420debf7 100644 --- a/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryActionTest.java +++ b/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/plugin/RestSQLQueryActionTest.java @@ -57,15 +57,15 @@ public void handleQueryThatCanSupport() { } @Test - public void skipExplainThatNotSupport() { + public void handleExplainThatCanSupport() { SQLQueryRequest request = new SQLQueryRequest( - new JSONObject("{\"query\": \"SELECT * FROM test\"}"), - "SELECT * FROM test", + new JSONObject("{\"query\": \"SELECT -123\"}"), + "SELECT -123", EXPLAIN_API_ENDPOINT, ""); RestSQLQueryAction queryAction = new RestSQLQueryAction(clusterService, settings); - assertSame(NOT_SUPPORTED_YET, queryAction.prepareRequest(request, nodeClient)); + assertNotSame(NOT_SUPPORTED_YET, queryAction.prepareRequest(request, nodeClient)); } @Test diff --git a/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/unittest/SqlRequestFactoryTest.java b/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/unittest/SqlRequestFactoryTest.java index 8bff20f5cc..39620eb567 100644 --- a/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/unittest/SqlRequestFactoryTest.java +++ b/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/unittest/SqlRequestFactoryTest.java @@ -15,12 +15,19 @@ package com.amazon.opendistroforelasticsearch.sql.legacy.unittest; +import static java.util.Collections.emptyList; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +import com.amazon.opendistroforelasticsearch.sql.legacy.esdomain.LocalClusterState; +import com.amazon.opendistroforelasticsearch.sql.legacy.plugin.SqlSettings; import com.amazon.opendistroforelasticsearch.sql.legacy.request.PreparedStatementRequest; import com.amazon.opendistroforelasticsearch.sql.legacy.request.SqlRequest; import com.amazon.opendistroforelasticsearch.sql.legacy.request.SqlRequestFactory; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.rest.RestRequest; import org.junit.Assert; +import org.junit.Before; import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; @@ -34,6 +41,15 @@ public class SqlRequestFactoryTest { @Mock private RestRequest restRequest; + @Before + public void setup() { + SqlSettings settings = spy(new SqlSettings()); + // Force return empty list to avoid ClusterSettings be invoked which is a final class and hard to mock. + // In this case, default value in Setting will be returned all the time. + doReturn(emptyList()).when(settings).getSettings(); + LocalClusterState.state().setSqlSettings(settings); + } + @Ignore("RestRequest is a final method, and Mockito 1.x cannot mock it." + "Ignore this test case till we can upgrade to Mockito 2.x") @Test diff --git a/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/util/AggregationUtils.java b/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/util/AggregationUtils.java index 0f270fb007..9f949a0ede 100644 --- a/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/util/AggregationUtils.java +++ b/legacy/src/test/java/com/amazon/opendistroforelasticsearch/sql/legacy/util/AggregationUtils.java @@ -16,6 +16,9 @@ package com.amazon.opendistroforelasticsearch.sql.legacy.util; import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.xcontent.ContextParser; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; @@ -47,10 +50,6 @@ import org.elasticsearch.search.aggregations.pipeline.ParsedPercentilesBucket; import org.elasticsearch.search.aggregations.pipeline.PercentilesBucketPipelineAggregationBuilder; -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; - public class AggregationUtils { private final static List entryList = new ImmutableMap.Builder>().put( diff --git a/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/request/PPLQueryRequestFactory.java b/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/request/PPLQueryRequestFactory.java index 99eb56e457..dde892e5af 100644 --- a/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/request/PPLQueryRequestFactory.java +++ b/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/request/PPLQueryRequestFactory.java @@ -51,7 +51,7 @@ private static PPLQueryRequest parsePPLRequestFromUrl(RestRequest restRequest) { if (ppl == null) { throw new IllegalArgumentException("Cannot find ppl parameter from the URL"); } - return new PPLQueryRequest(ppl, null); + return new PPLQueryRequest(ppl, null, restRequest.path()); } private static PPLQueryRequest parsePPLRequestFromPayload(RestRequest restRequest) { @@ -62,6 +62,7 @@ private static PPLQueryRequest parsePPLRequestFromPayload(RestRequest restReques } catch (JSONException e) { throw new IllegalArgumentException("Failed to parse request payload", e); } - return new PPLQueryRequest(jsonContent.getString(PPL_FIELD_NAME), jsonContent); + return new PPLQueryRequest(jsonContent.getString(PPL_FIELD_NAME), + jsonContent, restRequest.path()); } } diff --git a/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/rest/RestPPLQueryAction.java b/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/rest/RestPPLQueryAction.java index 8314462832..015f4aff16 100644 --- a/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/rest/RestPPLQueryAction.java +++ b/plugin/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/rest/RestPPLQueryAction.java @@ -17,6 +17,7 @@ import static com.amazon.opendistroforelasticsearch.sql.protocol.response.format.JsonResponseFormatter.Style.PRETTY; import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; +import static org.elasticsearch.rest.RestStatus.INTERNAL_SERVER_ERROR; import static org.elasticsearch.rest.RestStatus.OK; import static org.elasticsearch.rest.RestStatus.SERVICE_UNAVAILABLE; @@ -28,6 +29,7 @@ import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; import com.amazon.opendistroforelasticsearch.sql.exception.QueryEngineException; import com.amazon.opendistroforelasticsearch.sql.exception.SemanticCheckException; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.QueryResponse; import com.amazon.opendistroforelasticsearch.sql.legacy.metrics.MetricName; import com.amazon.opendistroforelasticsearch.sql.legacy.metrics.Metrics; @@ -35,11 +37,13 @@ import com.amazon.opendistroforelasticsearch.sql.plugin.request.PPLQueryRequestFactory; import com.amazon.opendistroforelasticsearch.sql.ppl.PPLService; import com.amazon.opendistroforelasticsearch.sql.ppl.config.PPLServiceConfig; +import com.amazon.opendistroforelasticsearch.sql.ppl.domain.PPLQueryRequest; import com.amazon.opendistroforelasticsearch.sql.protocol.response.QueryResult; +import com.amazon.opendistroforelasticsearch.sql.protocol.response.format.JsonResponseFormatter; import com.amazon.opendistroforelasticsearch.sql.protocol.response.format.SimpleJsonResponseFormatter; import java.io.IOException; import java.security.PrivilegedExceptionAction; -import java.util.Collections; +import java.util.Arrays; import java.util.List; import java.util.function.Supplier; import org.apache.logging.log4j.LogManager; @@ -57,6 +61,7 @@ public class RestPPLQueryAction extends BaseRestHandler { public static final String QUERY_API_ENDPOINT = "/_opendistro/_ppl"; + public static final String EXPLAIN_API_ENDPOINT = "/_opendistro/_ppl/_explain"; private static final Logger LOG = LogManager.getLogger(); @@ -88,8 +93,9 @@ public RestPPLQueryAction(RestController restController, ClusterService clusterS @Override public List routes() { - return Collections.singletonList( - new Route(RestRequest.Method.POST, QUERY_API_ENDPOINT) + return Arrays.asList( + new Route(RestRequest.Method.POST, QUERY_API_ENDPOINT), + new Route(RestRequest.Method.POST, EXPLAIN_API_ENDPOINT) ); } @@ -110,9 +116,13 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient nod "Either opendistro.ppl.enabled or rest.action.multi.allow_explicit_index setting is false" ), BAD_REQUEST); } + PPLService pplService = createPPLService(nodeClient); - return channel -> pplService.execute( - PPLQueryRequestFactory.getPPLRequest(request), createListener(channel)); + PPLQueryRequest pplRequest = PPLQueryRequestFactory.getPPLRequest(request); + if (pplRequest.isExplainRequest()) { + return channel -> pplService.explain(pplRequest, createExplainResponseListener(channel)); + } + return channel -> pplService.execute(pplRequest, createListener(channel)); } /** @@ -140,13 +150,40 @@ private PPLService createPPLService(NodeClient client) { }); } + /** + * TODO: need to extract an interface for both SQL and PPL action handler and move these + * common methods to the interface. This is not easy to do now because SQL action handler + * is still in legacy module. + */ + private ResponseListener createExplainResponseListener( + RestChannel channel) { + return new ResponseListener() { + @Override + public void onResponse(ExplainResponse response) { + sendResponse(channel, OK, new JsonResponseFormatter(PRETTY) { + @Override + protected Object buildJsonObject(ExplainResponse response) { + return response; + } + }.format(response)); + } + + @Override + public void onFailure(Exception e) { + LOG.error("Error happened during explain", e); + sendResponse(channel, INTERNAL_SERVER_ERROR, + "Failed to explain the query due to error: " + e.getMessage()); + } + }; + } + private ResponseListener createListener(RestChannel channel) { SimpleJsonResponseFormatter formatter = new SimpleJsonResponseFormatter(PRETTY); // TODO: decide format and pretty from URL param return new ResponseListener() { @Override public void onResponse(QueryResponse response) { - sendResponse(OK, formatter.format(new QueryResult(response.getSchema(), + sendResponse(channel, OK, formatter.format(new QueryResult(response.getSchema(), response.getResults()))); } @@ -161,11 +198,6 @@ public void onFailure(Exception e) { reportError(channel, e, SERVICE_UNAVAILABLE); } } - - private void sendResponse(RestStatus status, String content) { - channel.sendResponse( - new BytesRestResponse(status, "application/json; charset=UTF-8", content)); - } }; } @@ -177,6 +209,11 @@ private T doPrivileged(PrivilegedExceptionAction action) { } } + private void sendResponse(RestChannel channel, RestStatus status, String content) { + channel.sendResponse( + new BytesRestResponse(status, "application/json; charset=UTF-8", content)); + } + private void reportError(final RestChannel channel, final Exception e, final RestStatus status) { channel.sendResponse(new BytesRestResponse(status, ErrorMessageFactory.createErrorMessage(e, status.getStatus()).toString())); diff --git a/ppl/src/main/antlr/OpenDistroPPLLexer.g4 b/ppl/src/main/antlr/OpenDistroPPLLexer.g4 index 377bd38050..3c8377f103 100644 --- a/ppl/src/main/antlr/OpenDistroPPLLexer.g4 +++ b/ppl/src/main/antlr/OpenDistroPPLLexer.g4 @@ -28,6 +28,7 @@ STATS: 'STATS'; DEDUP: 'DEDUP'; SORT: 'SORT'; EVAL: 'EVAL'; +HEAD: 'HEAD'; TOP: 'TOP'; RARE: 'RARE'; @@ -50,11 +51,13 @@ NUM: 'NUM'; // ARGUMENT KEYWORDS KEEPEMPTY: 'KEEPEMPTY'; +KEEPLAST: 'KEEPLAST'; CONSECUTIVE: 'CONSECUTIVE'; DEDUP_SPLITVALUES: 'DEDUP_SPLITVALUES'; PARTITIONS: 'PARTITIONS'; ALLNUM: 'ALLNUM'; DELIM: 'DELIM'; +WHILE: 'WHILE'; // COMPARISON FUNCTION KEYWORDS CASE: 'CASE'; @@ -199,6 +202,7 @@ TAN: 'TAN'; DATE: 'DATE'; TIME: 'TIME'; TIMESTAMP: 'TIMESTAMP'; +ADDDATE: 'ADDDATE'; // TEXT FUNCTIONS SUBSTR: 'SUBSTR'; diff --git a/ppl/src/main/antlr/OpenDistroPPLParser.g4 b/ppl/src/main/antlr/OpenDistroPPLParser.g4 index 1de967b8aa..1c3f7f87bc 100644 --- a/ppl/src/main/antlr/OpenDistroPPLParser.g4 +++ b/ppl/src/main/antlr/OpenDistroPPLParser.g4 @@ -28,7 +28,7 @@ pplStatement /** commands */ commands - : whereCommand | fieldsCommand | renameCommand | statsCommand | dedupCommand | sortCommand | evalCommand + : whereCommand | fieldsCommand | renameCommand | statsCommand | dedupCommand | sortCommand | evalCommand | headCommand | topCommand | rareCommand; searchCommand @@ -75,6 +75,13 @@ evalCommand : EVAL evalClause (COMMA evalClause)* ; +headCommand + : HEAD + (KEEPLAST EQUAL keeplast=booleanLiteral)? + (WHILE LT_PRTHS whileExpr=logicalExpression RT_PRTHS)? + (number=integerLiteral)? + ; + topCommand : TOP (number=integerLiteral)? @@ -226,7 +233,7 @@ trigonometricFunctionName ; dateAndTimeFunctionBase - : DATE | TIME | TIMESTAMP + : DATE | TIME | TIMESTAMP | ADDDATE ; textFunctionBase diff --git a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLService.java b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLService.java index 08ca8f4383..8451a053b2 100644 --- a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLService.java +++ b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLService.java @@ -22,6 +22,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan; import com.amazon.opendistroforelasticsearch.sql.common.response.ResponseListener; import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; import com.amazon.opendistroforelasticsearch.sql.planner.Planner; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; @@ -51,21 +52,39 @@ public class PPLService { */ public void execute(PPLQueryRequest request, ResponseListener listener) { try { - // 1.Parse query and convert parse tree (CST) to abstract syntax tree (AST) - ParseTree cst = parser.analyzeSyntax(request.getRequest()); - UnresolvedPlan ast = cst.accept(new AstBuilder(new AstExpressionBuilder())); - - // 2.Analyze abstract syntax to generate logical plan - LogicalPlan logicalPlan = analyzer.analyze(UnresolvedPlanHelper.addSelectAll(ast), - new AnalysisContext()); - - // 3.Generate optimal physical plan from logical plan - PhysicalPlan physicalPlan = new Planner(storageEngine).plan(logicalPlan); + executionEngine.execute(plan(request), listener); + } catch (Exception e) { + listener.onFailure(e); + } + } - // 4.Execute physical plan and send response - executionEngine.execute(physicalPlan, listener); + /** + * Explain the query in {@link PPLQueryRequest} using {@link ResponseListener} to + * get and format explain response. + * + * @param request {@link PPLQueryRequest} + * @param listener {@link ResponseListener} for explain response + */ + public void explain(PPLQueryRequest request, ResponseListener listener) { + try { + executionEngine.explain(plan(request), listener); } catch (Exception e) { listener.onFailure(e); } } + + private PhysicalPlan plan(PPLQueryRequest request) { + // 1.Parse query and convert parse tree (CST) to abstract syntax tree (AST) + ParseTree cst = parser.analyzeSyntax(request.getRequest()); + UnresolvedPlan ast = cst.accept( + new AstBuilder(new AstExpressionBuilder(), request.getRequest())); + + // 2.Analyze abstract syntax to generate logical plan + LogicalPlan logicalPlan = analyzer.analyze(UnresolvedPlanHelper.addSelectAll(ast), + new AnalysisContext()); + + // 3.Generate optimal physical plan from logical plan + return new Planner(storageEngine).plan(logicalPlan); + } + } diff --git a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequest.java b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequest.java index 9bb348fd43..074acf062d 100644 --- a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequest.java +++ b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequest.java @@ -20,12 +20,22 @@ @RequiredArgsConstructor public class PPLQueryRequest { - public static final PPLQueryRequest NULL = new PPLQueryRequest("", null); + public static final PPLQueryRequest NULL = new PPLQueryRequest("", null, ""); private final String pplQuery; private final JSONObject jsonContent; + private final String path; public String getRequest() { return pplQuery; } + + /** + * Check if request is to explain rather than execute the query. + * @return true if it is a explain request + */ + public boolean isExplainRequest() { + return path.endsWith("/_explain"); + } + } diff --git a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilder.java b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilder.java index 1058bacb96..3f094bfab7 100644 --- a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilder.java +++ b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilder.java @@ -19,6 +19,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.EvalCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.FieldsCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.FromClauseContext; +import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.HeadCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.PplStatementContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.RareCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.RenameCommandContext; @@ -30,6 +31,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.TopCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.WhereCommandContext; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.Alias; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Argument; import com.amazon.opendistroforelasticsearch.sql.ast.expression.DataType; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Field; @@ -41,6 +43,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.Dedupe; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Eval; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Head; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN; import com.amazon.opendistroforelasticsearch.sql.ast.tree.RareTopN.CommandType; @@ -48,6 +51,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort; import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan; +import com.amazon.opendistroforelasticsearch.sql.common.utils.StringUtils; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser; import com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.ByClauseContext; @@ -59,6 +63,8 @@ import java.util.List; import java.util.stream.Collectors; import lombok.RequiredArgsConstructor; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.Token; import org.antlr.v4.runtime.tree.ParseTree; /** @@ -67,8 +73,15 @@ */ @RequiredArgsConstructor public class AstBuilder extends OpenDistroPPLParserBaseVisitor { + private final AstExpressionBuilder expressionBuilder; + /** + * PPL query to get original token text. This is necessary because token.getText() returns + * text without whitespaces or other characters discarded by lexer. + */ + private final String query; + @Override public UnresolvedPlan visitPplStatement(PplStatementContext ctx) { UnresolvedPlan search = visit(ctx.searchCommand()); @@ -138,25 +151,29 @@ public UnresolvedPlan visitRenameCommand(RenameCommandContext ctx) { @Override public UnresolvedPlan visitStatsCommand(StatsCommandContext ctx) { ImmutableList.Builder aggListBuilder = new ImmutableList.Builder<>(); - ImmutableList.Builder renameListBuilder = new ImmutableList.Builder<>(); for (OpenDistroPPLParser.StatsAggTermContext aggCtx : ctx.statsAggTerm()) { UnresolvedExpression aggExpression = visitExpression(aggCtx.statsFunction()); - aggListBuilder.add(aggExpression); - if (aggCtx.alias != null) { - renameListBuilder - .add(new Map(aggExpression, visitExpression(aggCtx.alias))); - } + String name = aggCtx.alias == null ? getTextInQuery(aggCtx) : StringUtils + .unquoteIdentifier(aggCtx.alias.getText()); + Alias alias = new Alias(name, aggExpression); + aggListBuilder.add(alias); } + List groupList = ctx.byClause() == null ? Collections.emptyList() : - getGroupByList(ctx.byClause()); + ctx.byClause() + .fieldList() + .fieldExpression() + .stream() + .map(groupCtx -> new Alias(getTextInQuery(groupCtx), visitExpression(groupCtx))) + .collect(Collectors.toList()); + Aggregation aggregation = new Aggregation( aggListBuilder.build(), Collections.emptyList(), groupList, ArgumentFactory.getArgumentList(ctx) ); - List renameList = renameListBuilder.build(); - return renameList.isEmpty() ? aggregation : new Rename(renameList, aggregation); + return aggregation; } /** @@ -170,6 +187,16 @@ public UnresolvedPlan visitDedupCommand(DedupCommandContext ctx) { ); } + /** + * Head command visitor. + */ + @Override + public UnresolvedPlan visitHeadCommand(HeadCommandContext ctx) { + UnresolvedExpression unresolvedExpr = + ctx.whileExpr != null ? visitExpression(ctx.logicalExpression()) : null; + return new Head(ArgumentFactory.getArgumentList(ctx, unresolvedExpr)); + } + /** * Sort command. */ @@ -266,4 +293,12 @@ protected UnresolvedPlan aggregateResult(UnresolvedPlan aggregate, UnresolvedPla return aggregate; } + /** + * Get original text in query. + */ + private String getTextInQuery(ParserRuleContext ctx) { + Token start = ctx.getStart(); + Token stop = ctx.getStop(); + return query.substring(start.getStartIndex(), stop.getStopIndex() + 1); + } } diff --git a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactory.java b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactory.java index abb525e06f..872e14c9dc 100644 --- a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactory.java +++ b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactory.java @@ -18,6 +18,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.BooleanLiteralContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.DedupCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.FieldsCommandContext; +import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.HeadCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.IntegerLiteralContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.RareCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.SortCommandContext; @@ -28,10 +29,13 @@ import com.amazon.opendistroforelasticsearch.sql.ast.expression.Argument; import com.amazon.opendistroforelasticsearch.sql.ast.expression.DataType; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Literal; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedArgument; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.common.utils.StringUtils; import java.util.Arrays; import java.util.Collections; import java.util.List; + import org.antlr.v4.runtime.ParserRuleContext; @@ -97,6 +101,27 @@ public static List getArgumentList(DedupCommandContext ctx) { ); } + /** + * Get list of {@link Argument}. + * + * @param ctx HeadCommandContext instance + * @return the list of arguments fetched from the head command + */ + public static List getArgumentList(HeadCommandContext ctx, + UnresolvedExpression unresolvedExpr) { + return Arrays.asList( + ctx.keeplast != null + ? new UnresolvedArgument("keeplast", getArgumentValue(ctx.keeplast)) + : new UnresolvedArgument("keeplast", new Literal(true, DataType.BOOLEAN)), + ctx.whileExpr != null && unresolvedExpr != null + ? new UnresolvedArgument("whileExpr", unresolvedExpr) + : new UnresolvedArgument("whileExpr", new Literal(true, DataType.BOOLEAN)), + ctx.number != null + ? new UnresolvedArgument("number", getArgumentValue(ctx.number)) + : new UnresolvedArgument("number", new Literal(10, DataType.INTEGER)) + ); + } + /** * Get list of {@link Argument}. * diff --git a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLServiceTest.java b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLServiceTest.java index ed8449bd0c..4d6acf62f0 100644 --- a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLServiceTest.java +++ b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/PPLServiceTest.java @@ -22,6 +22,8 @@ import com.amazon.opendistroforelasticsearch.sql.common.response.ResponseListener; import com.amazon.opendistroforelasticsearch.sql.data.type.ExprCoreType; import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponse; +import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.ExplainResponseNode; import com.amazon.opendistroforelasticsearch.sql.executor.ExecutionEngine.QueryResponse; import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan; import com.amazon.opendistroforelasticsearch.sql.ppl.config.PPLServiceConfig; @@ -83,7 +85,7 @@ public void testExecuteShouldPass() { return null; }).when(executionEngine).execute(any(), any()); - pplService.execute(new PPLQueryRequest("search source=t a=1", null), + pplService.execute(new PPLQueryRequest("search source=t a=1", null, null), new ResponseListener() { @Override public void onResponse(QueryResponse pplQueryResponse) { @@ -97,33 +99,72 @@ public void onFailure(Exception e) { }); } + @Test + public void testExplainShouldPass() { + doAnswer(invocation -> { + ResponseListener listener = invocation.getArgument(1); + listener.onResponse(new ExplainResponse(new ExplainResponseNode("test"))); + return null; + }).when(executionEngine).explain(any(), any()); + + pplService.explain(new PPLQueryRequest("search source=t a=1", null, null), + new ResponseListener() { + @Override + public void onResponse(ExplainResponse pplQueryResponse) { + } + + @Override + public void onFailure(Exception e) { + Assert.fail(); + } + }); + } + @Test public void testExecuteWithIllegalQueryShouldBeCaughtByHandler() { - pplService.execute(new PPLQueryRequest("search", null), new ResponseListener() { - @Override - public void onResponse(QueryResponse pplQueryResponse) { - Assert.fail(); - } + pplService.execute(new PPLQueryRequest("search", null, null), + new ResponseListener() { + @Override + public void onResponse(QueryResponse pplQueryResponse) { + Assert.fail(); + } + + @Override + public void onFailure(Exception e) { - @Override - public void onFailure(Exception e) { + } + }); + } + + @Test + public void testExplainWithIllegalQueryShouldBeCaughtByHandler() { + pplService.explain(new PPLQueryRequest("search", null, null), + new ResponseListener() { + @Override + public void onResponse(ExplainResponse pplQueryResponse) { + Assert.fail(); + } + + @Override + public void onFailure(Exception e) { - } - }); + } + }); } @Test public void test() { - pplService.execute(new PPLQueryRequest("search", null), new ResponseListener() { - @Override - public void onResponse(QueryResponse pplQueryResponse) { - Assert.fail(); - } + pplService.execute(new PPLQueryRequest("search", null, null), + new ResponseListener() { + @Override + public void onResponse(QueryResponse pplQueryResponse) { + Assert.fail(); + } - @Override - public void onFailure(Exception e) { + @Override + public void onFailure(Exception e) { - } - }); + } + }); } } \ No newline at end of file diff --git a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequestTest.java b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequestTest.java index 0b1e959d22..5e80f51158 100644 --- a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequestTest.java +++ b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/domain/PPLQueryRequestTest.java @@ -15,12 +15,22 @@ package com.amazon.opendistroforelasticsearch.sql.ppl.domain; +import static org.junit.Assert.assertTrue; + import org.junit.Test; public class PPLQueryRequestTest { @Test public void getRequestShouldPass() { - PPLQueryRequest request = new PPLQueryRequest("source=t a=1", null); + PPLQueryRequest request = new PPLQueryRequest("source=t a=1", null, null); request.getRequest(); } + + @Test + public void testExplainRequest() { + PPLQueryRequest request = new PPLQueryRequest( + "source=t a=1", null, "/_opendistro/_ppl/_explain"); + assertTrue(request.isExplainRequest()); + } + } diff --git a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilderTest.java b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilderTest.java index 0bff1d62bf..ac174d9019 100644 --- a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilderTest.java +++ b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstBuilderTest.java @@ -17,12 +17,14 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.agg; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.aggregate; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.alias; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.argument; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.booleanLiteral; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.compare; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.dedupe; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.defaultDedupArgs; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.defaultFieldsArgs; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.defaultHeadArgs; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.defaultSortFieldArgs; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.defaultSortOptions; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.defaultStatsArgs; @@ -31,6 +33,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.field; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.filter; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.function; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.head; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.intLiteral; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.let; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.map; @@ -42,6 +45,8 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.sort; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.sortOptions; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.stringLiteral; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.unresolvedArg; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.unresolvedArgList; import static java.util.Collections.emptyList; import static org.junit.Assert.assertEquals; @@ -54,7 +59,6 @@ public class AstBuilderTest { private PPLSyntaxParser parser = new PPLSyntaxParser(); - private AstBuilder astBuilder = new AstBuilder(new AstExpressionBuilder()); @Test public void testSearchCommand() { @@ -161,7 +165,10 @@ public void testStatsCommand() { agg( relation("t"), exprList( - aggregate("count", field("a")) + alias( + "count(a)", + aggregate("count", field("a")) + ) ), emptyList(), emptyList(), @@ -175,10 +182,17 @@ public void testStatsCommandWithByClause() { agg( relation("t"), exprList( - aggregate("count", field("a")) + alias( + "count(a)", + aggregate("count", field("a")) + ) ), emptyList(), - exprList(field("b")), + exprList( + alias( + "b", + field("b") + )), defaultStatsArgs() )); } @@ -186,17 +200,17 @@ public void testStatsCommandWithByClause() { @Test public void testStatsCommandWithAlias() { assertEqual("source=t | stats count(a) as alias", - rename( - agg( - relation("t"), - exprList( + agg( + relation("t"), + exprList( + alias( + "alias", aggregate("count", field("a")) - ), - emptyList(), - emptyList(), - defaultStatsArgs() + ) ), - map(aggregate("count", field("a")), field("alias")) + emptyList(), + emptyList(), + defaultStatsArgs() ) ); } @@ -207,10 +221,13 @@ public void testStatsCommandWithNestedFunctions() { agg( relation("t"), exprList( - aggregate( - "sum", - function("+", field("a"), field("b")) - )), + alias( + "sum(a+b)", + aggregate( + "sum", + function("+", field("a"), field("b")) + )) + ), emptyList(), emptyList(), defaultStatsArgs() @@ -219,12 +236,15 @@ public void testStatsCommandWithNestedFunctions() { agg( relation("t"), exprList( - aggregate( - "sum", - function( - "/", - function("abs", field("a")), - intLiteral(2) + alias( + "sum(abs(a)/2)", + aggregate( + "sum", + function( + "/", + function("abs", field("a")), + intLiteral(2) + ) ) ) ), @@ -259,6 +279,53 @@ public void testDedupCommandWithSortby() { )); } + @Test + public void testHeadCommand() { + assertEqual("source=t | head", + head( + relation("t"), + defaultHeadArgs() + )); + } + + @Test + public void testHeadCommandWithNumber() { + assertEqual("source=t | head 3", + head( + relation("t"), + unresolvedArgList( + unresolvedArg("keeplast", booleanLiteral(true)), + unresolvedArg("whileExpr", booleanLiteral(true)), + unresolvedArg("number", intLiteral(3))) + )); + } + + @Test + public void testHeadCommandWithWhileExpr() { + + assertEqual("source=t | head while(a < 5) 5", + head( + relation("t"), + unresolvedArgList( + unresolvedArg("keeplast", booleanLiteral(true)), + unresolvedArg("whileExpr", compare("<", field("a"), intLiteral(5))), + unresolvedArg("number", intLiteral(5))) + )); + } + + @Test + public void testHeadCommandWithKeepLast() { + + assertEqual("source=t | head keeplast=false while(a < 5) 5", + head( + relation("t"), + unresolvedArgList( + unresolvedArg("keeplast", booleanLiteral(false)), + unresolvedArg("whileExpr", compare("<", field("a"), intLiteral(5))), + unresolvedArg("number", intLiteral(5))) + )); + } + @Test public void testSortCommand() { assertEqual("source=t | sort f1, f2", @@ -400,6 +467,7 @@ protected void assertEqual(String query, String expected) { } private Node plan(String query) { + AstBuilder astBuilder = new AstBuilder(new AstExpressionBuilder(), query); return astBuilder.visit(parser.analyzeSyntax(query)); } } diff --git a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilderTest.java b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilderTest.java index b2598e3b1f..962cbcb34c 100644 --- a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilderTest.java +++ b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilderTest.java @@ -17,6 +17,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.agg; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.aggregate; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.alias; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.and; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.argument; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.booleanLiteral; @@ -295,11 +296,17 @@ public void testAggFuncCallExpr() { agg( relation("t"), exprList( - aggregate("avg", field("a")) - + alias( + "avg(a)", + aggregate("avg", field("a")) + ) ), emptyList(), - exprList(field("b")), + exprList( + alias( + "b", + field("b") + )), defaultStatsArgs() )); } @@ -310,10 +317,12 @@ public void testPercentileAggFuncExpr() { agg( relation("t"), exprList( - aggregate( - "percentile", - field("a"), - argument("rank", intLiteral(1)) + alias("percentile<1>(a)", + aggregate( + "percentile", + field("a"), + argument("rank", intLiteral(1)) + ) ) ), emptyList(), diff --git a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactoryTest.java b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactoryTest.java index f2400b5ade..9e8512fce8 100644 --- a/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactoryTest.java +++ b/ppl/src/test/java/com/amazon/opendistroforelasticsearch/sql/ppl/utils/ArgumentFactoryTest.java @@ -17,6 +17,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.agg; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.aggregate; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.alias; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.argument; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.booleanLiteral; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.dedupe; @@ -57,7 +58,11 @@ public void testStatsCommandArgument() { "source=t | stats partitions=1 allnum=false delim=',' avg(a) dedup_splitvalues=true", agg( relation("t"), - exprList(aggregate("avg", field("a"))), + exprList( + alias( + "avg(a)", + aggregate("avg", field("a"))) + ), emptyList(), emptyList(), exprList( diff --git a/release-notes/opendistro-for-elasticsearch-sql.release-notes-1.10.1.0.md b/release-notes/opendistro-for-elasticsearch-sql.release-notes-1.10.1.0.md index 9db4808eee..9485c912a5 100644 --- a/release-notes/opendistro-for-elasticsearch-sql.release-notes-1.10.1.0.md +++ b/release-notes/opendistro-for-elasticsearch-sql.release-notes-1.10.1.0.md @@ -38,7 +38,8 @@ ### Documentation * update user documentation for testing odbc driver connection on windows([#722](https://github.com/opendistro-for-elasticsearch/sql/pull/722)) * Added workaround for identifiers with special characters in troubleshooting page([#718](https://github.com/opendistro-for-elasticsearch/sql/pull/718)) -* Update release notes for OD 1.10.1 release([#699](https://github.com/opendistro-for-elasticsearch/sql/pull/699)) +* Update release notes for OD 1.10 release([#699](https://github.com/opendistro-for-elasticsearch/sql/pull/699)) ### Maintenance +* Bumped ES and Kibana versions to v7.9.0 ([#697](https://github.com/opendistro-for-elasticsearch/sql/pull/697)) * Bump ES and Kibana to 7.9.1 and release ODFE 1.10.1.0 ([#732](https://github.com/opendistro-for-elasticsearch/sql/pull/732)) diff --git a/sql-jdbc/src/main/java/ESConnect.java b/sql-jdbc/src/main/java/ESConnect.java new file mode 100644 index 0000000000..c53911d47c --- /dev/null +++ b/sql-jdbc/src/main/java/ESConnect.java @@ -0,0 +1,36 @@ +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.Statement; + +public class ESConnect { + static final String URL = "jdbc:elasticsearch://http://localhost:9200"; + + public static void main(String[] args) throws Exception { + System.out.println("Connection"); + Connection con = DriverManager.getConnection(URL, null, null); + System.out.println("Creating statement"); + Statement st = con.createStatement(); + String sql = "SELECT SUBSTRING('hello', 2)"; + System.out.println("Executing query"); + ResultSet rs = st.executeQuery(sql); + + System.out.println("Reading results."); + ResultSetMetaData metaData = rs.getMetaData(); + for (int i = 1; i <= metaData.getColumnCount(); i++) { + System.out.println(metaData.getColumnTypeName(i)); + } + System.out.println(); + + while (rs.next()) { + for (int i = 1; i <= metaData.getColumnCount(); i++) { + System.out.print("Column: " + rs.getObject(i)); + } + System.out.println(); + } + rs.close(); + st.close(); + con.close(); + } +} \ No newline at end of file diff --git a/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/BaseTypeConverter.java b/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/BaseTypeConverter.java index 15a88006f3..dcb2058ccd 100644 --- a/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/BaseTypeConverter.java +++ b/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/BaseTypeConverter.java @@ -18,6 +18,7 @@ import java.sql.Date; import java.sql.SQLException; +import java.sql.Time; import java.sql.Timestamp; import java.util.HashMap; import java.util.Map; @@ -42,6 +43,7 @@ public abstract class BaseTypeConverter implements TypeConverter { typeHandlerMap.put(Timestamp.class, TimestampType.INSTANCE); typeHandlerMap.put(Date.class, DateType.INSTANCE); + typeHandlerMap.put(Time.class, TimeType.INSTANCE); } diff --git a/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/ElasticsearchType.java b/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/ElasticsearchType.java index 6f754ca094..8bbca3e710 100644 --- a/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/ElasticsearchType.java +++ b/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/ElasticsearchType.java @@ -16,7 +16,9 @@ package com.amazon.opendistroforelasticsearch.jdbc.types; +import java.sql.Date; import java.sql.JDBCType; +import java.sql.Time; import java.sql.Timestamp; import java.util.HashMap; import java.util.Locale; @@ -71,6 +73,8 @@ public enum ElasticsearchType { NESTED(JDBCType.STRUCT, null, 0, 0, false), OBJECT(JDBCType.STRUCT, null, 0, 0, false), DATE(JDBCType.TIMESTAMP, Timestamp.class, 24, 24, false), + TIME(JDBCType.TIME, Time.class, 24, 24, false), + TIMESTAMP(JDBCType.TIMESTAMP, Timestamp.class, 24, 24, false), NULL(JDBCType.NULL, null, 0, 0, false), UNSUPPORTED(JDBCType.OTHER, null, 0, 0, false); @@ -89,7 +93,8 @@ public enum ElasticsearchType { jdbcTypeToESTypeMap.put(JDBCType.REAL, FLOAT); jdbcTypeToESTypeMap.put(JDBCType.FLOAT, DOUBLE); jdbcTypeToESTypeMap.put(JDBCType.VARCHAR, KEYWORD); - jdbcTypeToESTypeMap.put(JDBCType.TIMESTAMP, DATE); + jdbcTypeToESTypeMap.put(JDBCType.TIMESTAMP, TIMESTAMP); + jdbcTypeToESTypeMap.put(JDBCType.TIME, TIME); jdbcTypeToESTypeMap.put(JDBCType.DATE, DATE); } diff --git a/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/TimeType.java b/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/TimeType.java new file mode 100644 index 0000000000..6546e5d168 --- /dev/null +++ b/sql-jdbc/src/main/java/com/amazon/opendistroforelasticsearch/jdbc/types/TimeType.java @@ -0,0 +1,73 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazon.opendistroforelasticsearch.jdbc.types; + +import java.sql.SQLException; +import java.sql.Time; +import java.time.LocalTime; +import java.util.Map; + +public class TimeType implements TypeHelper