From 30703ea7ac9c6027218166980d448095ff8f53c8 Mon Sep 17 00:00:00 2001 From: Peng Huo Date: Mon, 18 May 2020 16:46:52 -0700 Subject: [PATCH] Add where stats rename command (#466) * add aggregator * update * update * add sum aggregator * update license header * update bindingtuple * address comments --- .../sql/analysis/Analyzer.java | 55 +++++- .../sql/analysis/ExpressionAnalyzer.java | 32 ++++ .../sql/ast/AbstractNodeVisitor.java | 5 + .../sql/ast/dsl/AstDSL.java | 7 +- .../sql/ast/expression/AggregateFunction.java | 10 +- .../sql/ast/tree/Rename.java | 58 +++++++ .../sql/data/model/ExprTupleValue.java | 42 ++++- .../sql/data/model/ExprValue.java | 9 + .../sql/data/model/ExprValueUtils.java | 7 +- .../sql/expression/DSL.java | 34 ++-- .../sql/expression/FunctionExpression.java | 3 +- .../aggregation/AggregationState.java | 29 ++++ .../expression/aggregation/Aggregator.java | 70 ++++++++ .../aggregation/AggregatorFunction.java | 73 ++++++++ .../expression/aggregation/AvgAggregator.java | 82 +++++++++ .../expression/aggregation/SumAggregator.java | 114 ++++++++++++ .../expression/config/ExpressionConfig.java | 7 +- .../function/BuiltinFunctionName.java | 6 + .../function/BuiltinFunctionRepository.java | 13 +- ...ssionBuilder.java => FunctionBuilder.java} | 12 +- .../function/FunctionImplementation.java | 36 ++++ .../expression/function/FunctionResolver.java | 12 +- .../sql/expression/scalar/OperatorUtils.java | 28 +-- .../scalar/arthmetic/ArithmeticFunction.java | 6 +- .../predicate/BinaryPredicateFunction.java | 20 +-- .../predicate/UnaryPredicateFunction.java | 4 +- .../sql/planner/PlanNode.java | 11 -- .../planner/logical/LogicalAggregation.java | 50 ++++++ .../sql/planner/logical/LogicalFilter.java | 7 +- .../sql/planner/logical/LogicalPlan.java | 10 ++ .../sql/planner/logical/LogicalPlanDSL.java | 18 +- .../LogicalPlanNodeVisitor.java} | 19 +- .../sql/planner/logical/LogicalRelation.java | 3 +- .../sql/planner/logical/LogicalRename.java | 48 ++++++ .../planner/physical/AggregationOperator.java | 163 ++++++++++++++++++ .../sql/planner/physical/FilterOperator.java | 55 ++++++ .../sql/planner/physical/PhysicalPlan.java | 46 +++++ .../sql/planner/physical/PhysicalPlanDSL.java | 43 +++++ .../physical/PhysicalPlanNodeVisitor.java | 41 +++++ .../sql/planner/physical/RenameOperator.java | 80 +++++++++ .../storage/bindingtuple/BindingTuple.java | 53 ++++++ .../bindingtuple/LazyBindingTuple.java | 34 ++++ .../sql/utils/ExpressionUtils.java | 36 ++++ .../sql/analysis/AnalyzerTest.java | 93 +++++++++- .../sql/analysis/AnalyzerTestBase.java | 5 +- .../sql/analysis/ExpressionAnalyzerTest.java | 8 + .../sql/data/model/ExprTupleValueTest.java | 45 +++++ .../sql/data/model/ExprValueUtilsTest.java | 13 ++ .../aggregation/AggregationTest.java | 66 +++++++ .../aggregation/AvgAggregatorTest.java | 61 +++++++ .../aggregation/SumAggregatorTest.java | 94 ++++++++++ .../BuiltinFunctionRepositoryTest.java | 2 +- .../function/FunctionResolverTest.java | 11 +- .../planner/AbstractPlanNodeVisitorTest.java | 73 -------- .../sql/planner/PlannerTest.java | 110 ++++++++++++ .../logical/LogicalPlanNodeVisitorTest.java | 111 ++++++++++++ .../physical/AggregationOperatorTest.java | 71 ++++++++ .../planner/physical/FilterOperatorTest.java | 40 +++++ .../physical/PhysicalPlanNodeVisitorTest.java | 125 ++++++++++++++ .../physical/PhysicalPlanTestBase.java | 111 ++++++++++++ .../planner/physical/RenameOperatorTest.java | 67 +++++++ .../bindingtuple/BindingTupleTest.java | 52 ++++++ .../sql/utils/MatcherUtils.java | 53 ++++++ ppl/src/main/antlr/OpenDistroPPLParser.g4 | 4 +- .../sql/ppl/parser/AstBuilder.java | 38 ++-- .../sql/ppl/parser/AstExpressionBuilder.java | 2 +- .../sql/ppl/parser/AstBuilderTest.java | 29 ++-- .../ppl/parser/AstExpressionBuilderTest.java | 17 +- .../sql/ppl/utils/ArgumentFactoryTest.java | 3 +- 69 files changed, 2566 insertions(+), 229 deletions(-) create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Rename.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationState.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/Aggregator.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregatorFunction.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregator.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregator.java rename core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/{FunctionExpressionBuilder.java => FunctionBuilder.java} (67%) create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionImplementation.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalAggregation.java rename core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/{AbstractPlanNodeVisitor.java => logical/LogicalPlanNodeVisitor.java} (71%) create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRename.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperator.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperator.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlan.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanDSL.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitor.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperator.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTuple.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/LazyBindingTuple.java create mode 100644 core/src/main/java/com/amazon/opendistroforelasticsearch/sql/utils/ExpressionUtils.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValueTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregatorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregatorTest.java delete mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/PlannerTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperatorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperatorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanTestBase.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTupleTest.java create mode 100644 core/src/test/java/com/amazon/opendistroforelasticsearch/sql/utils/MatcherUtils.java 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 ff9211e357..ad897358ab 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 @@ -16,16 +16,27 @@ package com.amazon.opendistroforelasticsearch.sql.analysis; import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.Field; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Aggregation; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename; 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.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +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; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRename; import com.amazon.opendistroforelasticsearch.sql.storage.StorageEngine; import com.amazon.opendistroforelasticsearch.sql.storage.Table; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import lombok.RequiredArgsConstructor; /** @@ -53,6 +64,48 @@ public LogicalPlan visitRelation(Relation node, AnalysisContext context) { public LogicalPlan visitFilter(Filter node, AnalysisContext context) { LogicalPlan child = node.getChild().get(0).accept(this, context); Expression condition = expressionAnalyzer.analyze(node.getCondition(), context); - return new LogicalFilter(condition, child); + return new LogicalFilter(child, condition); + } + + /** + * Build {@link LogicalRename} + */ + @Override + public LogicalPlan visitRename(Rename node, AnalysisContext context) { + LogicalPlan child = node.getChild().get(0).accept(this, context); + ImmutableMap.Builder renameMapBuilder = new ImmutableMap.Builder<>(); + for (com.amazon.opendistroforelasticsearch.sql.ast.expression.Map renameMap : node.getRenameList()) { + Expression origin = expressionAnalyzer.analyze(renameMap.getOrigin(), context); + // We should define the new target field in the context instead of analyze it. + if (renameMap.getTarget() instanceof Field) { + ReferenceExpression target = + new ReferenceExpression(((Field) renameMap.getTarget()).getField().toString()); + context.peek().define(target, origin.type(context.peek())); + renameMapBuilder.put(DSL.ref(origin.toString()), target); + } else { + throw new SemanticCheckException(String.format("the target expected to be field, but is %s", + renameMap.getTarget())); + } + } + + return new LogicalRename(child, renameMapBuilder.build()); + } + + /** + * Build {@link LogicalAggregation} + */ + @Override + public LogicalPlan visitAggregation(Aggregation node, AnalysisContext context) { + LogicalPlan child = node.getChild().get(0).accept(this, context); + ImmutableList.Builder aggregatorBuilder = new ImmutableList.Builder<>(); + for (UnresolvedExpression uExpr : node.getAggExprList()) { + aggregatorBuilder.add((Aggregator) expressionAnalyzer.analyze(uExpr, context)); + } + + ImmutableList.Builder groupbyBuilder = new ImmutableList.Builder<>(); + for (UnresolvedExpression uExpr : node.getGroupExprList()) { + groupbyBuilder.add(expressionAnalyzer.analyze(uExpr, context)); + } + return new LogicalAggregation(child, aggregatorBuilder.build(), groupbyBuilder.build()); } } 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 7d369bde94..9f834a1c0e 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 @@ -16,23 +16,33 @@ package com.amazon.opendistroforelasticsearch.sql.analysis; import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.AggregateFunction; import com.amazon.opendistroforelasticsearch.sql.ast.expression.And; import com.amazon.opendistroforelasticsearch.sql.ast.expression.EqualTo; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.Field; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Literal; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedAttribute; import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.exception.SemanticCheckException; 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; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; import lombok.RequiredArgsConstructor; +import java.util.Collections; +import java.util.Optional; + /** * Analyze the {@link UnresolvedExpression} in the {@link AnalysisContext} to construct the {@link Expression} */ @RequiredArgsConstructor public class ExpressionAnalyzer extends AbstractNodeVisitor { private final DSL dsl; + private final BuiltinFunctionRepository repository; public Expression analyze(UnresolvedExpression unresolved, AnalysisContext context) { return unresolved.accept(this, context); @@ -66,4 +76,26 @@ public Expression visitAnd(And node, AnalysisContext context) { return dsl.and(context.peek(), left, right); } + + @Override + public Expression visitAggregateFunction(AggregateFunction node, AnalysisContext context) { + Optional builtinFunctionName = BuiltinFunctionName.of(node.getFuncName()); + if (builtinFunctionName.isPresent()) { + Expression arg = node.getField().accept(this, context); + return (Aggregator) repository.compile(builtinFunctionName.get().getName(), + Collections.singletonList(arg), + context.peek()); + } else { + throw new SemanticCheckException("Unsupported aggregation function " + node.getFuncName()); + } + } + + @Override + public Expression visitField(Field node, AnalysisContext context) { + String attr = node.getField().toString(); + TypeEnvironment typeEnv = context.peek(); + ReferenceExpression ref = DSL.ref(attr); + typeEnv.resolve(ref); + return ref; + } } 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 67482edee2..685469c255 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 @@ -34,6 +34,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename; /** * AST nodes visitor @@ -138,5 +139,9 @@ public T visitField(Field node, C context) { public T visitQualifiedName(QualifiedName node, C context) { return visitChildren(node, context); } + + public T visitRename(Rename 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 c68f313ab7..f77f215c9b 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.UnresolvedAttribute; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Aggregation; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename; import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation; @@ -65,6 +66,10 @@ public static UnresolvedPlan agg(UnresolvedPlan input, List getChild() { - return Arrays.asList(field); + return Collections.singletonList(field); } @Override public R accept(AbstractNodeVisitor nodeVisitor, C context) { return nodeVisitor.visitAggregateFunction(this, context); } + + @Override + public String toString() { + return String.format("%s(%s)", funcName, field); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Rename.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Rename.java new file mode 100644 index 0000000000..b6745b2459 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/ast/tree/Rename.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 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.Map; +import com.google.common.collect.ImmutableList; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; + +import java.util.List; + +@ToString +@EqualsAndHashCode(callSuper = false) +@Getter +public class Rename extends UnresolvedPlan { + private final List renameList; + private UnresolvedPlan child; + + public Rename(List renameList, UnresolvedPlan child) { + this.renameList = renameList; + this.child = child; + } + + @Override + public Rename attach(UnresolvedPlan child) { + if (null == this.child) { + this.child = child; + } else { + this.child.attach(child); + } + return this; + } + + @Override + public List getChild() { + return ImmutableList.of(child); + } + + @Override + public T accept(AbstractNodeVisitor nodeVisitor, C context) { + return nodeVisitor.visitRename(this, context); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValue.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValue.java index 54a7188329..5be8e7b649 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValue.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValue.java @@ -15,16 +15,24 @@ package com.amazon.opendistroforelasticsearch.sql.data.model; -import lombok.EqualsAndHashCode; -import lombok.RequiredArgsConstructor; - +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.LazyBindingTuple; +import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.Map; +import java.util.Map.Entry; import java.util.stream.Collectors; +import lombok.RequiredArgsConstructor; -@EqualsAndHashCode @RequiredArgsConstructor public class ExprTupleValue implements ExprValue { - private final Map valueMap; + + private final LinkedHashMap valueMap; + + public static ExprTupleValue fromExprValueMap(Map map) { + LinkedHashMap linkedHashMap = new LinkedHashMap<>(map); + return new ExprTupleValue(linkedHashMap); + } @Override public Object value() { @@ -43,4 +51,28 @@ public String toString() { .map(entry -> String.format("%s:%s", entry.getKey(), entry.getValue())) .collect(Collectors.joining(",", "{", "}")); } + + @Override + public BindingTuple bindingTuples() { + return new LazyBindingTuple(bindingName -> valueMap.getOrDefault(bindingName, ExprMissingValue.of())); + } + + public boolean equals(Object o) { + if (o == this) { + return true; + } else if (!(o instanceof ExprTupleValue)) { + return false; + } else { + ExprTupleValue other = (ExprTupleValue) o; + Iterator> thisIterator = this.valueMap.entrySet().iterator(); + Iterator> otherIterator = other.valueMap.entrySet().iterator(); + while (thisIterator.hasNext() && otherIterator.hasNext()) { + if (!thisIterator.next().equals(otherIterator.next())) { + return false; + } + } + return !(thisIterator.hasNext() || otherIterator.hasNext()); + } + } + } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValue.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValue.java index 2c5953540c..dfb8e01a49 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValue.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValue.java @@ -15,6 +15,8 @@ package com.amazon.opendistroforelasticsearch.sql.data.model; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; + /** * The definition of the Expression Value. */ @@ -44,4 +46,11 @@ default boolean isNull() { default boolean isMissing() { return false; } + + /** + * Get the {@link BindingTuple} + */ + default BindingTuple bindingTuples() { + return BindingTuple.EMPTY; + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtils.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtils.java index b4c4d4e12c..fb4ecdc9cd 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtils.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtils.java @@ -17,10 +17,10 @@ import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; import com.google.common.annotations.VisibleForTesting; +import java.util.LinkedHashMap; import lombok.experimental.UtilityClass; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -64,7 +64,7 @@ public static ExprValue stringValue(String value) { } public static ExprValue tupleValue(Map map) { - Map valueMap = new HashMap<>(); + LinkedHashMap valueMap = new LinkedHashMap<>(); map.forEach((k, v) -> valueMap.put(k, fromObjectValue(v))); return new ExprTupleValue(valueMap); } @@ -84,6 +84,9 @@ public static ExprValue nullValue() { } public static ExprValue fromObjectValue(Object o) { + if ( null == o) { + return LITERAL_NULL; + } if (o instanceof Map) { return tupleValue((Map) o); } else if (o instanceof List) { 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 318f52dd2a..67a3f943fe 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 @@ -17,6 +17,8 @@ import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; @@ -28,6 +30,10 @@ public class DSL { private final BuiltinFunctionRepository repository; + public static LiteralExpression literal(Integer value) { + return new LiteralExpression(ExprValueUtils.integerValue(value)); + } + public static LiteralExpression literal(ExprValue value) { return new LiteralExpression(value); } @@ -37,42 +43,50 @@ public static ReferenceExpression ref(String ref) { } public FunctionExpression add(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.ADD.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.ADD.getName(), Arrays.asList(expressions), env); } public FunctionExpression subtract(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.SUBTRACT.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.SUBTRACT.getName(), Arrays.asList(expressions), env); } public FunctionExpression multiply(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.MULTIPLY.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.MULTIPLY.getName(), Arrays.asList(expressions), env); } public FunctionExpression divide(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.DIVIDE.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.DIVIDE.getName(), Arrays.asList(expressions), env); } public FunctionExpression module(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.MODULES.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.MODULES.getName(), Arrays.asList(expressions), env); } public FunctionExpression and(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.AND.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.AND.getName(), Arrays.asList(expressions), env); } public FunctionExpression or(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.OR.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.OR.getName(), Arrays.asList(expressions), env); } public FunctionExpression xor(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.XOR.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.XOR.getName(), Arrays.asList(expressions), env); } public FunctionExpression not(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.NOT.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.NOT.getName(), Arrays.asList(expressions), env); } public FunctionExpression equal(Environment env, Expression... expressions) { - return repository.compile(BuiltinFunctionName.EQUAL.getName(), Arrays.asList(expressions), env); + return (FunctionExpression) repository.compile(BuiltinFunctionName.EQUAL.getName(), Arrays.asList(expressions), env); + } + + public Aggregator avg(Environment env, Expression... expressions) { + return (Aggregator) repository.compile(BuiltinFunctionName.AVG.getName(), Arrays.asList(expressions), env); + } + + public Aggregator sum(Environment env, Expression... expressions) { + return (Aggregator) repository.compile(BuiltinFunctionName.SUM.getName(), Arrays.asList(expressions), env); } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/FunctionExpression.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/FunctionExpression.java index e4f0b37efd..e43a82192b 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/FunctionExpression.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/FunctionExpression.java @@ -15,6 +15,7 @@ package com.amazon.opendistroforelasticsearch.sql.expression; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionImplementation; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; import lombok.EqualsAndHashCode; import lombok.Getter; @@ -27,7 +28,7 @@ */ @EqualsAndHashCode @RequiredArgsConstructor -public abstract class FunctionExpression implements Expression { +public abstract class FunctionExpression implements Expression, FunctionImplementation { @Getter private final FunctionName functionName; diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationState.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationState.java new file mode 100644 index 0000000000..7a0a65c7d7 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationState.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 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.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; + +/** + * Maintain the state when {@link Aggregator} iterate on the {@link BindingTuple}. + */ +public interface AggregationState { + /** + * Get {@link ExprValue} result. + */ + ExprValue result(); +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/Aggregator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/Aggregator.java new file mode 100644 index 0000000000..79f80fe0fe --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/Aggregator.java @@ -0,0 +1,70 @@ +/* + * Copyright 2019 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.analysis.ExpressionAnalyzer; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionImplementation; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +import java.util.List; + +/** + * Aggregator which will iterate on the {@link BindingTuple}s to aggregate the result. + * The Aggregator is not well fit into Expression, because it has side effect. + * But we still want to make it implement {@link Expression} interface to make {@link ExpressionAnalyzer} easier. + */ +@EqualsAndHashCode +@RequiredArgsConstructor +public abstract class Aggregator implements FunctionImplementation, Expression { + @Getter + private final FunctionName functionName; + @Getter + private final List arguments; + protected final ExprType returnType; + + /** + * Create an {@link AggregationState} which will be used for aggregation + */ + public abstract S create(); + + /** + * Iterate on the {@link BindingTuple}. + * + * @param tuple {@link BindingTuple} + * @param state {@link AggregationState} + * @return {@link AggregationState} + */ + public abstract S iterate(BindingTuple tuple, S state); + + @Override + public ExprValue valueOf(Environment valueEnv) { + throw new ExpressionEvaluationException(String.format("can't evaluate on aggregator: %s", functionName)); + } + + @Override + public ExprType type(Environment typeEnv) { + return returnType; + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregatorFunction.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregatorFunction.java new file mode 100644 index 0000000000..ffc06c5a59 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregatorFunction.java @@ -0,0 +1,73 @@ +/* + * Copyright 2019 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.data.model.ExprType; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionResolver; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionSignature; +import com.google.common.collect.ImmutableMap; +import lombok.experimental.UtilityClass; + +import java.util.Collections; + +/** + * The definition of aggregator function + * avg, Accepts two numbers and produces a number. + * sum, Accepts two numbers and produces a number. + * max, Accepts two numbers and produces a number. + * min, Accepts two numbers and produces a number. + * count, Accepts two numbers and produces a number. + */ +@UtilityClass +public class AggregatorFunction { + + public static void register(BuiltinFunctionRepository repository) { + repository.register(avg()); + repository.register(sum()); + } + + private static FunctionResolver avg() { + FunctionName functionName = BuiltinFunctionName.AVG.getName(); + return new FunctionResolver( + functionName, + new ImmutableMap.Builder() + .put(new FunctionSignature(functionName, Collections.singletonList(ExprType.DOUBLE)), + arguments -> new AvgAggregator(arguments, ExprType.DOUBLE)) + .build() + ); + } + + private static FunctionResolver sum() { + FunctionName functionName = BuiltinFunctionName.SUM.getName(); + return new FunctionResolver( + functionName, + new ImmutableMap.Builder() + .put(new FunctionSignature(functionName, Collections.singletonList(ExprType.INTEGER)), + arguments -> new SumAggregator(arguments, ExprType.INTEGER)) + .put(new FunctionSignature(functionName, Collections.singletonList(ExprType.LONG)), + arguments -> new SumAggregator(arguments, ExprType.LONG)) + .put(new FunctionSignature(functionName, Collections.singletonList(ExprType.FLOAT)), + arguments -> new SumAggregator(arguments, ExprType.FLOAT)) + .put(new FunctionSignature(functionName, Collections.singletonList(ExprType.DOUBLE)), + arguments -> new SumAggregator(arguments, ExprType.DOUBLE)) + .build() + ); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregator.java new file mode 100644 index 0000000000..b7c95568a4 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregator.java @@ -0,0 +1,82 @@ +/* + * Copyright 2019 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.data.model.ExprNullValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; + +import java.util.List; +import java.util.Locale; + +import static com.amazon.opendistroforelasticsearch.sql.utils.ExpressionUtils.format; + +/** + * The average aggregator aggregate the value evaluated by the expression. + * If the expression evaluated result is NULL or MISSING, then the result is NULL. + */ +public class AvgAggregator extends Aggregator { + + public AvgAggregator(List arguments, ExprType returnType) { + super(BuiltinFunctionName.AVG.getName(), arguments, returnType); + } + + @Override + public AvgState create() { + return new AvgState(); + } + + @Override + public AvgState iterate(BindingTuple tuple, AvgState state) { + Expression expression = getArguments().get(0); + ExprValue value = expression.valueOf(tuple); + if (value.isNull() || value.isMissing()) { + state.isNullResult = true; + } else { + state.count++; + state.total += ExprValueUtils.getDoubleValue(value); + } + return state; + } + + @Override + public String toString() { + return String.format(Locale.ROOT, "avg(%s)", format(getArguments())); + } + + /** + * Average State. + */ + protected class AvgState implements AggregationState { + private int count; + private double total; + private boolean isNullResult = false; + + public AvgState() { + this.count = 0; + this.total = 0d; + } + + @Override + public ExprValue result() { + return isNullResult ? ExprNullValue.of() : ExprValueUtils.doubleValue(total / count); + } + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregator.java new file mode 100644 index 0000000000..00212cafb6 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregator.java @@ -0,0 +1,114 @@ +/* + * Copyright 2019 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 static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.doubleValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.floatValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.getDoubleValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.getFloatValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.getIntegerValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.getLongValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.integerValue; +import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.longValue; +import static com.amazon.opendistroforelasticsearch.sql.utils.ExpressionUtils.format; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprNullValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.SumAggregator.SumState; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import java.util.List; +import java.util.Locale; + +/** + * The sum aggregator aggregate the value evaluated by the expression. + * If the expression evaluated result is NULL or MISSING, then the result is NULL. + */ +public class SumAggregator extends Aggregator { + + public SumAggregator(List arguments, ExprType returnType) { + super(BuiltinFunctionName.SUM.getName(), arguments, returnType); + } + + @Override + public SumState create() { + return new SumState(returnType); + } + + @Override + public SumState iterate(BindingTuple tuple, SumState state) { + Expression expression = getArguments().get(0); + ExprValue value = expression.valueOf(tuple); + if (value.isNull() || value.isMissing()) { + state.isNullResult = true; + } else { + state.add(value); + } + return state; + } + + @Override + public String toString() { + return String.format(Locale.ROOT, "sum(%s)", format(getArguments())); + } + + /** + * Sum State. + */ + protected class SumState implements AggregationState { + + private final ExprType type; + private ExprValue sumResult; + private boolean isNullResult = false; + + public SumState(ExprType type) { + this.type = type; + sumResult = ExprValueUtils.integerValue(0); + } + + /** + * Add value to current sumResult + */ + public void add(ExprValue value) { + switch (type) { + case INTEGER: + sumResult = integerValue(getIntegerValue(sumResult) + getIntegerValue(value)); + break; + case LONG: + sumResult = longValue(getLongValue(sumResult) + getLongValue(value)); + break; + case FLOAT: + sumResult = floatValue(getFloatValue(sumResult) + getFloatValue(value)); + break; + case DOUBLE: + sumResult = doubleValue(getDoubleValue(sumResult) + getDoubleValue(value)); + break; + default: + throw new ExpressionEvaluationException( + String.format("unexpected type [%s] in sum aggregation", type)); + } + } + + @Override + public ExprValue result() { + return isNullResult ? ExprNullValue.of() : sumResult; + } + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/config/ExpressionConfig.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/config/ExpressionConfig.java index 18a588c5c7..015147cb5f 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/config/ExpressionConfig.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/config/ExpressionConfig.java @@ -15,10 +15,8 @@ package com.amazon.opendistroforelasticsearch.sql.expression.config; -import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; import com.amazon.opendistroforelasticsearch.sql.expression.DSL; -import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AggregatorFunction; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; import com.amazon.opendistroforelasticsearch.sql.expression.scalar.arthmetic.ArithmeticFunction; import com.amazon.opendistroforelasticsearch.sql.expression.scalar.predicate.BinaryPredicateFunction; @@ -34,11 +32,12 @@ @Configuration public class ExpressionConfig { @Bean - public BuiltinFunctionRepository functionRepository(Environment typeEnv) { + public BuiltinFunctionRepository functionRepository() { BuiltinFunctionRepository builtinFunctionRepository = new BuiltinFunctionRepository(new HashMap<>()); ArithmeticFunction.register(builtinFunctionRepository); BinaryPredicateFunction.register(builtinFunctionRepository); UnaryPredicateFunction.register(builtinFunctionRepository); + AggregatorFunction.register(builtinFunctionRepository); return builtinFunctionRepository; } 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 2606fa857e..5a4182ef1a 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 @@ -27,6 +27,12 @@ public enum BuiltinFunctionName { EQUAL(FunctionName.of("equal")), TOSTRING(FunctionName.of("tostring")), + + /** + * Aggregation Function. + */ + AVG(FunctionName.of("avg")), + SUM(FunctionName.of("sum")) ; private final FunctionName name; diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepository.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepository.java index 00ce1a8dae..d6ffe863a9 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepository.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepository.java @@ -4,7 +4,6 @@ import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.expression.FunctionExpression; import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; import lombok.RequiredArgsConstructor; @@ -31,20 +30,20 @@ public void register(FunctionResolver resolver) { /** * Compile FunctionExpression */ - public FunctionExpression compile(FunctionName functionName, List expressions, - Environment env) { - FunctionExpressionBuilder resolvedFunctionBuilder = resolve(new FunctionSignature(functionName, + public FunctionImplementation compile(FunctionName functionName, List expressions, + Environment env) { + FunctionBuilder resolvedFunctionBuilder = resolve(new FunctionSignature(functionName, expressions.stream().map(expression -> expression.type(env)).collect(Collectors.toList()))); return resolvedFunctionBuilder.apply(expressions); } /** - * Resolve the {@link FunctionExpressionBuilder} in Builtin Function Repository. + * Resolve the {@link FunctionBuilder} in Builtin Function Repository. * * @param functionSignature {@link FunctionSignature} - * @return {@link FunctionExpressionBuilder} + * @return {@link FunctionBuilder} */ - public FunctionExpressionBuilder resolve(FunctionSignature functionSignature) { + public FunctionBuilder resolve(FunctionSignature functionSignature) { FunctionName functionName = functionSignature.getFunctionName(); if (functionResolverMap.containsKey(functionName)) { return functionResolverMap.get(functionName).resolve(functionSignature); diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionExpressionBuilder.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionBuilder.java similarity index 67% rename from core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionExpressionBuilder.java rename to core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionBuilder.java index e8bb3d8e0e..30a889b28e 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionExpressionBuilder.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionBuilder.java @@ -16,19 +16,19 @@ package com.amazon.opendistroforelasticsearch.sql.expression.function; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.expression.FunctionExpression; import java.util.List; /** - * The definition of function which create {@link FunctionExpression} from input {@link Expression} list. + * The definition of function which create {@link FunctionImplementation} from input {@link Expression} list. */ -public interface FunctionExpressionBuilder { +public interface FunctionBuilder { /** - * Create {@link FunctionExpression} from input {@link Expression} list + * Create {@link FunctionImplementation} from input {@link Expression} list + * * @param arguments {@link Expression} list - * @return {@link FunctionExpression} + * @return {@link FunctionImplementation} */ - FunctionExpression apply(List arguments); + FunctionImplementation apply(List arguments); } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionImplementation.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionImplementation.java new file mode 100644 index 0000000000..852b0cc364 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionImplementation.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 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.function; + +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; + +import java.util.List; + +/** + * The definition of Function Implementation. + */ +public interface FunctionImplementation { + + /** + * Get Function Name. + */ + FunctionName getFunctionName(); + + /** + * Get Function Arguments. + */ + List getArguments(); +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolver.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolver.java index d250cad0eb..0727d9ddea 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolver.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolver.java @@ -13,9 +13,9 @@ import java.util.stream.Collectors; /** - * The Function Resolver hold the overload {@link FunctionExpressionBuilder} implementation. + * The Function Resolver hold the overload {@link FunctionBuilder} implementation. * is composed by {@link FunctionName} which identified the function name - * and a map of {@link FunctionSignature} and {@link FunctionExpressionBuilder} + * and a map of {@link FunctionSignature} and {@link FunctionBuilder} * to represent the overloaded implementation */ @Builder @@ -24,15 +24,15 @@ public class FunctionResolver { @Getter private final FunctionName functionName; @Singular("functionBundle") - private final Map functionBundle; + private final Map functionBundle; /** - * Resolve the {@link FunctionExpressionBuilder} by using input {@link FunctionSignature}. - * If found the {@link FunctionExpressionBuilder} exactly match the input {@link FunctionSignature}, return it. + * Resolve the {@link FunctionBuilder} by using input {@link FunctionSignature}. + * If found the {@link FunctionBuilder} exactly match the input {@link FunctionSignature}, return it. * If applying the widening rule, found the most match one, return it. * If nothing found, throw {@link ExpressionEvaluationException} */ - public FunctionExpressionBuilder resolve(FunctionSignature unresolvedSignature) { + public FunctionBuilder resolve(FunctionSignature unresolvedSignature) { PriorityQueue> functionMatchQueue = new PriorityQueue<>( Map.Entry.comparingByKey()); diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/OperatorUtils.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/OperatorUtils.java index ab00173fad..279a92b363 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/OperatorUtils.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/OperatorUtils.java @@ -21,7 +21,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.expression.FunctionExpression; import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; -import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionExpressionBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionBuilder; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; import lombok.experimental.UtilityClass; @@ -31,7 +31,7 @@ @UtilityClass public class OperatorUtils { /** - * Construct {@link FunctionExpressionBuilder} which call function with arguments produced by observer. + * Construct {@link FunctionBuilder} which call function with arguments produced by observer. * * @param functionName function name * @param function {@link BiFunction} @@ -39,17 +39,17 @@ public class OperatorUtils { * @param returnType return type * @param the type of the first and second argument to the function * @param the type of the result of the function - * @return {@link FunctionExpressionBuilder} + * @return {@link FunctionBuilder} */ - public static FunctionExpressionBuilder binaryOperator(FunctionName functionName, - BiFunction function, - Function observer, - ExprType returnType) { + public static FunctionBuilder binaryOperator(FunctionName functionName, + BiFunction function, + Function observer, + ExprType returnType) { return binaryOperator(functionName, function, observer, observer, returnType); } /** - * Construct {@link FunctionExpressionBuilder} which + * Construct {@link FunctionBuilder} which * call function with arguments produced by observer1 and observer2 * In general, if any operand evaluates to a MISSING value, the enclosing operator will return MISSING; * if none of operands evaluates to a MISSING value but there is an operand evaluates to a NULL value, @@ -63,13 +63,13 @@ public static FunctionExpressionBuilder binaryOperator(FunctionName funct * @param the type of the first argument to the function * @param the type of the second argument to the function * @param the type of the result of the function - * @return {@link FunctionExpressionBuilder} + * @return {@link FunctionBuilder} */ - public static FunctionExpressionBuilder binaryOperator(FunctionName functionName, - BiFunction function, - Function observer1, - Function observer2, - ExprType returnType) { + public static FunctionBuilder binaryOperator(FunctionName functionName, + BiFunction function, + Function observer1, + Function observer2, + ExprType returnType) { return arguments -> new FunctionExpression(functionName, arguments) { @Override public ExprValue valueOf(Environment env) { diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/arthmetic/ArithmeticFunction.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/arthmetic/ArithmeticFunction.java index cd2ec3a22e..c409b76fc0 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/arthmetic/ArithmeticFunction.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/arthmetic/ArithmeticFunction.java @@ -19,7 +19,7 @@ import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; -import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionExpressionBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionBuilder; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionResolver; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionSignature; @@ -107,13 +107,13 @@ private static FunctionResolver modules() { ); } - private static Map scalarFunction( + private static Map scalarFunction( FunctionName functionName, BiFunction integerFunc, BiFunction longFunc, BiFunction floatFunc, BiFunction doubleFunc) { - ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); builder.put(new FunctionSignature(functionName, Arrays.asList(ExprType.INTEGER, ExprType.INTEGER)), binaryOperator(functionName, integerFunc, ExprValueUtils::getIntegerValue, ExprType.INTEGER)); builder.put(new FunctionSignature(functionName, Arrays.asList(ExprType.LONG, ExprType.LONG)), diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/BinaryPredicateFunction.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/BinaryPredicateFunction.java index 718a151842..e1e962aad7 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/BinaryPredicateFunction.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/BinaryPredicateFunction.java @@ -23,7 +23,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; -import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionExpressionBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionBuilder; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionResolver; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionSignature; @@ -206,7 +206,7 @@ private static FunctionResolver equal() { ); } - private static Map predicateFunction( + private static Map predicateFunction( FunctionName functionName, BiFunction integerFunc, BiFunction longFunc, @@ -216,7 +216,7 @@ private static Map predicateFuncti BiFunction booleanFunc, BiFunction listFunc, BiFunction mapFunc) { - ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); builder.put(new FunctionSignature(functionName, Arrays.asList(ExprType.INTEGER, ExprType.INTEGER)), equal(functionName, integerFunc, ExprValueUtils::getIntegerValue, ExprType.BOOLEAN)); @@ -244,9 +244,9 @@ private static Map predicateFuncti return builder.build(); } - private static FunctionExpressionBuilder binaryPredicate(FunctionName functionName, - Table logicalTable, - ExprType returnType) { + private static FunctionBuilder binaryPredicate(FunctionName functionName, + Table logicalTable, + ExprType returnType) { return arguments -> new FunctionExpression(functionName, arguments) { @Override public ExprValue valueOf(Environment env) { @@ -266,10 +266,10 @@ public ExprType type(Environment env) { }; } - private static FunctionExpressionBuilder equal(FunctionName functionName, - BiFunction function, - Function observer, - ExprType returnType) { + private static FunctionBuilder equal(FunctionName functionName, + BiFunction function, + Function observer, + ExprType returnType) { return arguments -> new FunctionExpression(functionName, arguments) { @Override public ExprValue valueOf(Environment env) { diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/UnaryPredicateFunction.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/UnaryPredicateFunction.java index 450bfd4f85..d70f92b29f 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/UnaryPredicateFunction.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/scalar/predicate/UnaryPredicateFunction.java @@ -22,7 +22,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; -import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionExpressionBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionBuilder; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionName; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionResolver; import com.amazon.opendistroforelasticsearch.sql.expression.function.FunctionSignature; @@ -73,7 +73,7 @@ private static FunctionResolver not() { .build(); } - private static FunctionExpressionBuilder predicateFunction( + private static FunctionBuilder predicateFunction( FunctionName functionName, Map map, ExprType returnType) { diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/PlanNode.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/PlanNode.java index 80ccf9b285..2af2f381d8 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/PlanNode.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/PlanNode.java @@ -28,15 +28,4 @@ public interface PlanNode { * @return child nodes. */ List getChild(); - - /** - * Accept the visitor. - * - * @param visitor visitor. - * @param context visitor context. - * @param returned object type. - * @param context type. - * @return returned object. - */ - R accept(AbstractPlanNodeVisitor visitor, C context); } 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 new file mode 100644 index 0000000000..0e6c10cd71 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalAggregation.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 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 com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + +import java.util.Collections; +import java.util.List; + +/** + * Logical Aggregation. + */ +@ToString +@EqualsAndHashCode +@RequiredArgsConstructor +public class LogicalAggregation extends LogicalPlan { + private final LogicalPlan child; + @Getter + private final List aggregatorList; + @Getter + private final List groupByList; + + @Override + public List getChild() { + return Collections.singletonList(child); + } + + @Override + public R accept(LogicalPlanNodeVisitor visitor, C context) { + return visitor.visitAggregation(this, context); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalFilter.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalFilter.java index 3d807b9ce9..20d2a8476f 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalFilter.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalFilter.java @@ -16,8 +16,8 @@ package com.amazon.opendistroforelasticsearch.sql.planner.logical; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.planner.AbstractPlanNodeVisitor; import lombok.EqualsAndHashCode; +import lombok.Getter; import lombok.RequiredArgsConstructor; import lombok.ToString; @@ -31,8 +31,9 @@ @EqualsAndHashCode @RequiredArgsConstructor public class LogicalFilter extends LogicalPlan { - private final Expression condition; private final LogicalPlan child; + @Getter + private final Expression condition; @Override public List getChild() { @@ -40,7 +41,7 @@ public List getChild() { } @Override - public R accept(AbstractPlanNodeVisitor visitor, C context) { + public R accept(LogicalPlanNodeVisitor visitor, C context) { return visitor.visitFilter(this, context); } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlan.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlan.java index 2047ff8e03..5f1d60bed0 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlan.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlan.java @@ -21,4 +21,14 @@ * The abstract base class for all the Logical Plan node. */ public abstract class LogicalPlan implements PlanNode { + /** + * Accept the {@link LogicalPlanNodeVisitor}. + * + * @param visitor visitor. + * @param context visitor context. + * @param returned object type. + * @param context type. + * @return returned object. + */ + public abstract R accept(LogicalPlanNodeVisitor visitor, C 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 ea8c8135ce..458c4c3c74 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 @@ -16,18 +16,32 @@ package com.amazon.opendistroforelasticsearch.sql.planner.logical; import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; import lombok.experimental.UtilityClass; +import java.util.List; +import java.util.Map; + /** * Logical Plan DSL. */ @UtilityClass public class LogicalPlanDSL { - public static LogicalPlan filter(Expression expression, LogicalPlan input) { - return new LogicalFilter(expression, input); + public static LogicalPlan aggregation(LogicalPlan input, List aggregatorList, + List groupByList) { + return new LogicalAggregation(input, aggregatorList, groupByList); + } + + public static LogicalPlan filter(LogicalPlan input, Expression expression) { + return new LogicalFilter(input, expression); } public static LogicalPlan relation(String tableName) { return new LogicalRelation(tableName); } + + public static LogicalPlan rename(LogicalPlan input, Map renameMap) { + return new LogicalRename(input, renameMap); + } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitor.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java similarity index 71% rename from core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitor.java rename to core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java index 623149193e..26364bb903 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitor.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitor.java @@ -13,20 +13,15 @@ * permissions and limitations under the License. */ -package com.amazon.opendistroforelasticsearch.sql.planner; - -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalFilter; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation; +package com.amazon.opendistroforelasticsearch.sql.planner.logical; /** - * Abstract {@link PlanNode} Visitor. + * The visitor of {@link LogicalPlan}. * * @param return object type. * @param context type. */ -public abstract class AbstractPlanNodeVisitor { - +public abstract class LogicalPlanNodeVisitor { protected R visitNode(LogicalPlan plan, C context) { return null; } @@ -38,4 +33,12 @@ public R visitRelation(LogicalRelation plan, C context) { public R visitFilter(LogicalFilter plan, C context) { return visitNode(plan, context); } + + public R visitAggregation(LogicalAggregation 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/LogicalRelation.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRelation.java index 8351c3d858..3edcf5a60b 100644 --- a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRelation.java +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRelation.java @@ -15,7 +15,6 @@ package com.amazon.opendistroforelasticsearch.sql.planner.logical; -import com.amazon.opendistroforelasticsearch.sql.planner.AbstractPlanNodeVisitor; import com.google.common.collect.ImmutableList; import lombok.EqualsAndHashCode; import lombok.RequiredArgsConstructor; @@ -38,7 +37,7 @@ public List getChild() { } @Override - public R accept(AbstractPlanNodeVisitor visitor, C context) { + public R accept(LogicalPlanNodeVisitor visitor, C context) { return visitor.visitRelation(this, context); } } diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRename.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRename.java new file mode 100644 index 0000000000..14a8a26ba5 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalRename.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 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.ReferenceExpression; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + +/** + * Rename Operator. + * renameList is list of mapping of source and target. + */ +@ToString +@EqualsAndHashCode +@RequiredArgsConstructor +public class LogicalRename extends LogicalPlan { + private final LogicalPlan child; + @Getter + private final Map renameMap; + + @Override + public List getChild() { + return Collections.singletonList(child); + } + + @Override + public R accept(LogicalPlanNodeVisitor visitor, C context) { + return visitor.visitRename(this, context); + } +} 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 new file mode 100644 index 0000000000..b4525cfe9d --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperator.java @@ -0,0 +1,163 @@ +/* + * Copyright 2019 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.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.AggregationState; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + +/** + * Group the all the input {@link BindingTuple} by {@link AggregationOperator#groupByExprList}, calculate the + * aggregation result by using {@link AggregationOperator#aggregatorList} + */ +@EqualsAndHashCode +@ToString +public class AggregationOperator extends PhysicalPlan { + + private final PhysicalPlan input; + private final List aggregatorList; + private final List groupByExprList; + @EqualsAndHashCode.Exclude + private final Group group; + @EqualsAndHashCode.Exclude + private Iterator iterator; + + public AggregationOperator(PhysicalPlan input, List aggregatorList, + List groupByExprList) { + this.input = input; + this.aggregatorList = aggregatorList; + this.groupByExprList = groupByExprList; + this.group = new Group(); + } + + @Override + public R accept(PhysicalPlanNodeVisitor visitor, C context) { + return visitor.visitAggregation(this, context); + } + + @Override + public List getChild() { + return Collections.singletonList(input); + } + + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public ExprValue next() { + return iterator.next(); + } + + @Override + public void open() { + while (input.hasNext()) { + group.push(input.next()); + } + iterator = group.result().iterator(); + } + + @VisibleForTesting + @RequiredArgsConstructor + public class Group { + + private final Map>> groupListMap = new HashMap<>(); + + /** + * Push the BindingTuple to Group. Two functions will be applied to each BindingTuple to generate the + * {@link GroupKey} and {@link AggregationState} + * Key = GroupKey(bindingTuple), State = Aggregator(bindingTuple) + */ + public void push(ExprValue inputValue) { + GroupKey groupKey = new GroupKey(inputValue); + groupListMap.computeIfAbsent(groupKey, k -> + aggregatorList.stream() + .map(aggregator -> new AbstractMap.SimpleEntry<>(aggregator, + aggregator.create())) + .collect(Collectors.toList()) + ); + groupListMap.computeIfPresent(groupKey, (key, aggregatorList) -> { + aggregatorList + .forEach(entry -> entry.getKey().iterate(inputValue.bindingTuples(), entry.getValue())); + return aggregatorList; + }); + } + + /** + * Get the list of {@link BindingTuple} for each group. + */ + public List result() { + ImmutableList.Builder resultBuilder = new ImmutableList.Builder<>(); + 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()); + } + resultBuilder.add(ExprTupleValue.fromExprValueMap(map)); + } + return resultBuilder.build(); + } + } + + /** + * Group Key. + */ + @EqualsAndHashCode + @VisibleForTesting + public class GroupKey { + + private final List groupByValueList; + + public GroupKey(ExprValue value) { + this.groupByValueList = new ArrayList<>(); + for (Expression groupExpr : groupByExprList) { + this.groupByValueList.add(groupExpr.valueOf(value.bindingTuples())); + } + } + + /** + * Return the Map of group field and group field 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)); + } + return map; + } + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperator.java new file mode 100644 index 0000000000..0c24808074 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperator.java @@ -0,0 +1,55 @@ +package com.amazon.opendistroforelasticsearch.sql.planner.physical; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.scalar.predicate.BinaryPredicateFunction; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + +import java.util.Collections; +import java.util.List; + +/** + * The Filter operator use the conditions to evaluate the input {@link BindingTuple}. + * The Filter operator only return the results that evaluated to true. + * The NULL and MISSING are handled by the logic defined in {@link BinaryPredicateFunction}. + */ +@EqualsAndHashCode +@ToString +@RequiredArgsConstructor +public class FilterOperator extends PhysicalPlan { + private final PhysicalPlan input; + private final Expression conditions; + private ExprValue next = null; + + @Override + public R accept(PhysicalPlanNodeVisitor visitor, C context) { + return visitor.visitFilter(this, context); + } + + @Override + public List getChild() { + return Collections.singletonList(input); + } + + @Override + public boolean hasNext() { + while (input.hasNext()) { + ExprValue inputValue = input.next(); + ExprValue exprValue = conditions.valueOf(inputValue.bindingTuples()); + if (ExprValueUtils.getBooleanValue(exprValue)) { + next = inputValue; + return true; + } + } + return false; + } + + @Override + public ExprValue next() { + return next; + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlan.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlan.java new file mode 100644 index 0000000000..3fa9e09fcc --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlan.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 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.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.planner.PlanNode; +import java.util.Iterator; + +/** + * Physical plan + */ +public abstract class PhysicalPlan implements PlanNode, + Iterator, + AutoCloseable { + /** + * Accept the {@link PhysicalPlanNodeVisitor}. + * + * @param visitor visitor. + * @param context visitor context. + * @param returned object type. + * @param context type. + * @return returned object. + */ + public abstract R accept(PhysicalPlanNodeVisitor visitor, C context); + + public void open() { + getChild().forEach(PhysicalPlan::open); + } + + public void close() { + getChild().forEach(PhysicalPlan::close); + } +} 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 new file mode 100644 index 0000000000..9bbf604784 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanDSL.java @@ -0,0 +1,43 @@ +/* + * Copyright 2019 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.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import lombok.experimental.UtilityClass; + +import java.util.List; +import java.util.Map; + +/** + * Physical Plan DSL. + */ +@UtilityClass +public class PhysicalPlanDSL { + + public static AggregationOperator agg(PhysicalPlan input, List aggregators, List groups) { + return new AggregationOperator(input, aggregators, groups); + } + + public static FilterOperator filter(PhysicalPlan input, Expression condition) { + return new FilterOperator(input, condition); + } + + public static RenameOperator rename(PhysicalPlan input, Map renameMap) { + return new RenameOperator(input, renameMap); + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000..6e000d1b6a --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitor.java @@ -0,0 +1,41 @@ +/* + * Copyright 2019 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; + +/** + * The visitor of {@link PhysicalPlan}. + * + * @param return object type. + * @param context type. + */ +public abstract class PhysicalPlanNodeVisitor { + + protected R visitNode(PhysicalPlan node, C context) { + return null; + } + + public R visitFilter(FilterOperator node, C context) { + return visitNode(node, context); + } + + public R visitAggregation(AggregationOperator node, C context) { + return visitNode(node, context); + } + + public R visitRename(RenameOperator node, C context) { + return visitNode(node, context); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperator.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperator.java new file mode 100644 index 0000000000..8fb029e0c2 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperator.java @@ -0,0 +1,80 @@ +/* + * Copyright 2019 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.ExprType.STRUCT; + +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.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMap.Builder; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + +/** + * Rename the binding name in {@link BindingTuple}. + * The mapping maintain the relation between source and target. + * it means BindingTuple.resolve(target) = BindingTuple.resolve(source). + */ +@EqualsAndHashCode +@ToString +@RequiredArgsConstructor +public class RenameOperator extends PhysicalPlan { + private final PhysicalPlan input; + private final Map mapping; + + @Override + public R accept(PhysicalPlanNodeVisitor visitor, C context) { + return visitor.visitRename(this, context); + } + + @Override + public List getChild() { + return Collections.singletonList(input); + } + + @Override + public boolean hasNext() { + return input.hasNext(); + } + + @Override + public ExprValue next() { + ExprValue inputValue = input.next(); + if (STRUCT == inputValue.type()) { + Map tupleValue = ExprValueUtils.getTupleValue(inputValue); + ImmutableMap.Builder mapBuilder = new Builder<>(); + for (String bindName : tupleValue.keySet()) { + if (mapping.containsKey(DSL.ref(bindName))) { + mapBuilder.put(mapping.get(DSL.ref(bindName)).getAttr(), tupleValue.get(bindName)); + } else { + mapBuilder.put(bindName, tupleValue.get(bindName)); + } + } + return ExprTupleValue.fromExprValueMap(mapBuilder.build()); + } else { + return inputValue; + } + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTuple.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTuple.java new file mode 100644 index 0000000000..a102d82dad --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTuple.java @@ -0,0 +1,53 @@ +/* + * Copyright 2019 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.storage.bindingtuple; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprMissingValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; + +/** + * BindingTuple represents the a relationship between bindingName and ExprValue. + * e.g. The operation output column name is bindingName, the value is the ExprValue. + */ +public abstract class BindingTuple implements Environment { + public static BindingTuple EMPTY = new BindingTuple() { + @Override + public ExprValue resolve(ReferenceExpression ref) { + return ExprMissingValue.of(); + } + }; + + /** + * Resolve {@link Expression} in the BindingTuple environment. + */ + @Override + public ExprValue resolve(Expression var) { + if (var instanceof ReferenceExpression) { + return resolve(((ReferenceExpression) var)); + } else { + throw new ExpressionEvaluationException(String.format("can resolve expression: %s", var)); + } + } + + /** + * Resolve the {@link ReferenceExpression} in BindingTuple context. + */ + public abstract ExprValue resolve(ReferenceExpression ref); +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/LazyBindingTuple.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/LazyBindingTuple.java new file mode 100644 index 0000000000..4aa19293d4 --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/LazyBindingTuple.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 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.storage.bindingtuple; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import java.util.function.Function; +import lombok.RequiredArgsConstructor; + +/** + * Lazy Implementation of {@link BindingTuple}. + */ +@RequiredArgsConstructor +public class LazyBindingTuple extends BindingTuple { + private final Function lazyBinding; + + @Override + public ExprValue resolve(ReferenceExpression ref) { + return lazyBinding.apply(ref.getAttr()); + } +} diff --git a/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/utils/ExpressionUtils.java b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/utils/ExpressionUtils.java new file mode 100644 index 0000000000..313674adfc --- /dev/null +++ b/core/src/main/java/com/amazon/opendistroforelasticsearch/sql/utils/ExpressionUtils.java @@ -0,0 +1,36 @@ +/* + * Copyright 2019 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.utils; + +import com.amazon.opendistroforelasticsearch.sql.expression.Expression; +import lombok.experimental.UtilityClass; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Utils for {@link Expression}. + */ +@UtilityClass +public class ExpressionUtils { + + /** + * Format the list of {@link Expression} + */ + public static String format(List expressionList) { + return expressionList.stream().map(Expression::toString).collect(Collectors.joining(",")); + } +} 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 db0a9557bb..084788d5dc 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 @@ -17,30 +17,115 @@ import com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL; 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.planner.logical.LogicalPlan; import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.field; import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.integerValue; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; class AnalyzerTest extends AnalyzerTestBase { - @Test public void filter_relation() { assertAnalyzeEqual( LogicalPlanDSL.filter( - dsl.equal(typeEnv, DSL.ref("integer_value"), DSL.literal(integerValue(1))), - LogicalPlanDSL.relation("schema") + LogicalPlanDSL.relation("schema"), + dsl.equal(typeEnv, DSL.ref("integer_value"), DSL.literal(integerValue(1))) ), AstDSL.filter( AstDSL.relation("schema"), - AstDSL.equalTo(AstDSL.unresolvedAttr("integer_value"), AstDSL.intLiteral(1)) + AstDSL.equalTo(AstDSL.field("integer_value"), AstDSL.intLiteral(1)) + ) + ); + } + + + @Test + public void rename_relation() { + assertAnalyzeEqual( + LogicalPlanDSL.rename( + LogicalPlanDSL.relation("schema"), + ImmutableMap.of(DSL.ref("integer_value"), DSL.ref("ivalue")) + ), + AstDSL.rename( + AstDSL.relation("schema"), + 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(typeEnv, DSL.ref("integer_value"))), + ImmutableList.of() + ), + ImmutableMap.of(DSL.ref("avg(integer_value)"), DSL.ref("ivalue")) + ), + 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(typeEnv, DSL.ref("integer_value"))), + ImmutableList.of(DSL.ref("string_value")) + ), + AstDSL.agg( + AstDSL.relation("schema"), + AstDSL.exprList( + AstDSL.aggregate("avg", field("integer_value")) + ), + null, + ImmutableList.of(field("string_value")), + AstDSL.defaultStatsArgs() ) ); } + @Test + public void rename_to_invalid_expression() { + SemanticCheckException exception = assertThrows(SemanticCheckException.class, () -> analyze( + 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")), AstDSL.aggregate("avg", field("integer_value"))) + ) + )); + assertEquals("the target expected to be field, but is avg(Field(field=integer_value, fieldArgs=null))", exception.getMessage()); + } + protected void assertAnalyzeEqual(LogicalPlan expected, UnresolvedPlan unresolvedPlan) { assertEquals(expected, analyze(unresolvedPlan)); } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTestBase.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTestBase.java index 643b6a18d5..395defc9ed 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTestBase.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/AnalyzerTestBase.java @@ -22,6 +22,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import com.amazon.opendistroforelasticsearch.sql.expression.config.ExpressionConfig; import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; +import com.amazon.opendistroforelasticsearch.sql.expression.function.BuiltinFunctionRepository; import com.amazon.opendistroforelasticsearch.sql.storage.StorageEngine; import org.junit.jupiter.api.extension.ExtendWith; import org.springframework.beans.factory.annotation.Autowired; @@ -67,7 +68,7 @@ protected AnalysisContext analysisContext(TypeEnvironment typeEnvironment) { } @Bean - protected ExpressionAnalyzer expressionAnalyzer(DSL dsl) { - return new ExpressionAnalyzer(dsl); + protected ExpressionAnalyzer expressionAnalyzer(DSL dsl, BuiltinFunctionRepository repo) { + return new ExpressionAnalyzer(dsl, repo); } } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzerTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzerTest.java index 19866f3671..6f1831a94c 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzerTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/analysis/ExpressionAnalyzerTest.java @@ -22,6 +22,7 @@ import com.amazon.opendistroforelasticsearch.sql.expression.Expression; import org.junit.jupiter.api.Test; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.field; import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.LITERAL_TRUE; import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.integerValue; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -53,6 +54,13 @@ public void undefined_var_semantic_check_failed() { assertEquals("can't resolve expression undefined_field in type env", exception.getMessage()); } + @Test + public void undefined_aggregation_function() { + SemanticCheckException exception = assertThrows(SemanticCheckException.class, + () -> analyze(AstDSL.aggregate("ESTDC_ERROR", field("integer_value")))); + assertEquals("Unsupported aggregation function ESTDC_ERROR", exception.getMessage()); + } + protected Expression analyze(UnresolvedExpression unresolvedExpression) { return expressionAnalyzer.analyze(unresolvedExpression, analysisContext); } diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValueTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValueTest.java new file mode 100644 index 0000000000..0004915b17 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprTupleValueTest.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 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.data.model; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +class ExprTupleValueTest { + @Test + public void equal_to_itself() { + ExprValue tupleValue = ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 2)); + assertTrue(tupleValue.equals(tupleValue)); + } + + @Test + public void tuple_compare_int() { + ExprValue tupleValue = ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 2)); + ExprValue intValue = ExprValueUtils.integerValue(10); + assertFalse(tupleValue.equals(intValue)); + } + + @Test + public void compre_tuple_with_different_size() { + ExprValue tupleValue1 = ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 2)); + ExprValue tupleValue2 = ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 2, "float_value", 1f)); + assertFalse(tupleValue1.equals(tupleValue2)); + assertFalse(tupleValue2.equals(tupleValue1)); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtilsTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtilsTest.java index 1dd426388d..26ef90bc6b 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtilsTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/data/model/ExprValueUtilsTest.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.data.model; import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.storage.bindingtuple.BindingTuple; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -36,6 +37,7 @@ import static com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils.integerValue; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @DisplayName("Test Expression Value Utils") @@ -156,4 +158,15 @@ public void unSupportedObject() { assertEquals("unsupported object class com.amazon.opendistroforelasticsearch.sql.data.model.ExprIntegerValue", exception.getMessage()); } + + @Test + public void bindingTuples() { + for (ExprValue value : allValues) { + if (ExprType.STRUCT == value.type()) { + assertNotEquals(BindingTuple.EMPTY, value.bindingTuples()); + } else { + assertEquals(BindingTuple.EMPTY, value.bindingTuples()); + } + } + } } \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationTest.java new file mode 100644 index 0000000000..5621c5d705 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AggregationTest.java @@ -0,0 +1,66 @@ +/* + * Copyright 2019 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.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.ExpressionTestBase; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class AggregationTest extends ExpressionTestBase { + + protected static List tuples = Arrays.asList( + ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 2, + "long_value", 2L, + "string_value", "m", + "double_value", 2d, + "float_value", 2f)), + ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 1, + "long_value", 1L, + "string_value", "f", + "double_value", 1d, + "float_value", 1f)), + ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 3, + "long_value", 3L, + "string_value", "m", + "double_value", 3d, + "float_value", 3f)), + ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 4, + "long_value", 4L, + "string_value", "f", + "double_value", 4d, + "float_value", 4f))); + + protected static List tuples_with_null_and_missing = Arrays.asList( + ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 2, + "string_value", "m", + "double_value", 3d)), + ExprValueUtils.tupleValue(ImmutableMap.of("integer_value", 1, + "string_value", "f", + "double_value", 4d)), + ExprValueUtils.tupleValue(Collections.singletonMap("double_value", null))); + + protected ExprValue aggregation(Aggregator aggregator, List tuples) { + AggregationState state = aggregator.create(); + for (ExprValue tuple : tuples) { + aggregator.iterate(tuple.bindingTuples(), state); + } + return state.result(); + } +} diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregatorTest.java new file mode 100644 index 0000000000..31cd295968 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/AvgAggregatorTest.java @@ -0,0 +1,61 @@ +/* + * Copyright 2019 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.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class AvgAggregatorTest extends AggregationTest { + + @Test + public void avg_field_expression() { + ExprValue result = aggregation(dsl.avg(typeEnv,DSL.ref("integer_value")), tuples); + assertEquals(2.5, result.value()); + } + + @Test + public void avg_arithmetic_expression() { + ExprValue result = aggregation(dsl.avg(typeEnv, + dsl.multiply(typeEnv, DSL.ref("integer_value"), DSL.literal(ExprValueUtils.integerValue(10)))), tuples); + assertEquals(25.0, result.value()); + } + + @Test + public void avg_with_missing() { + ExprValue result = aggregation(dsl.avg(typeEnv,DSL.ref("integer_value")), tuples_with_null_and_missing); + assertTrue(result.isNull()); + } + + @Test + public void avg_with_null() { + ExprValue result = aggregation(dsl.avg(typeEnv,DSL.ref("double_value")), tuples_with_null_and_missing); + assertTrue(result.isNull()); + } + + @Test + public void valueOf() { + ExpressionEvaluationException exception = assertThrows(ExpressionEvaluationException.class, + () -> dsl.avg(typeEnv, DSL.ref("double_value")).valueOf(valueEnv())); + assertEquals("can't evaluate on aggregator: avg", exception.getMessage()); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregatorTest.java new file mode 100644 index 0000000000..f7bf317ed3 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/aggregation/SumAggregatorTest.java @@ -0,0 +1,94 @@ +/* + * Copyright 2019 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 static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.SumAggregator.SumState; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +class SumAggregatorTest extends AggregationTest { + + @Test + public void sum_integer_field_expression() { + ExprValue result = aggregation(dsl.sum(typeEnv, DSL.ref("integer_value")), tuples); + assertEquals(10, result.value()); + } + + @Test + public void sum_long_field_expression() { + ExprValue result = aggregation(dsl.sum(typeEnv, DSL.ref("long_value")), tuples); + assertEquals(10L, result.value()); + } + + @Test + public void sum_float_field_expression() { + ExprValue result = aggregation(dsl.sum(typeEnv, DSL.ref("float_value")), tuples); + assertEquals(10f, result.value()); + } + + @Test + public void sum_double_field_expression() { + ExprValue result = aggregation(dsl.sum(typeEnv, DSL.ref("double_value")), tuples); + assertEquals(10d, result.value()); + } + + @Test + public void sum_arithmetic_expression() { + ExprValue result = aggregation(dsl.sum(typeEnv, + dsl.multiply(typeEnv, DSL.ref("integer_value"), DSL.literal(ExprValueUtils.integerValue(10)))), tuples); + assertEquals(100, result.value()); + } + + @Test + public void sum_string_field_expression() { + SumAggregator sumAggregator = new SumAggregator(ImmutableList.of(DSL.ref("string_value")), ExprType.STRING); + SumState sumState = sumAggregator.create(); + ExpressionEvaluationException exception = assertThrows(ExpressionEvaluationException.class, + () -> sumAggregator + .iterate(ExprValueUtils.tupleValue(ImmutableMap.of("string_value", "m")).bindingTuples(), sumState) + ); + assertEquals("unexpected type [STRING] in sum aggregation", exception.getMessage()); + } + + @Test + public void sum_with_missing() { + ExprValue result = aggregation(dsl.sum(typeEnv, DSL.ref("integer_value")), tuples_with_null_and_missing); + assertTrue(result.isNull()); + } + + @Test + public void sum_with_null() { + ExprValue result = aggregation(dsl.sum(typeEnv, DSL.ref("double_value")), tuples_with_null_and_missing); + assertTrue(result.isNull()); + } + + @Test + public void valueOf() { + ExpressionEvaluationException exception = assertThrows(ExpressionEvaluationException.class, + () -> dsl.sum(typeEnv, DSL.ref("double_value")).valueOf(valueEnv())); + assertEquals("can't evaluate on aggregator: sum", exception.getMessage()); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepositoryTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepositoryTest.java index 36ab7ec12b..f53b0efab0 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepositoryTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/BuiltinFunctionRepositoryTest.java @@ -44,7 +44,7 @@ class BuiltinFunctionRepositoryTest { @Mock private FunctionName mockFunctionName; @Mock - private FunctionExpressionBuilder functionExpressionBuilder; + private FunctionBuilder functionExpressionBuilder; @Mock private FunctionSignature functionSignature; @Mock diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolverTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolverTest.java index 08a1cce182..1ad39af6ed 100644 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolverTest.java +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/expression/function/FunctionResolverTest.java @@ -24,7 +24,8 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.when; @DisplayNameGeneration(DisplayNameGenerator.ReplaceUnderscores.class) @@ -41,13 +42,13 @@ class FunctionResolverTest { @Mock private FunctionSignature functionSignature; @Mock - private FunctionExpressionBuilder exactlyMatchBuilder; + private FunctionBuilder exactlyMatchBuilder; @Mock - private FunctionExpressionBuilder bestMatchBuilder; + private FunctionBuilder bestMatchBuilder; @Mock - private FunctionExpressionBuilder leastMatchBuilder; + private FunctionBuilder leastMatchBuilder; @Mock - private FunctionExpressionBuilder notMatchBuilder; + private FunctionBuilder notMatchBuilder; private FunctionName functionName = FunctionName.of("add"); diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitorTest.java deleted file mode 100644 index 01f9c1ba04..0000000000 --- a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/AbstractPlanNodeVisitorTest.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2019 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; - -import com.amazon.opendistroforelasticsearch.sql.expression.Expression; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalFilter; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanDSL; -import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; - -import java.util.stream.Collectors; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; - -/** - * Temporary added for UT coverage, Will be removed. - */ -@ExtendWith(MockitoExtension.class) -class AbstractPlanNodeVisitorTest { - @Mock - Expression expression; - - @Test - public void logicalPlanShouldTraversable() { - LogicalPlan logicalPlan = LogicalPlanDSL.filter( - expression, - LogicalPlanDSL.relation("schema") - ); - - Integer result = logicalPlan.accept(new NodesCount(), null); - assertEquals(2, result); - } - - @Test - public void testAbstractPlanNodeVisitorShouldReturnNull() { - LogicalPlan relation = LogicalPlanDSL.relation("schema"); - LogicalPlan filter = LogicalPlanDSL.filter(expression,relation); - - assertNull(relation.accept(new AbstractPlanNodeVisitor() {}, null)); - assertNull(filter.accept(new AbstractPlanNodeVisitor() {}, null)); - } - - private static class NodesCount extends AbstractPlanNodeVisitor { - @Override - public Integer visitRelation(LogicalRelation plan, Object context) { - return 1; - } - - @Override - public Integer visitFilter(LogicalFilter plan, Object context) { - return 1 + plan.getChild().stream() - .map(child -> child.accept(this, context)).collect(Collectors.summingInt(Integer::intValue)); - } - } -} \ No newline at end of file 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 new file mode 100644 index 0000000000..7c78736760 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/PlannerTest.java @@ -0,0 +1,110 @@ +/* + * Copyright 2019 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; + +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +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.LogicalPlanDSL; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanNodeVisitor; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation; +import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRename; +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; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlanTestBase; +import com.amazon.opendistroforelasticsearch.sql.planner.physical.RenameOperator; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class PlannerTest extends PhysicalPlanTestBase { + @Mock + private PhysicalPlan scan; + + @Test + public void planner_test() { + assertPhysicalPlan( + PhysicalPlanDSL.rename( + PhysicalPlanDSL.agg( + PhysicalPlanDSL.filter( + scan, + dsl.equal(typeEnv(), DSL.ref("response"), DSL.literal(10)) + ), + ImmutableList.of(dsl.avg(typeEnv(), DSL.ref("response"))), + ImmutableList.of() + ), + ImmutableMap.of(DSL.ref("ivalue"), DSL.ref("avg(response)")) + ), + LogicalPlanDSL.rename( + LogicalPlanDSL.aggregation( + LogicalPlanDSL.filter( + LogicalPlanDSL.relation("schema"), + dsl.equal(typeEnv(), DSL.ref("response"), DSL.literal(10)) + ), + ImmutableList.of(dsl.avg(typeEnv(), DSL.ref("response"))), + ImmutableList.of() + ), + ImmutableMap.of(DSL.ref("ivalue"), DSL.ref("avg(response)")) + ) + ); + } + + protected void assertPhysicalPlan(PhysicalPlan expected, LogicalPlan logicalPlan) { + assertEquals(expected, analyze(logicalPlan)); + } + + protected PhysicalPlan analyze(LogicalPlan logicalPlan) { + return new TestPlanner().plan(logicalPlan, ImmutableMap.of()); + } + + /** + * Todo, For test coverage only. Remove it when Planner is implemented. + */ + protected class TestPlanner extends LogicalPlanNodeVisitor { + public PhysicalPlan plan(LogicalPlan plan, Object o) { + return plan.accept(this, o); + } + + @Override + public PhysicalPlan visitRelation(LogicalRelation plan, Object context) { + return scan; + } + + @Override + public PhysicalPlan visitFilter(LogicalFilter plan, Object context) { + return new FilterOperator(plan.getChild().get(0).accept(this, context), plan.getCondition()); + } + + @Override + public PhysicalPlan visitAggregation(LogicalAggregation plan, Object context) { + return new AggregationOperator(plan.getChild().get(0).accept(this, context), + plan.getAggregatorList(), plan.getGroupByList() + ); + } + + @Override + public PhysicalPlan visitRename(LogicalRename plan, Object context) { + return new RenameOperator(plan.getChild().get(0).accept(this, context), + plan.getRenameMap()); + } + } +} 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 new file mode 100644 index 0000000000..4f8a53cbf5 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/logical/LogicalPlanNodeVisitorTest.java @@ -0,0 +1,111 @@ +/* + * Copyright 2019 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 com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression; +import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** + * Todo. Temporary added for UT coverage, Will be removed. + */ +@ExtendWith(MockitoExtension.class) +class LogicalPlanNodeVisitorTest { + @Mock + Expression expression; + @Mock + ReferenceExpression ref; + @Mock + Aggregator aggregator; + + @Test + public void logicalPlanShouldTraversable() { + LogicalPlan logicalPlan = + LogicalPlanDSL.rename( + LogicalPlanDSL.aggregation( + LogicalPlanDSL.filter( + LogicalPlanDSL.relation("schema"), + expression), + ImmutableList.of(aggregator), + ImmutableList.of(expression) + ), + ImmutableMap.of(ref, ref) + ); + + Integer result = logicalPlan.accept(new NodesCount(), null); + assertEquals(4, result); + } + + @Test + public void testAbstractPlanNodeVisitorShouldReturnNull() { + LogicalPlan relation = LogicalPlanDSL.relation("schema"); + LogicalPlan filter = LogicalPlanDSL.filter(relation, expression); + LogicalPlan aggregation = LogicalPlanDSL.aggregation( + filter, + ImmutableList.of(aggregator), + ImmutableList.of(expression) + ); + LogicalPlan rename = LogicalPlanDSL.rename( + aggregation, + ImmutableMap.of(ref, ref) + ); + + assertNull(relation.accept(new LogicalPlanNodeVisitor() { + }, null)); + assertNull(filter.accept(new LogicalPlanNodeVisitor() { + }, null)); + assertNull(aggregation.accept(new LogicalPlanNodeVisitor() { + }, null)); + assertNull(rename.accept(new LogicalPlanNodeVisitor() { + }, null)); + } + + private static class NodesCount extends LogicalPlanNodeVisitor { + @Override + public Integer visitRelation(LogicalRelation plan, Object context) { + return 1; + } + + @Override + public Integer visitFilter(LogicalFilter 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 + plan.getChild().stream() + .map(child -> child.accept(this, context)).collect(Collectors.summingInt(Integer::intValue)); + } + + @Override + public Integer visitRename(LogicalRename plan, Object context) { + return 1 + plan.getChild().stream() + .map(child -> child.accept(this, context)).collect(Collectors.summingInt(Integer::intValue)); + } + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000..d703c50577 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/AggregationOperatorTest.java @@ -0,0 +1,71 @@ +/* + * Copyright 2019 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; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertEquals; + +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.Arrays; +import java.util.Collections; +import java.util.List; +import org.junit.jupiter.api.Test; + +class AggregationOperatorTest extends PhysicalPlanTestBase { + @Test + public void avg_with_one_groups() { + PhysicalPlan plan = new AggregationOperator(new TestScan(), + Collections.singletonList(dsl.avg(typeEnv(), DSL.ref("response"))), + Collections.singletonList(DSL.ref("action"))); + List result = execute(plan); + assertEquals(2, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("action", "GET", "avg(response)", 268d)), + ExprValueUtils.tupleValue(ImmutableMap.of("action", "POST", "avg(response)", 350d)) + )); + } + + @Test + public void avg_with_two_groups() { + PhysicalPlan plan = new AggregationOperator(new TestScan(), + Collections.singletonList(dsl.avg(typeEnv(), DSL.ref("response"))), + Arrays.asList(DSL.ref("action"), DSL.ref("ip"))); + List result = execute(plan); + assertEquals(3, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("action", "GET", "ip", "209.160.24.63", "avg(response)", 302d)), + ExprValueUtils.tupleValue(ImmutableMap.of("action", "GET", "ip", "112.111.162.4", "avg(response)", 200d)), + ExprValueUtils.tupleValue(ImmutableMap.of("action", "POST", "ip", "74.125.19.106", "avg(response)", 350d)) + )); + } + + @Test + public void sum_with_one_groups() { + PhysicalPlan plan = new AggregationOperator(new TestScan(), + Collections.singletonList(dsl.sum(typeEnv(), DSL.ref("response"))), + Collections.singletonList(DSL.ref("action"))); + List result = execute(plan); + assertEquals(2, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("action", "GET", "sum(response)", 804)), + ExprValueUtils.tupleValue(ImmutableMap.of("action", "POST", "sum(response)", 700)) + )); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperatorTest.java new file mode 100644 index 0000000000..f3828b3126 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/FilterOperatorTest.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 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; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertEquals; + +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.List; +import org.junit.jupiter.api.Test; + +class FilterOperatorTest extends PhysicalPlanTestBase { + + @Test + public void filterTest() { + FilterOperator plan = new FilterOperator(new TestScan(), + dsl.equal(typeEnv(), DSL.ref("response"), DSL.literal(404))); + List result = execute(plan); + assertEquals(1, result.size()); + assertThat(result, containsInAnyOrder(ExprValueUtils + .tupleValue(ImmutableMap.of("ip", "209.160.24.63", "action", "GET", "response", 404, "referer", "www.amazon.com")))); + } +} \ 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 new file mode 100644 index 0000000000..26e9f69b07 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanNodeVisitorTest.java @@ -0,0 +1,125 @@ +/* + * Copyright 2019 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.expression.DSL; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** + * Todo, testing purpose, delete later. + */ +class PhysicalPlanNodeVisitorTest extends PhysicalPlanTestBase { + @Test + public void print_physical_plan() { + PhysicalPlan plan = PhysicalPlanDSL.rename( + PhysicalPlanDSL.agg( + PhysicalPlanDSL.filter( + new TestScan(), + dsl.equal(typeEnv(), DSL.ref("response"), DSL.literal(10)) + ), + ImmutableList.of(dsl.avg(typeEnv(), DSL.ref("response"))), + ImmutableList.of() + ), + ImmutableMap.of(DSL.ref("ivalue"), DSL.ref("avg(response)")) + ); + + PhysicalPlanPrinter printer = new PhysicalPlanPrinter(); + assertEquals("Rename->\n" + + "\tAggregation->\n" + + "\t\tFilter->", printer.print(plan)); + } + + @Test + public void test_PhysicalPlanVisitor_should_return_null() { + PhysicalPlan filter = PhysicalPlanDSL.filter( + new TestScan(), + dsl.equal(typeEnv(), DSL.ref("response"), DSL.literal(10)) + ); + PhysicalPlan aggregation = PhysicalPlanDSL.agg( + filter, + ImmutableList.of(dsl.avg(typeEnv(), DSL.ref("response"))), + ImmutableList.of() + ); + PhysicalPlan rename = PhysicalPlanDSL.rename( + aggregation, + ImmutableMap.of(DSL.ref("ivalue"), DSL.ref("avg(response)")) + ); + assertNull(filter.accept(new PhysicalPlanNodeVisitor() { + }, null)); + assertNull(aggregation.accept(new PhysicalPlanNodeVisitor() { + }, null)); + assertNull(rename.accept(new PhysicalPlanNodeVisitor() { + }, null)); + } + + public static class PhysicalPlanPrinter extends PhysicalPlanNodeVisitor { + + public String print(PhysicalPlan node) { + return node.accept(this, 0); + } + + @Override + public String visitFilter(FilterOperator node, Integer tabs) { + String child = node.getChild().get(0).accept(this, tabs + 1); + StringBuilder sb = new StringBuilder(); + for (Integer i = 0; i < tabs; i++) { + sb.append("\t"); + } + sb.append("Filter->"); + if (!Strings.isNullOrEmpty(child)) { + sb.append("\n"); + sb.append(child); + } + return sb.toString(); + } + + @Override + public String visitAggregation(AggregationOperator node, Integer tabs) { + String child = node.getChild().get(0).accept(this, tabs + 1); + StringBuilder sb = new StringBuilder(); + for (Integer i = 0; i < tabs; i++) { + sb.append("\t"); + } + sb.append("Aggregation->"); + if (!Strings.isNullOrEmpty(child)) { + sb.append("\n"); + sb.append(child); + } + return sb.toString(); + } + + @Override + public String visitRename(RenameOperator node, Integer tabs) { + String child = node.getChild().get(0).accept(this, tabs + 1); + StringBuilder sb = new StringBuilder(); + for (Integer i = 0; i < tabs; i++) { + sb.append("\t"); + } + sb.append("Rename->"); + if (!Strings.isNullOrEmpty(child)) { + sb.append("\n"); + sb.append(child); + } + return sb.toString(); + } + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000..a4a89f8897 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/PhysicalPlanTestBase.java @@ -0,0 +1,111 @@ +/* + * Copyright 2019 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.ExprType; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +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.config.ExpressionConfig; +import com.amazon.opendistroforelasticsearch.sql.expression.env.Environment; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.junit.jupiter.api.extension.ExtendWith; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.annotation.Bean; +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}) +public class PhysicalPlanTestBase { + @Autowired + protected DSL dsl; + + private 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"))) + .add(ExprValueUtils.tupleValue(ImmutableMap.of("ip", "209.160.24.63", "action", "GET", "response", 404, "referer", "www.amazon.com"))) + .add(ExprValueUtils.tupleValue(ImmutableMap.of("ip", "112.111.162.4", "action", "GET", "response", 200, "referer", "www.amazon.com"))) + .add(ExprValueUtils.tupleValue(ImmutableMap.of("ip", "74.125.19.106", "action", "POST", "response", 200, "referer", "www.google.com"))) + .add(ExprValueUtils.tupleValue(ImmutableMap.of("ip", "74.125.19.106", "action", "POST", "response", 500))) + .build(); + + private static Map typeMapping = new ImmutableMap.Builder() + .put("ip", ExprType.STRING) + .put("action", ExprType.STRING) + .put("response", ExprType.INTEGER) + .put("referer", ExprType.STRING) + .build(); + + @Bean + protected Environment typeEnv() { + return var -> { + if (var instanceof ReferenceExpression) { + ReferenceExpression refExpr = (ReferenceExpression) var; + if (typeMapping.containsKey(refExpr.getAttr())) { + return typeMapping.get(refExpr.getAttr()); + } + } + throw new ExpressionEvaluationException("type resolved failed"); + }; + } + + protected List execute(PhysicalPlan plan) { + ImmutableList.Builder builder = new ImmutableList.Builder<>(); + plan.open(); + while (plan.hasNext()) { + builder.add(plan.next()); + } + plan.close(); + return builder.build(); + } + + protected static class TestScan extends PhysicalPlan { + private final Iterator iterator; + + public TestScan() { + iterator = inputs.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/RenameOperatorTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java new file mode 100644 index 0000000000..ff50d3c9a4 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/planner/physical/RenameOperatorTest.java @@ -0,0 +1,67 @@ +/* + * Copyright 2019 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; +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.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.Collections; +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) +public class RenameOperatorTest extends PhysicalPlanTestBase { + @Mock + private PhysicalPlan inputPlan; + + @Test + public void avg_aggregation_rename() { + PhysicalPlan plan = new RenameOperator( + new AggregationOperator(new TestScan(), + Collections.singletonList(dsl.avg(typeEnv(), DSL.ref("response"))), + Collections.singletonList(DSL.ref("action"))), + ImmutableMap.of(DSL.ref("avg(response)"), DSL.ref("avg")) + ); + List result = execute(plan); + assertEquals(2, result.size()); + assertThat(result, containsInAnyOrder( + ExprValueUtils.tupleValue(ImmutableMap.of("action", "GET", "avg", 268d)), + ExprValueUtils.tupleValue(ImmutableMap.of("action", "POST", "avg", 350d)) + )); + } + + @Test + public void rename_int_value() { + when(inputPlan.hasNext()).thenReturn(true, false); + when(inputPlan.next()).thenReturn(ExprValueUtils.integerValue(1)); + PhysicalPlan plan = new RenameOperator( + inputPlan, + ImmutableMap.of(DSL.ref("avg(response)"), DSL.ref("avg")) + ); + List result = execute(plan); + assertEquals(1, result.size()); + assertThat(result, containsInAnyOrder(ExprValueUtils.integerValue(1))); + } +} diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTupleTest.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTupleTest.java new file mode 100644 index 0000000000..1f8bb9c741 --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/storage/bindingtuple/BindingTupleTest.java @@ -0,0 +1,52 @@ +/* + * Copyright 2019 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.storage.bindingtuple; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.exception.ExpressionEvaluationException; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +class BindingTupleTest { + @Test + public void resolve_ref_expression() { + BindingTuple bindingTuple = ExprValueUtils.tupleValue(ImmutableMap.of("ip", "209.160.24.63")).bindingTuples(); + assertEquals(ExprValueUtils.stringValue("209.160.24.63"), bindingTuple.resolve(DSL.ref("ip"))); + } + + @Test + public void resolve_missing_expression() { + BindingTuple bindingTuple = ExprValueUtils.tupleValue(ImmutableMap.of("ip", "209.160.24.63")).bindingTuples(); + assertEquals(ExprValueUtils.LITERAL_MISSING, bindingTuple.resolve(DSL.ref("ip_missing"))); + } + + @Test + public void resolve_from_empty_tuple() { + assertEquals(ExprValueUtils.LITERAL_MISSING, BindingTuple.EMPTY.resolve(DSL.ref("ip_missing"))); + } + + @Test + public void resolve_literal_expression_throw_exception() { + BindingTuple bindingTuple = ExprValueUtils.tupleValue(ImmutableMap.of("ip", "209.160.24.63")).bindingTuples(); + ExpressionEvaluationException exception = assertThrows + (ExpressionEvaluationException.class, () -> bindingTuple.resolve(DSL.literal(1))); + assertEquals("can resolve expression: 1", exception.getMessage()); + } +} \ No newline at end of file diff --git a/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/utils/MatcherUtils.java b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/utils/MatcherUtils.java new file mode 100644 index 0000000000..b5ee2efe4b --- /dev/null +++ b/core/src/test/java/com/amazon/opendistroforelasticsearch/sql/utils/MatcherUtils.java @@ -0,0 +1,53 @@ +/* + * Copyright 2019 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.utils; + +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprTupleValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprType; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.data.model.ExprValueUtils; +import com.amazon.opendistroforelasticsearch.sql.expression.DSL; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; + +import java.util.Map; + +public class MatcherUtils { + public static TypeSafeMatcher tuple(Map map) { + return new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText(map.toString()); + } + + @Override + protected boolean matchesSafely(ExprValue value) { + if (ExprType.STRUCT == value.type()) { + ExprTupleValue tupleValue = (ExprTupleValue) value; + for (Map.Entry entry : map.entrySet()) { + if (!tupleValue.bindingTuples().resolve(DSL.ref(entry.getKey())) + .equals(ExprValueUtils.fromObjectValue(entry.getValue()))) { + return false; + } + } + return true; + } else { + return false; + } + } + }; + } +} diff --git a/ppl/src/main/antlr/OpenDistroPPLParser.g4 b/ppl/src/main/antlr/OpenDistroPPLParser.g4 index f32926fbfc..41034c9a7f 100644 --- a/ppl/src/main/antlr/OpenDistroPPLParser.g4 +++ b/ppl/src/main/antlr/OpenDistroPPLParser.g4 @@ -54,7 +54,7 @@ statsCommand (PARTITIONS EQUAL partitions=integerLiteral)? (ALLNUM EQUAL allnum=booleanLiteral)? (DELIM EQUAL delim=stringLiteral)? - statsAggTerm (AS alias=wcFieldExpression)? + statsAggTerm (COMMA statsAggTerm)* (byClause)? (DEDUP_SPLITVALUES EQUAL dedupsplit=booleanLiteral)? ; @@ -93,7 +93,7 @@ sortbyClause /** aggregation terms */ statsAggTerm - : statsFunction + : statsFunction (AS alias=wcFieldExpression)? ; sparklineAggTerm 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 2fc1dfb29e..ada5238cfa 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 @@ -15,21 +15,25 @@ package com.amazon.opendistroforelasticsearch.sql.ppl.parser; -import com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParserBaseVisitor; -import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Map; +import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Aggregation; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Filter; -import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Project; import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename; +import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan; +import com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser; +import com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParserBaseVisitor; import com.amazon.opendistroforelasticsearch.sql.ppl.utils.ArgumentFactory; +import com.google.common.collect.ImmutableList; +import lombok.RequiredArgsConstructor; +import org.antlr.v4.runtime.tree.ParseTree; + import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; -import lombok.RequiredArgsConstructor; -import org.antlr.v4.runtime.tree.ParseTree; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.DedupCommandContext; import static com.amazon.opendistroforelasticsearch.sql.ppl.antlr.parser.OpenDistroPPLParser.EvalCommandContext; @@ -111,13 +115,21 @@ public UnresolvedPlan visitRenameCommand(RenameCommandContext ctx) { ); } - /** Stats command */ + /** + * Stats command + */ @Override public UnresolvedPlan visitStatsCommand(StatsCommandContext ctx) { - List aggList = Collections.singletonList( - new Map(visitExpression(ctx.statsAggTerm()), - ctx.alias != null ? visitExpression(ctx.alias) : null) - ); + 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))); + } + } List groupList = ctx.byClause() == null ? null : ctx.byClause() .fieldList() @@ -125,12 +137,14 @@ public UnresolvedPlan visitStatsCommand(StatsCommandContext ctx) { .stream() .map(this::visitExpression) .collect(Collectors.toList()); - return new Aggregation( - aggList, + Aggregation aggregation = new Aggregation( + aggListBuilder.build(), null, groupList, ArgumentFactory.getArgumentList(ctx) ); + List renameList = renameListBuilder.build(); + return renameList.isEmpty() ? aggregation : new Rename(renameList, aggregation); } /** Dedup command */ diff --git a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilder.java b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilder.java index 8404795201..5effdacf9a 100644 --- a/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilder.java +++ b/ppl/src/main/java/com/amazon/opendistroforelasticsearch/sql/ppl/parser/AstExpressionBuilder.java @@ -33,7 +33,7 @@ import com.amazon.opendistroforelasticsearch.sql.ast.expression.Not; import com.amazon.opendistroforelasticsearch.sql.ast.expression.Or; import com.amazon.opendistroforelasticsearch.sql.ppl.utils.ArgumentFactory; -import java.util.Arrays; + import java.util.Collections; import java.util.stream.Collectors; import org.antlr.v4.runtime.ParserRuleContext; 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 aa3e56edb9..fb832ac060 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 @@ -18,6 +18,7 @@ import com.amazon.opendistroforelasticsearch.sql.ppl.antlr.PPLSyntaxParser; import com.amazon.opendistroforelasticsearch.sql.ast.Node; import org.junit.Test; + 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.compare; @@ -36,6 +37,7 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.project; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.projectWithArg; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.relation; +import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.rename; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.stringLiteral; import static org.junit.Assert.assertEquals; @@ -115,7 +117,7 @@ public void testStatsCommand() { agg( relation("t"), exprList( - map(aggregate("count", field("a")), null) + aggregate("count", field("a")) ), null, null, @@ -129,7 +131,7 @@ public void testStatsCommandWithByClause() { agg( relation("t"), exprList( - map(aggregate("count", field("a")), null) + aggregate("count", field("a")) ), null, exprList(field("b")), @@ -141,18 +143,19 @@ public void testStatsCommandWithByClause() { @Test public void testStatsCommandWithAlias() { assertEqual("source=t | stats count(a) as alias", - agg( - relation("t"), - exprList( - map( - aggregate("count", field("a")), - field("alias") - ) + rename( + agg( + relation("t"), + exprList( + aggregate("count", field("a")) + ), + null, + null, + defaultStatsArgs() ), - null, - null, - defaultStatsArgs() - )); + map(aggregate("count", field("a")), field("alias")) + ) + ); } @Test 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 5c0015dbc0..8bab7bfd44 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 @@ -34,7 +34,6 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.function; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.in; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.intLiteral; -import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.map; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.not; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.nullLiteral; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.or; @@ -243,10 +242,7 @@ public void testAggFuncCallExpr() { agg( relation("t"), exprList( - map( - aggregate("avg", field("a")), - null - ) + aggregate("avg", field("a")) ), null, @@ -261,13 +257,10 @@ public void testPercentileAggFuncExpr() { agg( relation("t"), exprList( - map( - aggregate( - "percentile", - field("a"), - argument("rank", intLiteral(1)) - ), - null + aggregate( + "percentile", + field("a"), + argument("rank", intLiteral(1)) ) ), null, 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 276a04add2..347514b799 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 @@ -26,7 +26,6 @@ import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.exprList; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.field; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.intLiteral; -import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.map; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.projectWithArg; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.relation; import static com.amazon.opendistroforelasticsearch.sql.ast.dsl.AstDSL.stringLiteral; @@ -53,7 +52,7 @@ public void testStatsCommandArgument() { assertEqual("source=t | stats partitions=1 allnum=false delim=',' avg(a) dedup_splitvalues=true", agg( relation("t"), - exprList(map(aggregate("avg", field("a")), null)), + exprList(aggregate("avg", field("a"))), null, null, exprList(