From 6fe52e8b142d52bea8a04b06644a3cb69263a8ea Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 6 Dec 2024 13:40:00 -0800 Subject: [PATCH 01/37] Update grammar def Signed-off-by: Andy Kwok --- ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 | 1 + .../src/main/antlr4/OpenSearchPPLParser.g4 | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 index d15f5c8e3..01ea8768a 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 @@ -42,6 +42,7 @@ FILLNULL: 'FILLNULL'; EXPAND: 'EXPAND'; FLATTEN: 'FLATTEN'; TRENDLINE: 'TRENDLINE'; +APPENDCOL: 'APPENDCOL'; //Native JOIN KEYWORDS JOIN: 'JOIN'; diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 index eb6cd1a35..853ab653d 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 @@ -56,6 +56,7 @@ commands | flattenCommand | expandCommand | trendlineCommand + | appendcolCommand ; commandName @@ -90,6 +91,7 @@ commandName | FIELDSUMMARY | FLATTEN | TRENDLINE + | APPENDCOL ; searchCommand @@ -275,6 +277,10 @@ trendlineType | WMA ; +appendcolCommand + : APPENDCOL LT_SQR_PRTHS commands (PIPE commands)? RT_SQR_PRTHS + ; + kmeansCommand : KMEANS (kmeansParameter)* ; From 0b3b50cfc0fbc4867d78b77f28675cb7083205dd Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 6 Dec 2024 14:25:38 -0800 Subject: [PATCH 02/37] Skeleton for Append fields Signed-off-by: Andy Kwok --- .../src/main/antlr4/OpenSearchPPLParser.g4 | 2 +- .../java/org/opensearch/sql/ast/AbstractNodeVisitor.java | 4 ++++ .../org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java | 7 +++++++ .../java/org/opensearch/sql/ppl/parser/AstBuilder.java | 7 +++++++ 4 files changed, 19 insertions(+), 1 deletion(-) diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 index 853ab653d..47d3c3f94 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 @@ -278,7 +278,7 @@ trendlineType ; appendcolCommand - : APPENDCOL LT_SQR_PRTHS commands (PIPE commands)? RT_SQR_PRTHS + : APPENDCOL LT_SQR_PRTHS commands (PIPE commands)* RT_SQR_PRTHS ; kmeansCommand diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java index dadf6b968..b3d1ce964 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java @@ -121,6 +121,10 @@ public T visitTrendline(Trendline node, C context) { return visitChildren(node, context); } + public T visitAppendCol(Trendline node, C context) { + return visitChildren(node, context); + } + public T visitCorrelation(Correlation node, C context) { return visitChildren(node, context); } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index d7f59bae3..2a8f36bc3 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -43,6 +43,7 @@ import org.opensearch.sql.ast.expression.ParseMethod; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.ast.expression.WindowFunction; +import org.opensearch.sql.ast.expression.subquery.InSubquery; import org.opensearch.sql.ast.statement.Explain; import org.opensearch.sql.ast.statement.Query; import org.opensearch.sql.ast.statement.Statement; @@ -253,6 +254,12 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { return context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Project(seq(trendlineProjectExpressions), p)); } + @Override + public LogicalPlan visitAppendCol(Trendline node, CatalystPlanContext context) { + System.out.println("Printing node detail:" + node.toString()); + return super.visitAppendCol(node, context); + } + @Override public LogicalPlan visitCorrelation(Correlation node, CatalystPlanContext context) { visitFirstChild(node, context); diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index b1254bf8f..864425468 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -417,6 +417,13 @@ public UnresolvedPlan visitTrendlineCommand(OpenSearchPPLParser.TrendlineCommand .orElse(new Trendline(Optional.empty(), trendlineComputations)); } + @Override + public UnresolvedPlan visitAppendcolCommand(OpenSearchPPLParser.AppendcolCommandContext ctx) { + + System.out.println("No. of cmd: " + ctx.commands().size()); + return super.visitAppendcolCommand(ctx); + } + private Trendline.TrendlineComputation toTrendlineComputation(OpenSearchPPLParser.TrendlineClauseContext ctx) { int numberOfDataPoints = Integer.parseInt(ctx.numberOfDataPoints.getText()); if (numberOfDataPoints < 1) { From 50f4bd5e428d8cd550b9ab0d4eaf98bb22754d9a Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 6 Dec 2024 17:17:49 -0800 Subject: [PATCH 03/37] Visitor skeleton Signed-off-by: Andy Kwok --- .../sql/ast/AbstractNodeVisitor.java | 2 +- .../opensearch/sql/ast/tree/AppendCol.java | 32 ++++++++++++++++ .../sql/ppl/CatalystQueryPlanVisitor.java | 38 +++++++------------ .../opensearch/sql/ppl/parser/AstBuilder.java | 9 ++++- 4 files changed, 54 insertions(+), 27 deletions(-) create mode 100644 ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java index b3d1ce964..529c569e0 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java @@ -121,7 +121,7 @@ public T visitTrendline(Trendline node, C context) { return visitChildren(node, context); } - public T visitAppendCol(Trendline node, C context) { + public T visitAppendCol(AppendCol node, C context) { return visitChildren(node, context); } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java new file mode 100644 index 000000000..48c2b78f9 --- /dev/null +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java @@ -0,0 +1,32 @@ +package org.opensearch.sql.ast.tree; + +import org.opensearch.sql.ast.AbstractNodeVisitor; + +import java.util.List; + +/** + * A composite object which store the subQuery along with some more ad-hoc option like override + */ +public class AppendCol extends UnresolvedPlan { + + + public boolean override = true; + + private UnresolvedPlan child; + + + public AppendCol(UnresolvedPlan child) { + this.child = child; + } + + @Override + public UnresolvedPlan attach(UnresolvedPlan child) { + this.child = child; + return this; + } + + @Override + public T accept(AbstractNodeVisitor visitor, C context) { + return visitor.visitAppendCol(this, context); + } +} diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 2a8f36bc3..39a658518 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -47,30 +47,7 @@ import org.opensearch.sql.ast.statement.Explain; import org.opensearch.sql.ast.statement.Query; import org.opensearch.sql.ast.statement.Statement; -import org.opensearch.sql.ast.tree.Aggregation; -import org.opensearch.sql.ast.tree.Correlation; -import org.opensearch.sql.ast.tree.CountedAggregation; -import org.opensearch.sql.ast.tree.Dedupe; -import org.opensearch.sql.ast.tree.DescribeRelation; -import org.opensearch.sql.ast.tree.Eval; -import org.opensearch.sql.ast.tree.FieldSummary; -import org.opensearch.sql.ast.tree.FillNull; -import org.opensearch.sql.ast.tree.Filter; -import org.opensearch.sql.ast.tree.Flatten; -import org.opensearch.sql.ast.tree.Head; -import org.opensearch.sql.ast.tree.Join; -import org.opensearch.sql.ast.tree.Kmeans; -import org.opensearch.sql.ast.tree.Lookup; -import org.opensearch.sql.ast.tree.Parse; -import org.opensearch.sql.ast.tree.Project; -import org.opensearch.sql.ast.tree.RareAggregation; -import org.opensearch.sql.ast.tree.RareTopN; -import org.opensearch.sql.ast.tree.Relation; -import org.opensearch.sql.ast.tree.Rename; -import org.opensearch.sql.ast.tree.Sort; -import org.opensearch.sql.ast.tree.SubqueryAlias; -import org.opensearch.sql.ast.tree.Trendline; -import org.opensearch.sql.ast.tree.Window; +import org.opensearch.sql.ast.tree.*; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.ppl.utils.FieldSummaryTransformer; import org.opensearch.sql.ppl.utils.ParseTransformer; @@ -255,7 +232,18 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { } @Override - public LogicalPlan visitAppendCol(Trendline node, CatalystPlanContext context) { + public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { + + + // To-do: + // 1. Fetch the table name from ctx + // 2. Add to child + // + // Append it to the sub query + // Add the rowNumber( ) + + // With the aim to produce a select t1, t2 * query with row_number( ) as join. + System.out.println("Printing node detail:" + node.toString()); return super.visitAppendCol(node, context); } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index 864425468..2ca37c5b8 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -91,6 +91,7 @@ public AstBuilder(String query) { @Override public UnresolvedPlan visitQueryStatement(OpenSearchPPLParser.QueryStatementContext ctx) { UnresolvedPlan pplCommand = visit(ctx.pplCommands()); + visit(ctx.commands().get(0)); return ctx.commands().stream().map(this::visit).reduce(pplCommand, (r, e) -> e.attach(r)); } @@ -420,8 +421,14 @@ public UnresolvedPlan visitTrendlineCommand(OpenSearchPPLParser.TrendlineCommand @Override public UnresolvedPlan visitAppendcolCommand(OpenSearchPPLParser.AppendcolCommandContext ctx) { + // All args validation should happen here. System.out.println("No. of cmd: " + ctx.commands().size()); - return super.visitAppendcolCommand(ctx); + + // TD: Pass on AppendColSpecific option also + final Optional pplCmd = ctx.commands().stream().map(this::visit).reduce((r, e) -> e.attach(r)); + + // Throw some exception. + return pplCmd.map(AppendCol::new).orElse(null); } private Trendline.TrendlineComputation toTrendlineComputation(OpenSearchPPLParser.TrendlineClauseContext ctx) { From c74ac1aa1ff7bb7f7033892f1118ad75f01b5bf6 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 9 Dec 2024 12:06:14 -0800 Subject: [PATCH 04/37] Update import Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 26 +++++++++++++++++-- .../opensearch/sql/ppl/parser/AstBuilder.java | 1 - 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 39a658518..fcd852e94 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -43,11 +43,33 @@ import org.opensearch.sql.ast.expression.ParseMethod; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.ast.expression.WindowFunction; -import org.opensearch.sql.ast.expression.subquery.InSubquery; import org.opensearch.sql.ast.statement.Explain; import org.opensearch.sql.ast.statement.Query; import org.opensearch.sql.ast.statement.Statement; -import org.opensearch.sql.ast.tree.*; +import org.opensearch.sql.ast.tree.Aggregation; +import org.opensearch.sql.ast.tree.Correlation; +import org.opensearch.sql.ast.tree.CountedAggregation; +import org.opensearch.sql.ast.tree.Dedupe; +import org.opensearch.sql.ast.tree.DescribeRelation; +import org.opensearch.sql.ast.tree.Eval; +import org.opensearch.sql.ast.tree.FieldSummary; +import org.opensearch.sql.ast.tree.FillNull; +import org.opensearch.sql.ast.tree.Filter; +import org.opensearch.sql.ast.tree.Flatten; +import org.opensearch.sql.ast.tree.Head; +import org.opensearch.sql.ast.tree.Join; +import org.opensearch.sql.ast.tree.Kmeans; +import org.opensearch.sql.ast.tree.Lookup; +import org.opensearch.sql.ast.tree.Parse; +import org.opensearch.sql.ast.tree.Project; +import org.opensearch.sql.ast.tree.RareAggregation; +import org.opensearch.sql.ast.tree.RareTopN; +import org.opensearch.sql.ast.tree.Relation; +import org.opensearch.sql.ast.tree.Rename; +import org.opensearch.sql.ast.tree.Sort; +import org.opensearch.sql.ast.tree.SubqueryAlias; +import org.opensearch.sql.ast.tree.Trendline; +import org.opensearch.sql.ast.tree.Window; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.ppl.utils.FieldSummaryTransformer; import org.opensearch.sql.ppl.utils.ParseTransformer; diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index 2ca37c5b8..34b455174 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -91,7 +91,6 @@ public AstBuilder(String query) { @Override public UnresolvedPlan visitQueryStatement(OpenSearchPPLParser.QueryStatementContext ctx) { UnresolvedPlan pplCommand = visit(ctx.pplCommands()); - visit(ctx.commands().get(0)); return ctx.commands().stream().map(this::visit).reduce(pplCommand, (r, e) -> e.attach(r)); } From f23c2dbc2c859728107a2b148e583e13e3926d74 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 9 Dec 2024 12:07:19 -0800 Subject: [PATCH 05/37] Update import Signed-off-by: Andy Kwok --- .../java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java | 1 + 1 file changed, 1 insertion(+) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index fcd852e94..75fd4d384 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -47,6 +47,7 @@ import org.opensearch.sql.ast.statement.Query; import org.opensearch.sql.ast.statement.Statement; import org.opensearch.sql.ast.tree.Aggregation; +import org.opensearch.sql.ast.tree.AppendCol; import org.opensearch.sql.ast.tree.Correlation; import org.opensearch.sql.ast.tree.CountedAggregation; import org.opensearch.sql.ast.tree.Dedupe; From 893146a686b7f1f80d24adbc94aca6c43ca4f1e3 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 9 Dec 2024 22:53:50 -0800 Subject: [PATCH 06/37] Update osrt Signed-off-by: Andy Kwok --- .../opensearch/sql/ast/tree/AppendCol.java | 25 ++- .../function/BuiltinFunctionName.java | 1 + .../sql/ppl/CatalystQueryPlanVisitor.java | 158 ++++++++++++++++-- .../opensearch/sql/ppl/parser/AstBuilder.java | 6 +- .../sql/ppl/utils/AggregatorTransformer.java | 2 + .../sql/ppl/utils/TrendlineCatalystUtils.java | 2 +- 6 files changed, 179 insertions(+), 15 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java index 48c2b78f9..92122aa2c 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java @@ -1,22 +1,38 @@ package org.opensearch.sql.ast.tree; +import com.google.common.collect.ImmutableList; +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.Setter; +import lombok.ToString; import org.opensearch.sql.ast.AbstractNodeVisitor; +import org.opensearch.sql.ast.Node; import java.util.List; /** * A composite object which store the subQuery along with some more ad-hoc option like override */ +@Getter +@Setter +@ToString +@EqualsAndHashCode(callSuper = false) +@AllArgsConstructor public class AppendCol extends UnresolvedPlan { public boolean override = true; + + private UnresolvedPlan subSearch; + private UnresolvedPlan child; - public AppendCol(UnresolvedPlan child) { - this.child = child; + public AppendCol(UnresolvedPlan subSearch) { + this.subSearch = subSearch; } @Override @@ -25,6 +41,11 @@ public UnresolvedPlan attach(UnresolvedPlan child) { return this; } + @Override + public List getChild() { + return ImmutableList.of(child); + } + @Override public T accept(AbstractNodeVisitor visitor, C context) { return visitor.visitAppendCol(this, context); diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java index 86970cefb..b101e664c 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java @@ -333,6 +333,7 @@ public FunctionName getName() { .put("percentile", BuiltinFunctionName.PERCENTILE) .put("percentile_approx", BuiltinFunctionName.PERCENTILE_APPROX) .put("approx_count_distinct", BuiltinFunctionName.APPROX_COUNT_DISTINCT) + .put("row_number", BuiltinFunctionName.ROW_NUMBER) .build(); public static Optional of(String str) { diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 75fd4d384..b19733cdf 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -10,13 +10,21 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation; import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; import org.apache.spark.sql.catalyst.expressions.Ascending$; +import org.apache.spark.sql.catalyst.expressions.CurrentRow$; import org.apache.spark.sql.catalyst.expressions.Descending$; import org.apache.spark.sql.catalyst.expressions.Explode; import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.GeneratorOuter; import org.apache.spark.sql.catalyst.expressions.NamedExpression; +import org.apache.spark.sql.catalyst.expressions.RowFrame$; +import org.apache.spark.sql.catalyst.expressions.RowNumber; import org.apache.spark.sql.catalyst.expressions.SortDirection; import org.apache.spark.sql.catalyst.expressions.SortOrder; +import org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame; +import org.apache.spark.sql.catalyst.expressions.UnspecifiedFrame; +import org.apache.spark.sql.catalyst.expressions.UnspecifiedFrame$; +import org.apache.spark.sql.catalyst.expressions.WindowExpression; +import org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition; import org.apache.spark.sql.catalyst.plans.logical.Aggregate; import org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns$; import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation$; @@ -33,14 +41,18 @@ import org.opensearch.flint.spark.FlattenGenerator; import org.opensearch.sql.ast.AbstractNodeVisitor; import org.opensearch.sql.ast.Node; +import org.opensearch.sql.ast.expression.AggregateFunction; import org.opensearch.sql.ast.expression.Alias; import org.opensearch.sql.ast.expression.Argument; +import org.opensearch.sql.ast.expression.Compare; +import org.opensearch.sql.ast.expression.DataType; import org.opensearch.sql.ast.expression.Field; import org.opensearch.sql.ast.expression.Function; import org.opensearch.sql.ast.expression.In; import org.opensearch.sql.ast.expression.Let; import org.opensearch.sql.ast.expression.Literal; import org.opensearch.sql.ast.expression.ParseMethod; +import org.opensearch.sql.ast.expression.QualifiedName; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.ast.expression.WindowFunction; import org.opensearch.sql.ast.statement.Explain; @@ -70,6 +82,7 @@ import org.opensearch.sql.ast.tree.Sort; import org.opensearch.sql.ast.tree.SubqueryAlias; import org.opensearch.sql.ast.tree.Trendline; +import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Window; import org.opensearch.sql.common.antlr.SyntaxCheckException; import org.opensearch.sql.ppl.utils.FieldSummaryTransformer; @@ -83,6 +96,7 @@ import scala.collection.Seq; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -258,17 +272,130 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { - // To-do: - // 1. Fetch the table name from ctx - // 2. Add to child - // - // Append it to the sub query - // Add the rowNumber( ) +// +// // Composite sort clause +// AggregateFunction rowNumber = new AggregateFunction("row_number", new Literal("1", DataType.STRING)); +// Alias rowNumberAlias = new Alias("window_row", rowNumber); +// Field dummySortField = new Field(new QualifiedName("1")); +// +// // Add row Number +// Window window = new Window(of(rowNumberAlias), Collections.emptyList(), of(dummySortField)); +//// windowFunction. +// window.attach((UnresolvedPlan) node.getChild().get(0)); +// +// // Add the alias +// SubqueryAlias t1 = new SubqueryAlias("t1", window); +//// SubqueryAlias t1 = new SubqueryAlias("t1", (UnresolvedPlan) node.getChild().get(0)); +// visitSubqueryAlias(t1, context); - // With the aim to produce a select t1, t2 * query with row_number( ) as join. - System.out.println("Printing node detail:" + node.toString()); - return super.visitAppendCol(node, context); + //---------------------------------------------------------------- + + // T2 + + + // Add row Number +// visitFirstChild(t1, context); + + /* + ------------ Answer ----------------- + 'Project [t1.*, t2.*] + +- 'Join LeftOuter, ('t1.row_org = 't2.row_app) + :- 'SubqueryAlias t1 + : +- 'Project [*, 'row_number() windowspecdefinition(1 ASC NULLS FIRST, unspecifiedframe$()) AS row_org#26] + : +- 'UnresolvedRelation [employees], [], false + +- 'SubqueryAlias t2 + +- 'Project [*, 'row_number() windowspecdefinition(1 ASC NULLS FIRST, unspecifiedframe$()) AS row_app#27] + +- 'UnresolvedRelation [employees], [], false + */ + + System.out.println(context.getPlan()); + /* + + row_number() windowspecdefinition(1 DESC NULLS LAST, unspecifiedframe$()) + + 'Project ['name, 'dept] + +- 'UnresolvedRelation [employees], [], false + */ + + + // To have the row_number( ) on ORG + + + expressionAnalyzer.visitLiteral(new Literal("1", DataType.STRING), context); + Expression strExp = context.popNamedParseExpressions().get(); + SortOrder sortOrder = SortUtils.sortOrder(strExp, false); + WindowSpecDefinition windowDefinition = new WindowSpecDefinition( + seq(), + seq(sortOrder), + UnspecifiedFrame$.MODULE$); + WindowExpression windowExp = new WindowExpression(new RowNumber(), windowDefinition); + + NamedExpression appendCol = TrendlineCatalystUtils.getAlias("APPENDCOL_ID", windowExp); + + + List projectList = new ArrayList<>(); + + if (context.getNamedParseExpressions().isEmpty()) { + // Create an UnresolvedStar for all-fields projection + projectList.add(UnresolvedStar$.MODULE$.apply(Option.empty())); + } + +// + projectList.add(appendCol); + + // Left hand side done. + LogicalPlan queryWIthAppendCol = context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Project(seq( + projectList), p)); + + + // Do the right hand side + + + // Then Join with Spark join api call. + +// +// System.out.println(queryWIthAppendCol); + + /* + 'Project [*, row_number() windowspecdefinition(1 DESC NULLS LAST, unspecifiedframe$()) AS APPENDCOL_ID#8] + +- 'Project ['name, 'dept] + +- 'UnresolvedRelation [employees], [], false + */ + + // Wrap it with T1 + + // Composite the join condition + +// Compare innerJoinCondition = new Compare("=", +// new Field(new QualifiedName("age")), +// new Field(new QualifiedName("age"))); +// +// Join joinNode = new Join(node.getSubSearch(), +// Optional.of("T1"), Optional.of("T2"), +// Join.JoinType.LEFT, Optional.of(innerJoinCondition), null); + +// context.apply(left -> { +// +// LogicalPlan right = joinNode.getRight().accept(this, context); +// +// Optional joinCondition = joinNode.getJoinCondition() +// .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); +// +// context.retainAllNamedParseExpressions(p -> p); +// context.retainAllPlans(p -> p); + +// return join(left, right, joinNode.getJoinType(), joinCondition, joinNode.getJoinHint()); +// }); + + + // Return all. + + + +// System.out.println("Printing node detail:" + node.toString()); +// return super.visitAppendCol(node, context); + return context.getPlan(); } @Override @@ -370,7 +497,18 @@ public LogicalPlan visitWindow(Window node, CatalystPlanContext context) { visitExpression(span, context); } Seq partitionSpec = context.retainAllNamedParseExpressions(p -> p); - Seq orderSpec = seq(new ArrayList()); + // Visit the sort clause, if any. + + Seq orderSpecTemp = null; + if (!Objects.isNull(node.getSortExprList()) && !node.getSortExprList().isEmpty()) { + visitExpressionList(node.getSortExprList(), context); + orderSpecTemp = context.retainAllNamedParseExpressions(exp -> + SortUtils.sortOrder(exp, true)); + } + + Seq orderSpec = (orderSpecTemp != null ) + ?orderSpecTemp :seq(new ArrayList<>()); + Seq aggregatorFunctions = seq( seqAsJavaList(windowFunctionExpressions).stream() .map(w -> WindowSpecTransformer.buildAggregateWindowFunction(w, partitionSpec, orderSpec)) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index 34b455174..6f905e629 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -426,8 +426,10 @@ public UnresolvedPlan visitAppendcolCommand(OpenSearchPPLParser.AppendcolCommand // TD: Pass on AppendColSpecific option also final Optional pplCmd = ctx.commands().stream().map(this::visit).reduce((r, e) -> e.attach(r)); - // Throw some exception. - return pplCmd.map(AppendCol::new).orElse(null); + // Composite the AppendCol object and pass on itstead. + + // Throw some exception. + return new AppendCol(pplCmd.get()); } private Trendline.TrendlineComputation toTrendlineComputation(OpenSearchPPLParser.TrendlineClauseContext ctx) { diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java index c06f37aa3..2b5443f45 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java @@ -59,6 +59,8 @@ static Expression aggregator(org.opensearch.sql.ast.expression.AggregateFunction return new UnresolvedFunction(seq("PERCENTILE_APPROX"), seq(arg, new Literal(getPercentDoubleValue(aggregateFunction), DataTypes.DoubleType)), distinct, empty(),false); case APPROX_COUNT_DISTINCT: return new UnresolvedFunction(seq("APPROX_COUNT_DISTINCT"), seq(arg), distinct, empty(),false); + case ROW_NUMBER: + return new UnresolvedFunction(seq("ROW_NUMBER"), seq(arg), distinct, empty(),false); } throw new IllegalStateException("Not Supported value: " + aggregateFunction.getFuncName()); } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java index 647f4542e..1315165e5 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java @@ -153,7 +153,7 @@ private static NamedExpression getWMAComputationExpression(CatalystExpressionVis * @param expression The expression which will be evaluated. * @return An Alias instance with logical plan representation of `expression AS name`. */ - private static NamedExpression getAlias(String name, Expression expression) { + public static NamedExpression getAlias(String name, Expression expression) { return org.apache.spark.sql.catalyst.expressions.Alias$.MODULE$.apply(expression, name, NamedExpression.newExprId(), From a9f10f034fd5e9074a0a2733bbd7fe2279f7fcb0 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Tue, 10 Dec 2024 17:51:27 -0800 Subject: [PATCH 07/37] Changes Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 160 +++++++++--------- 1 file changed, 82 insertions(+), 78 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index b19733cdf..3127bb338 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -6,6 +6,8 @@ package org.opensearch.sql.ppl; import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$; import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction; import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation; import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; @@ -33,6 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$; import org.apache.spark.sql.catalyst.plans.logical.Project$; +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias$; import org.apache.spark.sql.execution.ExplainMode; import org.apache.spark.sql.execution.command.DescribeTableCommand; import org.apache.spark.sql.execution.command.ExplainCommand; @@ -85,6 +88,7 @@ import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Window; import org.opensearch.sql.common.antlr.SyntaxCheckException; +import org.opensearch.sql.ppl.utils.DataTypeTransformer; import org.opensearch.sql.ppl.utils.FieldSummaryTransformer; import org.opensearch.sql.ppl.utils.ParseTransformer; import org.opensearch.sql.ppl.utils.SortUtils; @@ -270,56 +274,11 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { @Override public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { + visitFirstChild(node, context); -// -// // Composite sort clause -// AggregateFunction rowNumber = new AggregateFunction("row_number", new Literal("1", DataType.STRING)); -// Alias rowNumberAlias = new Alias("window_row", rowNumber); -// Field dummySortField = new Field(new QualifiedName("1")); -// -// // Add row Number -// Window window = new Window(of(rowNumberAlias), Collections.emptyList(), of(dummySortField)); -//// windowFunction. -// window.attach((UnresolvedPlan) node.getChild().get(0)); -// -// // Add the alias -// SubqueryAlias t1 = new SubqueryAlias("t1", window); -//// SubqueryAlias t1 = new SubqueryAlias("t1", (UnresolvedPlan) node.getChild().get(0)); -// visitSubqueryAlias(t1, context); - - - //---------------------------------------------------------------- - - // T2 - - - // Add row Number -// visitFirstChild(t1, context); - - /* - ------------ Answer ----------------- - 'Project [t1.*, t2.*] - +- 'Join LeftOuter, ('t1.row_org = 't2.row_app) - :- 'SubqueryAlias t1 - : +- 'Project [*, 'row_number() windowspecdefinition(1 ASC NULLS FIRST, unspecifiedframe$()) AS row_org#26] - : +- 'UnresolvedRelation [employees], [], false - +- 'SubqueryAlias t2 - +- 'Project [*, 'row_number() windowspecdefinition(1 ASC NULLS FIRST, unspecifiedframe$()) AS row_app#27] - +- 'UnresolvedRelation [employees], [], false - */ System.out.println(context.getPlan()); - /* - - row_number() windowspecdefinition(1 DESC NULLS LAST, unspecifiedframe$()) - - 'Project ['name, 'dept] - +- 'UnresolvedRelation [employees], [], false - */ - - - // To have the row_number( ) on ORG expressionAnalyzer.visitLiteral(new Literal("1", DataType.STRING), context); @@ -333,7 +292,6 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { NamedExpression appendCol = TrendlineCatalystUtils.getAlias("APPENDCOL_ID", windowExp); - List projectList = new ArrayList<>(); if (context.getNamedParseExpressions().isEmpty()) { @@ -349,52 +307,98 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { projectList), p)); + System.out.println("After row_number"); + System.out.println(queryWIthAppendCol); + + // Adding the alias + + LogicalPlan t1 = context.apply(p -> { + var alias = SubqueryAlias$.MODULE$.apply("T1", p); + context.withSubqueryAlias(alias); + return alias; + }); + + System.out.println("------------ T1 --------------"); + System.out.println(t1); + System.out.println(context.getPlan()); + System.out.println("------------ End T1 --------------"); + + // Do the right hand side - // Then Join with Spark join api call. + Node subSearch = node.getSubSearch(); -// -// System.out.println(queryWIthAppendCol); - /* - 'Project [*, row_number() windowspecdefinition(1 DESC NULLS LAST, unspecifiedframe$()) AS APPENDCOL_ID#8] - +- 'Project ['name, 'dept] - +- 'UnresolvedRelation [employees], [], false - */ + // Till traverse till the end then append. + Relation table = new Relation(of(new QualifiedName("employees"))); - // Wrap it with T1 - // Composite the join condition + while (subSearch != null) { + try { + System.out.println("Node: " + subSearch.getClass().getSimpleName()); + Node node1 = subSearch.getChild().get(0); + if (node1 != null) { + System.out.println("Non Null: " + node1.getClass().getSimpleName()); + } else { + System.out.println("Node is null"); + } + subSearch = node1; + } catch (NullPointerException ex) { + System.out.println("Null when getting the child "); + ((UnresolvedPlan) subSearch).attach(table); + break; + } + } + + + // Add a database expression. + + + Compare innerJoinCondition = new Compare("=", + new Field(QualifiedName.of("T1" ,"APPENDCOL_ID")), + new Field(QualifiedName.of("T2", "APPENDCOL_ID"))); + -// Compare innerJoinCondition = new Compare("=", -// new Field(new QualifiedName("age")), -// new Field(new QualifiedName("age"))); -// -// Join joinNode = new Join(node.getSubSearch(), -// Optional.of("T1"), Optional.of("T2"), -// Join.JoinType.LEFT, Optional.of(innerJoinCondition), null); -// context.apply(left -> { -// -// LogicalPlan right = joinNode.getRight().accept(this, context); -// -// Optional joinCondition = joinNode.getJoinCondition() -// .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); -// -// context.retainAllNamedParseExpressions(p -> p); -// context.retainAllPlans(p -> p); -// return join(left, right, joinNode.getJoinType(), joinCondition, joinNode.getJoinHint()); -// }); + context.apply(left -> { + LogicalPlan right = node.getSubSearch().accept(this, context); - // Return all. + // Add the row_number + LogicalPlan t2WithRowNumber = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( + projectList), right); + // To wrap it into T2 + var alias = SubqueryAlias$.MODULE$.apply("T2", t2WithRowNumber); + context.withSubqueryAlias(alias); + + + Optional joinCondition = Optional.of(innerJoinCondition) + .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); + context.retainAllNamedParseExpressions(p -> p); + context.retainAllPlans(p -> p); + LogicalPlan joinedQuery = join(left, alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); + // Remove the APPEND_ID + +// +// List excludeFields = of( +// UnresolvedAttribute$.MODULE$.apply("T1.APPENDCOL_ID"), +// UnresolvedAttribute$.MODULE$.apply("T2.APPENDCOL_ID")); + scala.collection.mutable.Seq seq = seq( + UnresolvedAttribute$.MODULE$.apply("T1.APPENDCOL_ID"), + UnresolvedAttribute$.MODULE$.apply("T2.APPENDCOL_ID")); + return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(seq, joinedQuery); + + }); + + System.out.println("------------ Sub query --------------"); +// System.out.println(right); + System.out.println(context.getPlan()); + System.out.println("------------ End Subquery --------------"); -// System.out.println("Printing node detail:" + node.toString()); -// return super.visitAppendCol(node, context); return context.getPlan(); } From ed71d582c7279bad546b88a6c2af00cd4423b6bd Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 11:49:14 -0800 Subject: [PATCH 08/37] Consolidate String constant Signed-off-by: Andy Kwok --- .../opensearch/sql/ast/tree/AppendCol.java | 3 - .../sql/ppl/CatalystQueryPlanVisitor.java | 77 ++++--------------- .../opensearch/sql/ppl/parser/AstBuilder.java | 14 +--- .../sql/ppl/utils/AggregatorTransformer.java | 2 - 4 files changed, 21 insertions(+), 75 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java index 92122aa2c..e52c37f11 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java @@ -22,15 +22,12 @@ @AllArgsConstructor public class AppendCol extends UnresolvedPlan { - public boolean override = true; - private UnresolvedPlan subSearch; private UnresolvedPlan child; - public AppendCol(UnresolvedPlan subSearch) { this.subSearch = subSearch; } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 3127bb338..83cd05764 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -276,21 +276,23 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { visitFirstChild(node, context); + final String APPENDCOL_ID = "APPENDCOL_ID"; + final String TABLE_LHS = "T1"; + final String TABLE_RHS = "T2"; + final String DUMMY_SORT_FIELD = "1"; - - System.out.println(context.getPlan()); - - - expressionAnalyzer.visitLiteral(new Literal("1", DataType.STRING), context); + expressionAnalyzer.visitLiteral( + new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); Expression strExp = context.popNamedParseExpressions().get(); SortOrder sortOrder = SortUtils.sortOrder(strExp, false); + WindowSpecDefinition windowDefinition = new WindowSpecDefinition( seq(), seq(sortOrder), UnspecifiedFrame$.MODULE$); WindowExpression windowExp = new WindowExpression(new RowNumber(), windowDefinition); - NamedExpression appendCol = TrendlineCatalystUtils.getAlias("APPENDCOL_ID", windowExp); + NamedExpression appendCol = TrendlineCatalystUtils.getAlias(APPENDCOL_ID, windowExp); List projectList = new ArrayList<>(); @@ -299,7 +301,6 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { projectList.add(UnresolvedStar$.MODULE$.apply(Option.empty())); } -// projectList.add(appendCol); // Left hand side done. @@ -307,42 +308,26 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { projectList), p)); - System.out.println("After row_number"); - System.out.println(queryWIthAppendCol); - // Adding the alias LogicalPlan t1 = context.apply(p -> { - var alias = SubqueryAlias$.MODULE$.apply("T1", p); + var alias = SubqueryAlias$.MODULE$.apply(TABLE_LHS, p); context.withSubqueryAlias(alias); return alias; }); - System.out.println("------------ T1 --------------"); - System.out.println(t1); - System.out.println(context.getPlan()); - System.out.println("------------ End T1 --------------"); - - - // Do the right hand side - - Node subSearch = node.getSubSearch(); - // Till traverse till the end then append. Relation table = new Relation(of(new QualifiedName("employees"))); + // Replace it with a function to look up the search command and extract the index name. + while (subSearch != null) { try { System.out.println("Node: " + subSearch.getClass().getSimpleName()); Node node1 = subSearch.getChild().get(0); - if (node1 != null) { - System.out.println("Non Null: " + node1.getClass().getSimpleName()); - } else { - System.out.println("Node is null"); - } subSearch = node1; } catch (NullPointerException ex) { System.out.println("Null when getting the child "); @@ -351,15 +336,9 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { } } - - // Add a database expression. - - Compare innerJoinCondition = new Compare("=", - new Field(QualifiedName.of("T1" ,"APPENDCOL_ID")), - new Field(QualifiedName.of("T2", "APPENDCOL_ID"))); - - + new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), + new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); context.apply(left -> { @@ -371,10 +350,9 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { projectList), right); // To wrap it into T2 - var alias = SubqueryAlias$.MODULE$.apply("T2", t2WithRowNumber); + var alias = SubqueryAlias$.MODULE$.apply(TABLE_RHS, t2WithRowNumber); context.withSubqueryAlias(alias); - Optional joinCondition = Optional.of(innerJoinCondition) .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); context.retainAllNamedParseExpressions(p -> p); @@ -382,23 +360,13 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { LogicalPlan joinedQuery = join(left, alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); // Remove the APPEND_ID -// -// List excludeFields = of( -// UnresolvedAttribute$.MODULE$.apply("T1.APPENDCOL_ID"), -// UnresolvedAttribute$.MODULE$.apply("T2.APPENDCOL_ID")); scala.collection.mutable.Seq seq = seq( - UnresolvedAttribute$.MODULE$.apply("T1.APPENDCOL_ID"), - UnresolvedAttribute$.MODULE$.apply("T2.APPENDCOL_ID")); + UnresolvedAttribute$.MODULE$.apply(TABLE_LHS + "." + APPENDCOL_ID), + UnresolvedAttribute$.MODULE$.apply(TABLE_RHS + "." + APPENDCOL_ID)); return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(seq, joinedQuery); }); - System.out.println("------------ Sub query --------------"); -// System.out.println(right); - System.out.println(context.getPlan()); - System.out.println("------------ End Subquery --------------"); - - return context.getPlan(); } @@ -501,18 +469,7 @@ public LogicalPlan visitWindow(Window node, CatalystPlanContext context) { visitExpression(span, context); } Seq partitionSpec = context.retainAllNamedParseExpressions(p -> p); - // Visit the sort clause, if any. - - Seq orderSpecTemp = null; - if (!Objects.isNull(node.getSortExprList()) && !node.getSortExprList().isEmpty()) { - visitExpressionList(node.getSortExprList(), context); - orderSpecTemp = context.retainAllNamedParseExpressions(exp -> - SortUtils.sortOrder(exp, true)); - } - - Seq orderSpec = (orderSpecTemp != null ) - ?orderSpecTemp :seq(new ArrayList<>()); - + Seq orderSpec = seq(new ArrayList()); Seq aggregatorFunctions = seq( seqAsJavaList(windowFunctionExpressions).stream() .map(w -> WindowSpecTransformer.buildAggregateWindowFunction(w, partitionSpec, orderSpec)) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index 6f905e629..dd9208abe 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -419,16 +419,10 @@ public UnresolvedPlan visitTrendlineCommand(OpenSearchPPLParser.TrendlineCommand @Override public UnresolvedPlan visitAppendcolCommand(OpenSearchPPLParser.AppendcolCommandContext ctx) { - - // All args validation should happen here. - System.out.println("No. of cmd: " + ctx.commands().size()); - - // TD: Pass on AppendColSpecific option also - final Optional pplCmd = ctx.commands().stream().map(this::visit).reduce((r, e) -> e.attach(r)); - - // Composite the AppendCol object and pass on itstead. - - // Throw some exception. + final Optional pplCmd = ctx.commands().stream() + .map(this::visit) + .reduce((r, e) -> e.attach(r)); + // ANTLR parser check guarantee pplCmd won't be null. return new AppendCol(pplCmd.get()); } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java index 2b5443f45..c06f37aa3 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AggregatorTransformer.java @@ -59,8 +59,6 @@ static Expression aggregator(org.opensearch.sql.ast.expression.AggregateFunction return new UnresolvedFunction(seq("PERCENTILE_APPROX"), seq(arg, new Literal(getPercentDoubleValue(aggregateFunction), DataTypes.DoubleType)), distinct, empty(),false); case APPROX_COUNT_DISTINCT: return new UnresolvedFunction(seq("APPROX_COUNT_DISTINCT"), seq(arg), distinct, empty(),false); - case ROW_NUMBER: - return new UnresolvedFunction(seq("ROW_NUMBER"), seq(arg), distinct, empty(),false); } throw new IllegalStateException("Not Supported value: " + aggregateFunction.getFuncName()); } From 3d7b8b17c1ab938557619de055f5b34ac1d9aac0 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 12:13:34 -0800 Subject: [PATCH 09/37] Update projection clause Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 37 ++++++++++--------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 83cd05764..6ff9a42e5 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -276,40 +276,38 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { visitFirstChild(node, context); - final String APPENDCOL_ID = "APPENDCOL_ID"; + final String APPENDCOL_ID = WindowSpecTransformer.ROW_NUMBER_COLUMN_NAME; final String TABLE_LHS = "T1"; final String TABLE_RHS = "T2"; final String DUMMY_SORT_FIELD = "1"; + + // Add a new projection layer with * and ROW_NUMBER (Main-search) + // Inject an addition search command into sub-search + // Add a new projection layer with * and ROW_NUMBER (Sub-search) + + + // Add a new projection layer with * and ROW_NUMBER (Main-search) + expressionAnalyzer.visitLiteral( new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); Expression strExp = context.popNamedParseExpressions().get(); SortOrder sortOrder = SortUtils.sortOrder(strExp, false); - WindowSpecDefinition windowDefinition = new WindowSpecDefinition( - seq(), - seq(sortOrder), - UnspecifiedFrame$.MODULE$); - WindowExpression windowExp = new WindowExpression(new RowNumber(), windowDefinition); - - NamedExpression appendCol = TrendlineCatalystUtils.getAlias(APPENDCOL_ID, windowExp); - - List projectList = new ArrayList<>(); - - if (context.getNamedParseExpressions().isEmpty()) { - // Create an UnresolvedStar for all-fields projection - projectList.add(UnresolvedStar$.MODULE$.apply(Option.empty())); - } + NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); - projectList.add(appendCol); + List projectList = (context.getNamedParseExpressions().isEmpty()) + ? List.of(appendCol, UnresolvedStar$.MODULE$.apply(Option.empty())) + : List.of(appendCol); // Left hand side done. LogicalPlan queryWIthAppendCol = context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Project(seq( projectList), p)); + // Compile the Left hand side separately. - // Adding the alias + // Adding the alias LogicalPlan t1 = context.apply(p -> { var alias = SubqueryAlias$.MODULE$.apply(TABLE_LHS, p); context.withSubqueryAlias(alias); @@ -323,7 +321,6 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { // Replace it with a function to look up the search command and extract the index name. - while (subSearch != null) { try { System.out.println("Node: " + subSearch.getClass().getSimpleName()); @@ -336,6 +333,7 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { } } + Compare innerJoinCondition = new Compare("=", new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); @@ -367,6 +365,9 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { }); + + System.out.println(context); + return context.getPlan(); } From 77fddf1a0a6e39205bf4efbb90022d7150a63226 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 12:56:34 -0800 Subject: [PATCH 10/37] Remove dep on parent method Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 48 ++++++++++--------- 1 file changed, 25 insertions(+), 23 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 6ff9a42e5..3a14e3541 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -41,6 +41,7 @@ import org.apache.spark.sql.execution.command.ExplainCommand; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.jetbrains.annotations.NotNull; import org.opensearch.flint.spark.FlattenGenerator; import org.opensearch.sql.ast.AbstractNodeVisitor; import org.opensearch.sql.ast.Node; @@ -274,45 +275,32 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { @Override public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { - visitFirstChild(node, context); + final String APPENDCOL_ID = WindowSpecTransformer.ROW_NUMBER_COLUMN_NAME; final String TABLE_LHS = "T1"; final String TABLE_RHS = "T2"; final String DUMMY_SORT_FIELD = "1"; - // Add a new projection layer with * and ROW_NUMBER (Main-search) // Inject an addition search command into sub-search // Add a new projection layer with * and ROW_NUMBER (Sub-search) + LogicalPlan leftTemp = node.getChild().get(0).accept(this, context); // Add a new projection layer with * and ROW_NUMBER (Main-search) + List projectList = getRowNumStarProjection(context, DUMMY_SORT_FIELD); - expressionAnalyzer.visitLiteral( - new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); - Expression strExp = context.popNamedParseExpressions().get(); - SortOrder sortOrder = SortUtils.sortOrder(strExp, false); - - NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); + // Add the row_number + LogicalPlan t1WithRowNumber = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( + projectList), leftTemp); - List projectList = (context.getNamedParseExpressions().isEmpty()) - ? List.of(appendCol, UnresolvedStar$.MODULE$.apply(Option.empty())) - : List.of(appendCol); - - // Left hand side done. - LogicalPlan queryWIthAppendCol = context.apply(p -> new org.apache.spark.sql.catalyst.plans.logical.Project(seq( - projectList), p)); + // To wrap it into T2 + var t1Table = SubqueryAlias$.MODULE$.apply(TABLE_LHS, t1WithRowNumber); + context.withSubqueryAlias(t1Table); - // Compile the Left hand side separately. - // Adding the alias - LogicalPlan t1 = context.apply(p -> { - var alias = SubqueryAlias$.MODULE$.apply(TABLE_LHS, p); - context.withSubqueryAlias(alias); - return alias; - }); Node subSearch = node.getSubSearch(); @@ -355,7 +343,7 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - LogicalPlan joinedQuery = join(left, alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); + LogicalPlan joinedQuery = join(t1Table, alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); // Remove the APPEND_ID scala.collection.mutable.Seq seq = seq( @@ -371,6 +359,20 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { return context.getPlan(); } + private @NotNull List getRowNumStarProjection(CatalystPlanContext context, String DUMMY_SORT_FIELD) { + expressionAnalyzer.visitLiteral( + new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); + Expression strExp = context.popNamedParseExpressions().get(); + SortOrder sortOrder = SortUtils.sortOrder(strExp, false); + + NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); + + List projectList = (context.getNamedParseExpressions().isEmpty()) + ? List.of(appendCol, UnresolvedStar$.MODULE$.apply(Option.empty())) + : List.of(appendCol); + return projectList; + } + @Override public LogicalPlan visitCorrelation(Correlation node, CatalystPlanContext context) { visitFirstChild(node, context); From 0e7e65ab019ee7dff84974392abe0b3782fb0d71 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 13:09:55 -0800 Subject: [PATCH 11/37] Consolidate relation inject logic Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 3a14e3541..744f833bd 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -286,48 +286,28 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { // Inject an addition search command into sub-search // Add a new projection layer with * and ROW_NUMBER (Sub-search) - LogicalPlan leftTemp = node.getChild().get(0).accept(this, context); // Add a new projection layer with * and ROW_NUMBER (Main-search) + LogicalPlan leftTemp = node.getChild().get(0).accept(this, context); List projectList = getRowNumStarProjection(context, DUMMY_SORT_FIELD); // Add the row_number LogicalPlan t1WithRowNumber = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( projectList), leftTemp); - // To wrap it into T2 + // To wrap it into T1 var t1Table = SubqueryAlias$.MODULE$.apply(TABLE_LHS, t1WithRowNumber); context.withSubqueryAlias(t1Table); + context.apply(left -> { + Compare innerJoinCondition = new Compare("=", + new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), + new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); - Node subSearch = node.getSubSearch(); - - // Till traverse till the end then append. - Relation table = new Relation(of(new QualifiedName("employees"))); - // Replace it with a function to look up the search command and extract the index name. - - - while (subSearch != null) { - try { - System.out.println("Node: " + subSearch.getClass().getSimpleName()); - Node node1 = subSearch.getChild().get(0); - subSearch = node1; - } catch (NullPointerException ex) { - System.out.println("Null when getting the child "); - ((UnresolvedPlan) subSearch).attach(table); - break; - } - } - - - Compare innerJoinCondition = new Compare("=", - new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), - new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); - - - context.apply(left -> { + // Inject an addition search command into sub-search (T2) + addSearchCmd(node.getSubSearch(), "employees"); LogicalPlan right = node.getSubSearch().accept(this, context); @@ -359,6 +339,26 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { return context.getPlan(); } + private static void addSearchCmd(Node subSearch, String relationName) { + + // Till traverse till the end then append. + Relation table = new Relation(of(new QualifiedName(relationName))); + // Replace it with a function to look up the search command and extract the index name. + + + while (subSearch != null) { + try { + System.out.println("Node: " + subSearch.getClass().getSimpleName()); + subSearch = subSearch.getChild().get(0); +// subSearch = node1; + } catch (NullPointerException ex) { + System.out.println("Null when getting the child "); + ((UnresolvedPlan) subSearch).attach(table); + break; + } + } + } + private @NotNull List getRowNumStarProjection(CatalystPlanContext context, String DUMMY_SORT_FIELD) { expressionAnalyzer.visitLiteral( new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); From d2aa1466e0299b77f8930e1ca7603ab6cbacecb4 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 13:44:58 -0800 Subject: [PATCH 12/37] Move constant Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 744f833bd..dcf89a16c 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -280,7 +280,7 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { final String APPENDCOL_ID = WindowSpecTransformer.ROW_NUMBER_COLUMN_NAME; final String TABLE_LHS = "T1"; final String TABLE_RHS = "T2"; - final String DUMMY_SORT_FIELD = "1"; + final List projectList = getRowNumStarProjection(context); // Add a new projection layer with * and ROW_NUMBER (Main-search) // Inject an addition search command into sub-search @@ -289,7 +289,7 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { // Add a new projection layer with * and ROW_NUMBER (Main-search) LogicalPlan leftTemp = node.getChild().get(0).accept(this, context); - List projectList = getRowNumStarProjection(context, DUMMY_SORT_FIELD); + // Add the row_number LogicalPlan t1WithRowNumber = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( @@ -316,14 +316,14 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { projectList), right); // To wrap it into T2 - var alias = SubqueryAlias$.MODULE$.apply(TABLE_RHS, t2WithRowNumber); - context.withSubqueryAlias(alias); + var t2Alias = SubqueryAlias$.MODULE$.apply(TABLE_RHS, t2WithRowNumber); + context.withSubqueryAlias(t2Alias); Optional joinCondition = Optional.of(innerJoinCondition) .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - LogicalPlan joinedQuery = join(t1Table, alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); + LogicalPlan joinedQuery = join(t1Table, t2Alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); // Remove the APPEND_ID scala.collection.mutable.Seq seq = seq( @@ -334,6 +334,7 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { }); + System.out.println(context); return context.getPlan(); @@ -359,7 +360,10 @@ private static void addSearchCmd(Node subSearch, String relationName) { } } - private @NotNull List getRowNumStarProjection(CatalystPlanContext context, String DUMMY_SORT_FIELD) { + private @NotNull List getRowNumStarProjection(CatalystPlanContext context) { + + final String DUMMY_SORT_FIELD = "1"; + expressionAnalyzer.visitLiteral( new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); Expression strExp = context.popNamedParseExpressions().get(); From 477c4fc13bcc31465643a5fd62f2a46e10936992 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 13:48:20 -0800 Subject: [PATCH 13/37] Move out constant from lambda Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index dcf89a16c..c9c9636be 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -281,6 +281,13 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { final String TABLE_LHS = "T1"; final String TABLE_RHS = "T2"; final List projectList = getRowNumStarProjection(context); + scala.collection.mutable.Seq fieldsToRemove = seq( + UnresolvedAttribute$.MODULE$.apply(TABLE_LHS + "." + APPENDCOL_ID), + UnresolvedAttribute$.MODULE$.apply(TABLE_RHS + "." + APPENDCOL_ID)); + final Compare innerJoinCondition = new Compare("=", + new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), + new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); + // Add a new projection layer with * and ROW_NUMBER (Main-search) // Inject an addition search command into sub-search @@ -302,10 +309,6 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { context.apply(left -> { - Compare innerJoinCondition = new Compare("=", - new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), - new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); - // Inject an addition search command into sub-search (T2) addSearchCmd(node.getSubSearch(), "employees"); @@ -321,15 +324,13 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { Optional joinCondition = Optional.of(innerJoinCondition) .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); + context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); LogicalPlan joinedQuery = join(t1Table, t2Alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); - // Remove the APPEND_ID - scala.collection.mutable.Seq seq = seq( - UnresolvedAttribute$.MODULE$.apply(TABLE_LHS + "." + APPENDCOL_ID), - UnresolvedAttribute$.MODULE$.apply(TABLE_RHS + "." + APPENDCOL_ID)); - return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(seq, joinedQuery); + // Remove the APPEND_ID + return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(fieldsToRemove, joinedQuery); }); From 662a57cb9ba11504eaa9c7bc8c72a3b9e8af07f2 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 15:15:11 -0800 Subject: [PATCH 14/37] Consolidate method Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 68 +++++-------------- 1 file changed, 17 insertions(+), 51 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index c9c9636be..bd60f8224 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -6,27 +6,18 @@ package org.opensearch.sql.ppl; import org.apache.spark.sql.catalyst.TableIdentifier; -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$; import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction; import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation; import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; import org.apache.spark.sql.catalyst.expressions.Ascending$; -import org.apache.spark.sql.catalyst.expressions.CurrentRow$; import org.apache.spark.sql.catalyst.expressions.Descending$; import org.apache.spark.sql.catalyst.expressions.Explode; import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.GeneratorOuter; import org.apache.spark.sql.catalyst.expressions.NamedExpression; -import org.apache.spark.sql.catalyst.expressions.RowFrame$; -import org.apache.spark.sql.catalyst.expressions.RowNumber; import org.apache.spark.sql.catalyst.expressions.SortDirection; import org.apache.spark.sql.catalyst.expressions.SortOrder; -import org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame; -import org.apache.spark.sql.catalyst.expressions.UnspecifiedFrame; -import org.apache.spark.sql.catalyst.expressions.UnspecifiedFrame$; -import org.apache.spark.sql.catalyst.expressions.WindowExpression; -import org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition; import org.apache.spark.sql.catalyst.plans.logical.Aggregate; import org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns$; import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation$; @@ -41,11 +32,9 @@ import org.apache.spark.sql.execution.command.ExplainCommand; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.jetbrains.annotations.NotNull; import org.opensearch.flint.spark.FlattenGenerator; import org.opensearch.sql.ast.AbstractNodeVisitor; import org.opensearch.sql.ast.Node; -import org.opensearch.sql.ast.expression.AggregateFunction; import org.opensearch.sql.ast.expression.Alias; import org.opensearch.sql.ast.expression.Argument; import org.opensearch.sql.ast.expression.Compare; @@ -89,7 +78,6 @@ import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Window; import org.opensearch.sql.common.antlr.SyntaxCheckException; -import org.opensearch.sql.ppl.utils.DataTypeTransformer; import org.opensearch.sql.ppl.utils.FieldSummaryTransformer; import org.opensearch.sql.ppl.utils.ParseTransformer; import org.opensearch.sql.ppl.utils.SortUtils; @@ -101,7 +89,6 @@ import scala.collection.Seq; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -276,11 +263,9 @@ public LogicalPlan visitTrendline(Trendline node, CatalystPlanContext context) { @Override public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { - final String APPENDCOL_ID = WindowSpecTransformer.ROW_NUMBER_COLUMN_NAME; final String TABLE_LHS = "T1"; final String TABLE_RHS = "T2"; - final List projectList = getRowNumStarProjection(context); scala.collection.mutable.Seq fieldsToRemove = seq( UnresolvedAttribute$.MODULE$.apply(TABLE_LHS + "." + APPENDCOL_ID), UnresolvedAttribute$.MODULE$.apply(TABLE_RHS + "." + APPENDCOL_ID)); @@ -288,60 +273,37 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); - - // Add a new projection layer with * and ROW_NUMBER (Main-search) - // Inject an addition search command into sub-search - // Add a new projection layer with * and ROW_NUMBER (Sub-search) - - // Add a new projection layer with * and ROW_NUMBER (Main-search) LogicalPlan leftTemp = node.getChild().get(0).accept(this, context); + var mainSearch = getRowNumStarProjection(context, leftTemp, TABLE_LHS); + context.withSubqueryAlias(mainSearch); - - // Add the row_number - LogicalPlan t1WithRowNumber = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( - projectList), leftTemp); - - // To wrap it into T1 - var t1Table = SubqueryAlias$.MODULE$.apply(TABLE_LHS, t1WithRowNumber); - context.withSubqueryAlias(t1Table); - + // Inject an addition search command into sub-search (T2) + appendRelationClause(node.getSubSearch(), "employees"); context.apply(left -> { - // Inject an addition search command into sub-search (T2) - addSearchCmd(node.getSubSearch(), "employees"); - + // Add a new projection layer with * and ROW_NUMBER (Sub-search) LogicalPlan right = node.getSubSearch().accept(this, context); - - // Add the row_number - LogicalPlan t2WithRowNumber = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( - projectList), right); - - // To wrap it into T2 - var t2Alias = SubqueryAlias$.MODULE$.apply(TABLE_RHS, t2WithRowNumber); - context.withSubqueryAlias(t2Alias); + var subSearch = getRowNumStarProjection(context, right, TABLE_RHS); + context.withSubqueryAlias(subSearch); Optional joinCondition = Optional.of(innerJoinCondition) .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); - context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - LogicalPlan joinedQuery = join(t1Table, t2Alias, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); + + LogicalPlan joinedQuery = join(mainSearch, subSearch, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); // Remove the APPEND_ID return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(fieldsToRemove, joinedQuery); - }); - - - System.out.println(context); - +// System.out.println(context); return context.getPlan(); } - private static void addSearchCmd(Node subSearch, String relationName) { + private static void appendRelationClause(Node subSearch, String relationName) { // Till traverse till the end then append. Relation table = new Relation(of(new QualifiedName(relationName))); @@ -361,7 +323,7 @@ private static void addSearchCmd(Node subSearch, String relationName) { } } - private @NotNull List getRowNumStarProjection(CatalystPlanContext context) { + private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { final String DUMMY_SORT_FIELD = "1"; @@ -375,7 +337,11 @@ private static void addSearchCmd(Node subSearch, String relationName) { List projectList = (context.getNamedParseExpressions().isEmpty()) ? List.of(appendCol, UnresolvedStar$.MODULE$.apply(Option.empty())) : List.of(appendCol); - return projectList; + + LogicalPlan lpWithProjection = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( + projectList), lp); + return SubqueryAlias$.MODULE$.apply(alias, lpWithProjection); + } @Override From 365cc12cb92fde1ace5fcb4c2eb624b4b39fca1e Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Wed, 11 Dec 2024 15:59:57 -0800 Subject: [PATCH 15/37] Update logic Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index bd60f8224..1652961e9 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -278,11 +278,11 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { var mainSearch = getRowNumStarProjection(context, leftTemp, TABLE_LHS); context.withSubqueryAlias(mainSearch); - // Inject an addition search command into sub-search (T2) - appendRelationClause(node.getSubSearch(), "employees"); + // Traverse to look for relation clause then append it into the sub-search. + Relation relation = retrieveRelationClause(node.getChild().get(0)); + appendRelationClause(node.getSubSearch(), relation); context.apply(left -> { - // Add a new projection layer with * and ROW_NUMBER (Sub-search) LogicalPlan right = node.getSubSearch().accept(this, context); var subSearch = getRowNumStarProjection(context, right, TABLE_RHS); @@ -298,23 +298,16 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { // Remove the APPEND_ID return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(fieldsToRemove, joinedQuery); }); - -// System.out.println(context); return context.getPlan(); } - private static void appendRelationClause(Node subSearch, String relationName) { + private static void appendRelationClause(Node subSearch, Relation relation) { - // Till traverse till the end then append. - Relation table = new Relation(of(new QualifiedName(relationName))); + Relation table = new Relation(relation.getTableNames()); // Replace it with a function to look up the search command and extract the index name. - - while (subSearch != null) { try { - System.out.println("Node: " + subSearch.getClass().getSimpleName()); subSearch = subSearch.getChild().get(0); -// subSearch = node1; } catch (NullPointerException ex) { System.out.println("Null when getting the child "); ((UnresolvedPlan) subSearch).attach(table); @@ -323,6 +316,22 @@ private static void appendRelationClause(Node subSearch, String relationName) { } } + private static Relation retrieveRelationClause(Node node) { + while (node != null) { + if (node instanceof Relation) { + return (Relation) node; + } else { + try { + node = node.getChild().get(0); + } catch (NullPointerException ex) { + // NPE will be thrown by some node.getChild() call. + break; + } + } + } + return null; + } + private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { final String DUMMY_SORT_FIELD = "1"; From 351ea88817efc28c38185e6be7284562dad0eeca Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Thu, 12 Dec 2024 15:09:27 -0800 Subject: [PATCH 16/37] Test 1 2 Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 1 + ...nAppendColCommandTranslatorTestSuite.scala | 184 ++++++++++++++++++ 2 files changed, 185 insertions(+) create mode 100644 ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 1652961e9..53e529575 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -339,6 +339,7 @@ private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarP expressionAnalyzer.visitLiteral( new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); Expression strExp = context.popNamedParseExpressions().get(); + // Literal("x") SortOrder sortOrder = SortUtils.sortOrder(strExp, false); NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala new file mode 100644 index 000000000..ef9a26ae1 --- /dev/null +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -0,0 +1,184 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.spark.ppl + +import org.opensearch.flint.spark.ppl.PlaneUtils.plan +import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} +import org.opensearch.sql.ppl.utils.SortUtils + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.{ + UnresolvedAttribute, + UnresolvedFunction, + UnresolvedRelation, + UnresolvedStar +} +import org.apache.spark.sql.catalyst.expressions.{ + Alias, + And, + CurrentRow, + EqualTo, + Literal, + RowFrame, + RowNumber, + SpecifiedWindowFrame, + UnboundedPreceding, + WindowExpression, + WindowSpecDefinition +} +import org.apache.spark.sql.catalyst.plans.{LeftOuter, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical._ + +import org.scalatest.matchers.should.Matchers + +class PPLLogicalPlanAppendColCommandTranslatorTestSuite + extends SparkFunSuite + with PlanTest + with LogicalPlanTestUtils + with Matchers { + + private val planTransformer = new CatalystQueryPlanVisitor() + private val pplParser = new PPLSyntaxParser() + + private val ROW_NUMBER_AGGREGATION = Alias( + WindowExpression( + RowNumber(), + WindowSpecDefinition( + Nil, + SortUtils.sortOrder(Literal("1"), false) :: Nil, + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), + "_row_number_")() + + private val COUNT_STAR = Alias( + UnresolvedFunction(Seq("COUNT"), + Seq(UnresolvedStar(None)), + isDistinct = false), + "count()")() + + private val AGE_ALIAS = Alias(UnresolvedAttribute("age"), "age")() + + private val RELATION_EMPLOYEES = UnresolvedRelation(Seq("employees")) + + private val T12_JOIN_CONDITION = EqualTo( + UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + + private val T12_COLUMNS_SEQ = Seq( + UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + + /** + * Expected: + 'Project [*] + +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#1, *] + : +- 'UnresolvedRelation [employees], [], false + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#5, *] + +- 'Aggregate ['age AS age#3], ['COUNT(*) AS count()#2, 'age AS age#3] + +- 'UnresolvedRelation [employees], [], false + */ + test("test AppendCol with NO transformation on main") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan(pplParser, "source=employees | APPENDCOL [stats count() by age];"), + context + ) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias("T1", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + RELATION_EMPLOYEES)) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] + +- 'UnresolvedRelation [relation], [], false + */ + val t2 = SubqueryAlias("T2", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate( + AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), + RELATION_EMPLOYEES) + )) + + val result = Project(Seq(UnresolvedStar(None)), + DataFrameDropColumns(T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + comparePlans(logicalPlan, result, checkAnalysis = false) + } + + /** + * 'Project [*] + * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * :- 'SubqueryAlias T1 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + * : +- 'Project ['age, 'dept, 'salary] + * : +- 'UnresolvedRelation [relation], [], false + * +- 'SubqueryAlias T2 + * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#15, *] + * +- 'Aggregate ['age AS age#13], ['COUNT(*) AS count()#12, 'age AS age#13] + * +- 'UnresolvedRelation [relation], [], false + */ + test("test AppendCol with transformation on main-search") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan(pplParser, "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), + context + ) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['age, 'dept, 'salary] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias("T1", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq( + UnresolvedAttribute("age"), + UnresolvedAttribute("dept"), + UnresolvedAttribute("salary")), RELATION_EMPLOYEES))) + + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] + +- 'UnresolvedRelation [relation], [], false + */ + val t2 = SubqueryAlias("T2", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate( + AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), + RELATION_EMPLOYEES) + )) + + val result = Project(Seq(UnresolvedStar(None)), + DataFrameDropColumns(T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + // scalastyle:off + println(logicalPlan) + println(result) + + comparePlans(logicalPlan, result, checkAnalysis = false) + + } + + + +} From 16406a0ad8c45c75c202fffa41229e8d48c6ae7c Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Thu, 12 Dec 2024 16:25:33 -0800 Subject: [PATCH 17/37] Test-cases 3 and 4 Signed-off-by: Andy Kwok --- ...nAppendColCommandTranslatorTestSuite.scala | 149 +++++++++++++++++- 1 file changed, 145 insertions(+), 4 deletions(-) diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index ef9a26ae1..847427f5c 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -171,14 +171,155 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite DataFrameDropColumns(T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - // scalastyle:off - println(logicalPlan) - println(result) - comparePlans(logicalPlan, result, checkAnalysis = false) + } + + /** + * 'Project [*] + * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * :- 'SubqueryAlias T1 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#427, *] + * : +- 'Project ['age, 'dept, 'salary] + * : +- 'UnresolvedRelation [employees], [], false + * +- 'SubqueryAlias T2 + * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + * +- 'DataFrameDropColumns ['m] + * +- 'Project [*, 1 AS m#430] + * +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + * +- 'UnresolvedRelation [employees], [], false + */ + test("test AppendCol with chained sub-search") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan(pplParser, "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), + context + ) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['age, 'dept, 'salary] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias("T1", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq( + UnresolvedAttribute("age"), + UnresolvedAttribute("dept"), + UnresolvedAttribute("salary")), RELATION_EMPLOYEES))) + + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + +- 'DataFrameDropColumns ['m] + +- 'Project [*, 1 AS m#430] + +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + +- 'UnresolvedRelation [employees], [], false + */ + val t2 = SubqueryAlias("T2", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns(Seq(UnresolvedAttribute("m")), + Project(Seq(UnresolvedStar(None), Alias(Literal(1),"m")()), + Aggregate( + AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), + RELATION_EMPLOYEES))) + )) + + val result = Project(Seq(UnresolvedStar(None)), + DataFrameDropColumns(T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + comparePlans(logicalPlan, result, checkAnalysis = false) } + /** + * == Parsed Logical Plan == + * 'Project [*] + * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * :- 'SubqueryAlias T1 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#551, *] + * : +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * : +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * : :- 'SubqueryAlias T1 + * : : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] + * : : +- 'Project ['name, 'age] + * : : +- 'UnresolvedRelation [employees], [], false + * : +- 'SubqueryAlias T2 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#549, *] + * : +- 'DataFrameDropColumns ['m] + * : +- 'Project [*, 1 AS m#547] + * : +- 'Aggregate ['age AS age#546], ['COUNT(*) AS count()#545, 'age AS age#546] + * : +- 'UnresolvedRelation [employees], [], false + * +- 'SubqueryAlias T2 + * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] + * +- 'Project ['dept] + * +- 'UnresolvedRelation [employees], [], false + */ + test("test multiple AppendCol clauses") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan(pplParser, "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), + context + ) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] + : +- 'Project ['name, 'age] + : +- 'UnresolvedRelation [employees], [], false + */ + val mainSearch = SubqueryAlias("T1", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq( + UnresolvedAttribute("name"), + UnresolvedAttribute("age")), RELATION_EMPLOYEES))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + +- 'DataFrameDropColumns ['m] + +- 'Project [*, 1 AS m#430] + +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + +- 'UnresolvedRelation [employees], [], false + */ + val firstAppenCol = SubqueryAlias("T2", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns(Seq(UnresolvedAttribute("m")), + Project(Seq(UnresolvedStar(None), Alias(Literal(1),"m")()), + Aggregate( + AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), + RELATION_EMPLOYEES))) + )) + + + val joinWithFirstAppendCol = SubqueryAlias("T1", Project(Seq( + ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns(T12_COLUMNS_SEQ, + Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) + + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] + +- 'Project ['dept] + +- 'UnresolvedRelation [employees], [], false + */ + val secondAppendCol = SubqueryAlias("T2", Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq( + UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) + + + val joinWithSecondAppendCol = Project(Seq(UnresolvedStar(None)), + DataFrameDropColumns(T12_COLUMNS_SEQ, + Join(joinWithFirstAppendCol, secondAppendCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) + } } From 13f4cb98f2880371eb6defa28abbaacead63cf9a Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Thu, 12 Dec 2024 16:34:46 -0800 Subject: [PATCH 18/37] Update code format Signed-off-by: Andy Kwok --- ...PLLogicalPlanAppendColCommandTranslatorTestSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index 847427f5c..ead6ac8cc 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -68,6 +68,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite private val T12_COLUMNS_SEQ = Seq( UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + // @formatter:off /** * Expected: 'Project [*] @@ -81,6 +82,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite +- 'Aggregate ['age AS age#3], ['COUNT(*) AS count()#2, 'age AS age#3] +- 'UnresolvedRelation [employees], [], false */ + // @formatter:on test("test AppendCol with NO transformation on main") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( @@ -118,6 +120,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite comparePlans(logicalPlan, result, checkAnalysis = false) } + // @formatter:off /** * 'Project [*] * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] @@ -131,6 +134,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite * +- 'Aggregate ['age AS age#13], ['COUNT(*) AS count()#12, 'age AS age#13] * +- 'UnresolvedRelation [relation], [], false */ + // @formatter:on test("test AppendCol with transformation on main-search") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( @@ -174,6 +178,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite comparePlans(logicalPlan, result, checkAnalysis = false) } + // @formatter:off /** * 'Project [*] * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] @@ -189,6 +194,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite * +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] * +- 'UnresolvedRelation [employees], [], false */ + // @formatter:on test("test AppendCol with chained sub-search") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( @@ -235,6 +241,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite comparePlans(logicalPlan, result, checkAnalysis = false) } + // @formatter:off /** * == Parsed Logical Plan == * 'Project [*] @@ -259,6 +266,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite * +- 'Project ['dept] * +- 'UnresolvedRelation [employees], [], false */ + // @formatter:on test("test multiple AppendCol clauses") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( From d34abf15fb2b6a5645eff36f92edbff9bd7f2ee1 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Thu, 12 Dec 2024 16:35:38 -0800 Subject: [PATCH 19/37] Update code style Signed-off-by: Andy Kwok --- ...nAppendColCommandTranslatorTestSuite.scala | 233 +++++++++--------- 1 file changed, 118 insertions(+), 115 deletions(-) diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index ead6ac8cc..c41518645 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -8,32 +8,14 @@ package org.opensearch.flint.spark.ppl import org.opensearch.flint.spark.ppl.PlaneUtils.plan import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} import org.opensearch.sql.ppl.utils.SortUtils +import org.scalatest.matchers.should.Matchers import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.analysis.{ - UnresolvedAttribute, - UnresolvedFunction, - UnresolvedRelation, - UnresolvedStar -} -import org.apache.spark.sql.catalyst.expressions.{ - Alias, - And, - CurrentRow, - EqualTo, - Literal, - RowFrame, - RowNumber, - SpecifiedWindowFrame, - UnboundedPreceding, - WindowExpression, - WindowSpecDefinition -} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, CurrentRow, EqualTo, Literal, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.{LeftOuter, PlanTest} import org.apache.spark.sql.catalyst.plans.logical._ -import org.scalatest.matchers.should.Matchers - class PPLLogicalPlanAppendColCommandTranslatorTestSuite extends SparkFunSuite with PlanTest @@ -53,20 +35,18 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite "_row_number_")() private val COUNT_STAR = Alias( - UnresolvedFunction(Seq("COUNT"), - Seq(UnresolvedStar(None)), - isDistinct = false), + UnresolvedFunction(Seq("COUNT"), Seq(UnresolvedStar(None)), isDistinct = false), "count()")() private val AGE_ALIAS = Alias(UnresolvedAttribute("age"), "age")() private val RELATION_EMPLOYEES = UnresolvedRelation(Seq("employees")) - private val T12_JOIN_CONDITION = EqualTo( - UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + private val T12_JOIN_CONDITION = + EqualTo(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) - private val T12_COLUMNS_SEQ = Seq( - UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + private val T12_COLUMNS_SEQ = + Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) // @formatter:off /** @@ -87,17 +67,16 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( plan(pplParser, "source=employees | APPENDCOL [stats count() by age];"), - context - ) + context) /* :- 'SubqueryAlias T1 : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] : +- 'UnresolvedRelation [relation], [], false */ - val t1 = SubqueryAlias("T1", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - RELATION_EMPLOYEES)) + val t1 = SubqueryAlias( + "T1", + Project(Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), RELATION_EMPLOYEES)) /* +- 'SubqueryAlias T2 @@ -106,16 +85,17 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] +- 'UnresolvedRelation [relation], [], false */ - val t2 = SubqueryAlias("T2", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Aggregate( - AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), - RELATION_EMPLOYEES) - )) + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) - val result = Project(Seq(UnresolvedStar(None)), - DataFrameDropColumns(T12_COLUMNS_SEQ, - Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + val result = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) comparePlans(logicalPlan, result, checkAnalysis = false) } @@ -138,9 +118,10 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite test("test AppendCol with transformation on main-search") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( - plan(pplParser, "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), - context - ) + plan( + pplParser, + "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), + context) /* :- 'SubqueryAlias T1 @@ -149,13 +130,16 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite : +- 'Project ['age, 'dept, 'salary] : +- 'UnresolvedRelation [relation], [], false */ - val t1 = SubqueryAlias("T1", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project(Seq( - UnresolvedAttribute("age"), - UnresolvedAttribute("dept"), - UnresolvedAttribute("salary")), RELATION_EMPLOYEES))) - + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("age"), + UnresolvedAttribute("dept"), + UnresolvedAttribute("salary")), + RELATION_EMPLOYEES))) /* +- 'SubqueryAlias T2 @@ -164,15 +148,16 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] +- 'UnresolvedRelation [relation], [], false */ - val t2 = SubqueryAlias("T2", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Aggregate( - AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), - RELATION_EMPLOYEES) - )) - - val result = Project(Seq(UnresolvedStar(None)), - DataFrameDropColumns(T12_COLUMNS_SEQ, + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) + + val result = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) comparePlans(logicalPlan, result, checkAnalysis = false) @@ -198,9 +183,10 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite test("test AppendCol with chained sub-search") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( - plan(pplParser, "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), - context - ) + plan( + pplParser, + "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), + context) /* :- 'SubqueryAlias T1 @@ -209,13 +195,16 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite : +- 'Project ['age, 'dept, 'salary] : +- 'UnresolvedRelation [relation], [], false */ - val t1 = SubqueryAlias("T1", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project(Seq( - UnresolvedAttribute("age"), - UnresolvedAttribute("dept"), - UnresolvedAttribute("salary")), RELATION_EMPLOYEES))) - + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("age"), + UnresolvedAttribute("dept"), + UnresolvedAttribute("salary")), + RELATION_EMPLOYEES))) /* +- 'SubqueryAlias T2 @@ -225,17 +214,20 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] +- 'UnresolvedRelation [employees], [], false */ - val t2 = SubqueryAlias("T2", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - DataFrameDropColumns(Seq(UnresolvedAttribute("m")), - Project(Seq(UnresolvedStar(None), Alias(Literal(1),"m")()), - Aggregate( - AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), - RELATION_EMPLOYEES))) - )) - - val result = Project(Seq(UnresolvedStar(None)), - DataFrameDropColumns(T12_COLUMNS_SEQ, + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + Seq(UnresolvedAttribute("m")), + Project( + Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) + + val result = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) comparePlans(logicalPlan, result, checkAnalysis = false) @@ -270,9 +262,10 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite test("test multiple AppendCol clauses") { val context = new CatalystPlanContext val logicalPlan = planTransformer.visit( - plan(pplParser, "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), - context - ) + plan( + pplParser, + "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), + context) /* :- 'SubqueryAlias T1 @@ -280,12 +273,13 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite : +- 'Project ['name, 'age] : +- 'UnresolvedRelation [employees], [], false */ - val mainSearch = SubqueryAlias("T1", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project(Seq( - UnresolvedAttribute("name"), - UnresolvedAttribute("age")), RELATION_EMPLOYEES))) - + val mainSearch = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq(UnresolvedAttribute("name"), UnresolvedAttribute("age")), + RELATION_EMPLOYEES))) /* +- 'SubqueryAlias T2 @@ -295,21 +289,23 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] +- 'UnresolvedRelation [employees], [], false */ - val firstAppenCol = SubqueryAlias("T2", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - DataFrameDropColumns(Seq(UnresolvedAttribute("m")), - Project(Seq(UnresolvedStar(None), Alias(Literal(1),"m")()), - Aggregate( - AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), - RELATION_EMPLOYEES))) - )) - - - val joinWithFirstAppendCol = SubqueryAlias("T1", Project(Seq( - ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - DataFrameDropColumns(T12_COLUMNS_SEQ, - Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) - + val firstAppenCol = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + Seq(UnresolvedAttribute("m")), + Project( + Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) + + val joinWithFirstAppendCol = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) /* +- 'SubqueryAlias T2 @@ -317,15 +313,22 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite +- 'Project ['dept] +- 'UnresolvedRelation [employees], [], false */ - val secondAppendCol = SubqueryAlias("T2", Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project(Seq( - UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) - - - val joinWithSecondAppendCol = Project(Seq(UnresolvedStar(None)), - DataFrameDropColumns(T12_COLUMNS_SEQ, - Join(joinWithFirstAppendCol, secondAppendCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + val secondAppendCol = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq(UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) + + val joinWithSecondAppendCol = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join( + joinWithFirstAppendCol, + secondAppendCol, + LeftOuter, + Some(T12_JOIN_CONDITION), + JoinHint.NONE))) comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) } From 822ebd57d947f29a60a0f71e52028052943584bb Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Thu, 12 Dec 2024 17:52:38 -0800 Subject: [PATCH 20/37] Update scala syntax Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 55 ++++++++++--------- ...nAppendColCommandTranslatorTestSuite.scala | 5 ++ 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 53e529575..42b7fe050 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -6,19 +6,25 @@ package org.opensearch.sql.ppl; import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$; import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction; import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation; +import org.apache.spark.sql.catalyst.analysis.UnresolvedStar; import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; import org.apache.spark.sql.catalyst.expressions.Ascending$; import org.apache.spark.sql.catalyst.expressions.Descending$; +import org.apache.spark.sql.catalyst.expressions.EqualTo; +import org.apache.spark.sql.catalyst.expressions.Equality$; import org.apache.spark.sql.catalyst.expressions.Explode; import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.GeneratorOuter; import org.apache.spark.sql.catalyst.expressions.NamedExpression; +import org.apache.spark.sql.catalyst.expressions.Predicate$; import org.apache.spark.sql.catalyst.expressions.SortDirection; import org.apache.spark.sql.catalyst.expressions.SortOrder; import org.apache.spark.sql.catalyst.plans.logical.Aggregate; +import org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns; import org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns$; import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation$; import org.apache.spark.sql.catalyst.plans.logical.Generate; @@ -32,6 +38,7 @@ import org.apache.spark.sql.execution.command.ExplainCommand; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.apache.spark.unsafe.types.UTF8String; import org.opensearch.flint.spark.FlattenGenerator; import org.opensearch.sql.ast.AbstractNodeVisitor; import org.opensearch.sql.ast.Node; @@ -266,37 +273,38 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { final String APPENDCOL_ID = WindowSpecTransformer.ROW_NUMBER_COLUMN_NAME; final String TABLE_LHS = "T1"; final String TABLE_RHS = "T2"; - scala.collection.mutable.Seq fieldsToRemove = seq( - UnresolvedAttribute$.MODULE$.apply(TABLE_LHS + "." + APPENDCOL_ID), - UnresolvedAttribute$.MODULE$.apply(TABLE_RHS + "." + APPENDCOL_ID)); - final Compare innerJoinCondition = new Compare("=", - new Field(QualifiedName.of(TABLE_LHS ,APPENDCOL_ID)), - new Field(QualifiedName.of(TABLE_RHS, APPENDCOL_ID))); + final UnresolvedAttribute t1Attr = new UnresolvedAttribute(seq(TABLE_LHS, APPENDCOL_ID)); + final UnresolvedAttribute t2Attr = new UnresolvedAttribute(seq(TABLE_RHS, APPENDCOL_ID)); + final Seq fieldsToRemove = seq(t1Attr, t2Attr); + final Node mainSearchNode = node.getChild().get(0); // Add a new projection layer with * and ROW_NUMBER (Main-search) - LogicalPlan leftTemp = node.getChild().get(0).accept(this, context); - var mainSearch = getRowNumStarProjection(context, leftTemp, TABLE_LHS); - context.withSubqueryAlias(mainSearch); + LogicalPlan leftTemp = mainSearchNode.accept(this, context); + var mainSearchWithRowNumber = getRowNumStarProjection(context, leftTemp, TABLE_LHS); + context.withSubqueryAlias(mainSearchWithRowNumber); // Traverse to look for relation clause then append it into the sub-search. - Relation relation = retrieveRelationClause(node.getChild().get(0)); + Relation relation = retrieveRelationClause(mainSearchNode); appendRelationClause(node.getSubSearch(), relation); context.apply(left -> { // Add a new projection layer with * and ROW_NUMBER (Sub-search) - LogicalPlan right = node.getSubSearch().accept(this, context); - var subSearch = getRowNumStarProjection(context, right, TABLE_RHS); - context.withSubqueryAlias(subSearch); + LogicalPlan subSearchNode = node.getSubSearch().accept(this, context); + var subSearchWithRowNumber = getRowNumStarProjection(context, subSearchNode, TABLE_RHS); + context.withSubqueryAlias(subSearchWithRowNumber); - Optional joinCondition = Optional.of(innerJoinCondition) - .map(c -> expressionAnalyzer.analyzeJoinCondition(c, context)); context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - LogicalPlan joinedQuery = join(mainSearch, subSearch, Join.JoinType.LEFT, joinCondition, new Join.JoinHint()); + // Composite the join clause + LogicalPlan joinedQuery = join( + mainSearchWithRowNumber, subSearchWithRowNumber, + Join.JoinType.LEFT, + Optional.of(new EqualTo(t1Attr, t2Attr)), + new Join.JoinHint()); // Remove the APPEND_ID - return new org.apache.spark.sql.catalyst.plans.logical.DataFrameDropColumns(fieldsToRemove, joinedQuery); + return new DataFrameDropColumns(fieldsToRemove, joinedQuery); }); return context.getPlan(); } @@ -334,24 +342,19 @@ private static Relation retrieveRelationClause(Node node) { private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { - final String DUMMY_SORT_FIELD = "1"; - - expressionAnalyzer.visitLiteral( - new Literal(DUMMY_SORT_FIELD, DataType.STRING), context); - Expression strExp = context.popNamedParseExpressions().get(); - // Literal("x") - SortOrder sortOrder = SortUtils.sortOrder(strExp, false); + SortOrder sortOrder = SortUtils.sortOrder( + new org.apache.spark.sql.catalyst.expressions.Literal( + UTF8String.fromString("1"), DataTypes.StringType), false); NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); List projectList = (context.getNamedParseExpressions().isEmpty()) - ? List.of(appendCol, UnresolvedStar$.MODULE$.apply(Option.empty())) + ? List.of(appendCol, new UnresolvedStar(Option.empty())) : List.of(appendCol); LogicalPlan lpWithProjection = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( projectList), lp); return SubqueryAlias$.MODULE$.apply(alias, lpWithProjection); - } @Override diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index c41518645..37b992c00 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -97,6 +97,11 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + // scalastyle:off + println(logicalPlan) + println(result) + // scalastyle:on + comparePlans(logicalPlan, result, checkAnalysis = false) } From 25857f2d0059dab2ed841f2761a37aa98e24849e Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 13:31:48 -0800 Subject: [PATCH 21/37] Override option Signed-off-by: Andy Kwok --- .../src/main/antlr4/OpenSearchPPLLexer.g4 | 3 + .../src/main/antlr4/OpenSearchPPLParser.g4 | 2 +- .../sql/ppl/CatalystQueryPlanVisitor.java | 28 +- ...nAppendColCommandTranslatorTestSuite.scala | 580 +++++++++--------- 4 files changed, 321 insertions(+), 292 deletions(-) diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 index 01ea8768a..277a0c2ac 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 @@ -99,6 +99,9 @@ NULLS: 'NULLS'; SMA: 'SMA'; WMA: 'WMA'; +// APPENDCOL options +OVERRIDE: 'OVERRIDE'; + // ARGUMENT KEYWORDS KEEPEMPTY: 'KEEPEMPTY'; CONSECUTIVE: 'CONSECUTIVE'; diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 index 47d3c3f94..be769a9fb 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 @@ -278,7 +278,7 @@ trendlineType ; appendcolCommand - : APPENDCOL LT_SQR_PRTHS commands (PIPE commands)* RT_SQR_PRTHS + : APPENDCOL (OVERRIDE EQUAL booleanLiteral)? LT_SQR_PRTHS commands (PIPE commands)* RT_SQR_PRTHS ; kmeansCommand diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 42b7fe050..676a54264 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -5,6 +5,7 @@ package org.opensearch.sql.ppl; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.TableIdentifier; import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$; @@ -13,6 +14,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedStar; import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; import org.apache.spark.sql.catalyst.expressions.Ascending$; +import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.Descending$; import org.apache.spark.sql.catalyst.expressions.EqualTo; import org.apache.spark.sql.catalyst.expressions.Equality$; @@ -33,7 +35,9 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$; import org.apache.spark.sql.catalyst.plans.logical.Project$; import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias$; +import org.apache.spark.sql.execution.CommandExecutionMode; import org.apache.spark.sql.execution.ExplainMode; +import org.apache.spark.sql.execution.QueryExecution; import org.apache.spark.sql.execution.command.DescribeTableCommand; import org.apache.spark.sql.execution.command.ExplainCommand; import org.apache.spark.sql.types.DataTypes; @@ -100,6 +104,7 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.util.Collections.emptyList; import static java.util.List.of; @@ -147,7 +152,11 @@ public LogicalPlan visitFirstChild(Node node, CatalystPlanContext context) { @Override public LogicalPlan visitExplain(Explain node, CatalystPlanContext context) { node.getStatement().accept(this, context); - return context.apply(p -> new ExplainCommand(p, ExplainMode.fromString(node.getExplainMode().name()))); + context.apply(p -> new ExplainCommand(p, ExplainMode.fromString(node.getExplainMode().name()))); + System.out.println(context.getPlan()); + Seq output = context.getPlan().output(); + System.out.println(output); + return context.getPlan(); } @Override @@ -296,6 +305,14 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); + SparkSession sparkSession = SparkSession.getActiveSession().get(); + + QueryExecution queryExecution = sparkSession.sessionState().executePlan(mainSearchWithRowNumber, CommandExecutionMode.ALL()); + QueryExecution queryExecutionSub = sparkSession.sessionState().executePlan(subSearchWithRowNumber, CommandExecutionMode.ALL()); + + Seq outputMain = queryExecution.analyzed().output(); + Seq outputSub = queryExecutionSub.analyzed().output(); + // Composite the join clause LogicalPlan joinedQuery = join( mainSearchWithRowNumber, subSearchWithRowNumber, @@ -306,6 +323,9 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { // Remove the APPEND_ID return new DataFrameDropColumns(fieldsToRemove, joinedQuery); }); + + System.out.println("Attributes: "); + System.out.println(context.getPlan().output()); return context.getPlan(); } @@ -335,6 +355,12 @@ private static Relation retrieveRelationClause(Node node) { // NPE will be thrown by some node.getChild() call. break; } + /* + if (node == null || node.getChild() == null || node.getChild().isEmpty()) { + break; + } + node = node.getChild().get(0); + */ } } return null; diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index 37b992c00..daae82225 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -47,295 +47,295 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite private val T12_COLUMNS_SEQ = Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) - - // @formatter:off - /** - * Expected: - 'Project [*] - +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] - +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) - :- 'SubqueryAlias T1 - : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#1, *] - : +- 'UnresolvedRelation [employees], [], false - +- 'SubqueryAlias T2 - +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#5, *] - +- 'Aggregate ['age AS age#3], ['COUNT(*) AS count()#2, 'age AS age#3] - +- 'UnresolvedRelation [employees], [], false - */ - // @formatter:on - test("test AppendCol with NO transformation on main") { - val context = new CatalystPlanContext - val logicalPlan = planTransformer.visit( - plan(pplParser, "source=employees | APPENDCOL [stats count() by age];"), - context) - - /* - :- 'SubqueryAlias T1 - : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] - : +- 'UnresolvedRelation [relation], [], false - */ - val t1 = SubqueryAlias( - "T1", - Project(Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), RELATION_EMPLOYEES)) - - /* - +- 'SubqueryAlias T2 - +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, - specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] - +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] - +- 'UnresolvedRelation [relation], [], false - */ - val t2 = SubqueryAlias( - "T2", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) - - val result = Project( - Seq(UnresolvedStar(None)), - DataFrameDropColumns( - T12_COLUMNS_SEQ, - Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - - // scalastyle:off - println(logicalPlan) - println(result) - // scalastyle:on - - comparePlans(logicalPlan, result, checkAnalysis = false) - } - - // @formatter:off - /** - * 'Project [*] - * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] - * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) - * :- 'SubqueryAlias T1 - * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] - * : +- 'Project ['age, 'dept, 'salary] - * : +- 'UnresolvedRelation [relation], [], false - * +- 'SubqueryAlias T2 - * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#15, *] - * +- 'Aggregate ['age AS age#13], ['COUNT(*) AS count()#12, 'age AS age#13] - * +- 'UnresolvedRelation [relation], [], false - */ - // @formatter:on - test("test AppendCol with transformation on main-search") { - val context = new CatalystPlanContext - val logicalPlan = planTransformer.visit( - plan( - pplParser, - "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), - context) - - /* - :- 'SubqueryAlias T1 - : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, - specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] - : +- 'Project ['age, 'dept, 'salary] - : +- 'UnresolvedRelation [relation], [], false - */ - val t1 = SubqueryAlias( - "T1", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project( - Seq( - UnresolvedAttribute("age"), - UnresolvedAttribute("dept"), - UnresolvedAttribute("salary")), - RELATION_EMPLOYEES))) - - /* - +- 'SubqueryAlias T2 - +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, - specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] - +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] - +- 'UnresolvedRelation [relation], [], false - */ - val t2 = SubqueryAlias( - "T2", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) - - val result = Project( - Seq(UnresolvedStar(None)), - DataFrameDropColumns( - T12_COLUMNS_SEQ, - Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - - comparePlans(logicalPlan, result, checkAnalysis = false) - } - - // @formatter:off - /** - * 'Project [*] - * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] - * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) - * :- 'SubqueryAlias T1 - * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#427, *] - * : +- 'Project ['age, 'dept, 'salary] - * : +- 'UnresolvedRelation [employees], [], false - * +- 'SubqueryAlias T2 - * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] - * +- 'DataFrameDropColumns ['m] - * +- 'Project [*, 1 AS m#430] - * +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] - * +- 'UnresolvedRelation [employees], [], false - */ - // @formatter:on - test("test AppendCol with chained sub-search") { - val context = new CatalystPlanContext - val logicalPlan = planTransformer.visit( - plan( - pplParser, - "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), - context) - - /* - :- 'SubqueryAlias T1 - : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, - specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] - : +- 'Project ['age, 'dept, 'salary] - : +- 'UnresolvedRelation [relation], [], false - */ - val t1 = SubqueryAlias( - "T1", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project( - Seq( - UnresolvedAttribute("age"), - UnresolvedAttribute("dept"), - UnresolvedAttribute("salary")), - RELATION_EMPLOYEES))) - - /* - +- 'SubqueryAlias T2 - +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] - +- 'DataFrameDropColumns ['m] - +- 'Project [*, 1 AS m#430] - +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] - +- 'UnresolvedRelation [employees], [], false - */ - val t2 = SubqueryAlias( - "T2", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - DataFrameDropColumns( - Seq(UnresolvedAttribute("m")), - Project( - Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), - Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) - - val result = Project( - Seq(UnresolvedStar(None)), - DataFrameDropColumns( - T12_COLUMNS_SEQ, - Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - - comparePlans(logicalPlan, result, checkAnalysis = false) - } - - // @formatter:off - /** - * == Parsed Logical Plan == - * 'Project [*] - * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] - * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) - * :- 'SubqueryAlias T1 - * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#551, *] - * : +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] - * : +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) - * : :- 'SubqueryAlias T1 - * : : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] - * : : +- 'Project ['name, 'age] - * : : +- 'UnresolvedRelation [employees], [], false - * : +- 'SubqueryAlias T2 - * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#549, *] - * : +- 'DataFrameDropColumns ['m] - * : +- 'Project [*, 1 AS m#547] - * : +- 'Aggregate ['age AS age#546], ['COUNT(*) AS count()#545, 'age AS age#546] - * : +- 'UnresolvedRelation [employees], [], false - * +- 'SubqueryAlias T2 - * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] - * +- 'Project ['dept] - * +- 'UnresolvedRelation [employees], [], false - */ - // @formatter:on - test("test multiple AppendCol clauses") { - val context = new CatalystPlanContext - val logicalPlan = planTransformer.visit( - plan( - pplParser, - "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), - context) - - /* - :- 'SubqueryAlias T1 - : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] - : +- 'Project ['name, 'age] - : +- 'UnresolvedRelation [employees], [], false - */ - val mainSearch = SubqueryAlias( - "T1", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project( - Seq(UnresolvedAttribute("name"), UnresolvedAttribute("age")), - RELATION_EMPLOYEES))) - - /* - +- 'SubqueryAlias T2 - +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] - +- 'DataFrameDropColumns ['m] - +- 'Project [*, 1 AS m#430] - +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] - +- 'UnresolvedRelation [employees], [], false - */ - val firstAppenCol = SubqueryAlias( - "T2", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - DataFrameDropColumns( - Seq(UnresolvedAttribute("m")), - Project( - Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), - Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) - - val joinWithFirstAppendCol = SubqueryAlias( - "T1", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - DataFrameDropColumns( - T12_COLUMNS_SEQ, - Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) - - /* - +- 'SubqueryAlias T2 - +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] - +- 'Project ['dept] - +- 'UnresolvedRelation [employees], [], false - */ - val secondAppendCol = SubqueryAlias( - "T2", - Project( - Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Project(Seq(UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) - - val joinWithSecondAppendCol = Project( - Seq(UnresolvedStar(None)), - DataFrameDropColumns( - T12_COLUMNS_SEQ, - Join( - joinWithFirstAppendCol, - secondAppendCol, - LeftOuter, - Some(T12_JOIN_CONDITION), - JoinHint.NONE))) - - comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) - } +// +// // @formatter:off +// /** +// * Expected: +// 'Project [*] +// +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] +// +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) +// :- 'SubqueryAlias T1 +// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#1, *] +// : +- 'UnresolvedRelation [employees], [], false +// +- 'SubqueryAlias T2 +// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#5, *] +// +- 'Aggregate ['age AS age#3], ['COUNT(*) AS count()#2, 'age AS age#3] +// +- 'UnresolvedRelation [employees], [], false +// */ +// // @formatter:on +// test("test AppendCol with NO transformation on main") { +// val context = new CatalystPlanContext +// val logicalPlan = planTransformer.visit( +// plan(pplParser, "source=employees | APPENDCOL [stats count() by age];"), +// context) +// +// /* +// :- 'SubqueryAlias T1 +// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] +// : +- 'UnresolvedRelation [relation], [], false +// */ +// val t1 = SubqueryAlias( +// "T1", +// Project(Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), RELATION_EMPLOYEES)) +// +// /* +// +- 'SubqueryAlias T2 +// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, +// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] +// +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] +// +- 'UnresolvedRelation [relation], [], false +// */ +// val t2 = SubqueryAlias( +// "T2", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) +// +// val result = Project( +// Seq(UnresolvedStar(None)), +// DataFrameDropColumns( +// T12_COLUMNS_SEQ, +// Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) +// +// // scalastyle:off +// println(logicalPlan) +// println(result) +// // scalastyle:on +// +// comparePlans(logicalPlan, result, checkAnalysis = false) +// } +// +// // @formatter:off +// /** +// * 'Project [*] +// * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] +// * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) +// * :- 'SubqueryAlias T1 +// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] +// * : +- 'Project ['age, 'dept, 'salary] +// * : +- 'UnresolvedRelation [relation], [], false +// * +- 'SubqueryAlias T2 +// * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#15, *] +// * +- 'Aggregate ['age AS age#13], ['COUNT(*) AS count()#12, 'age AS age#13] +// * +- 'UnresolvedRelation [relation], [], false +// */ +// // @formatter:on +// test("test AppendCol with transformation on main-search") { +// val context = new CatalystPlanContext +// val logicalPlan = planTransformer.visit( +// plan( +// pplParser, +// "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), +// context) +// +// /* +// :- 'SubqueryAlias T1 +// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, +// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] +// : +- 'Project ['age, 'dept, 'salary] +// : +- 'UnresolvedRelation [relation], [], false +// */ +// val t1 = SubqueryAlias( +// "T1", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// Project( +// Seq( +// UnresolvedAttribute("age"), +// UnresolvedAttribute("dept"), +// UnresolvedAttribute("salary")), +// RELATION_EMPLOYEES))) +// +// /* +// +- 'SubqueryAlias T2 +// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, +// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] +// +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] +// +- 'UnresolvedRelation [relation], [], false +// */ +// val t2 = SubqueryAlias( +// "T2", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) +// +// val result = Project( +// Seq(UnresolvedStar(None)), +// DataFrameDropColumns( +// T12_COLUMNS_SEQ, +// Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) +// +// comparePlans(logicalPlan, result, checkAnalysis = false) +// } +// +// // @formatter:off +// /** +// * 'Project [*] +// * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] +// * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) +// * :- 'SubqueryAlias T1 +// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#427, *] +// * : +- 'Project ['age, 'dept, 'salary] +// * : +- 'UnresolvedRelation [employees], [], false +// * +- 'SubqueryAlias T2 +// * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] +// * +- 'DataFrameDropColumns ['m] +// * +- 'Project [*, 1 AS m#430] +// * +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] +// * +- 'UnresolvedRelation [employees], [], false +// */ +// // @formatter:on +// test("test AppendCol with chained sub-search") { +// val context = new CatalystPlanContext +// val logicalPlan = planTransformer.visit( +// plan( +// pplParser, +// "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), +// context) +// +// /* +// :- 'SubqueryAlias T1 +// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, +// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] +// : +- 'Project ['age, 'dept, 'salary] +// : +- 'UnresolvedRelation [relation], [], false +// */ +// val t1 = SubqueryAlias( +// "T1", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// Project( +// Seq( +// UnresolvedAttribute("age"), +// UnresolvedAttribute("dept"), +// UnresolvedAttribute("salary")), +// RELATION_EMPLOYEES))) +// +// /* +// +- 'SubqueryAlias T2 +// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] +// +- 'DataFrameDropColumns ['m] +// +- 'Project [*, 1 AS m#430] +// +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] +// +- 'UnresolvedRelation [employees], [], false +// */ +// val t2 = SubqueryAlias( +// "T2", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// DataFrameDropColumns( +// Seq(UnresolvedAttribute("m")), +// Project( +// Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), +// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) +// +// val result = Project( +// Seq(UnresolvedStar(None)), +// DataFrameDropColumns( +// T12_COLUMNS_SEQ, +// Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) +// +// comparePlans(logicalPlan, result, checkAnalysis = false) +// } +// +// // @formatter:off +// /** +// * == Parsed Logical Plan == +// * 'Project [*] +// * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] +// * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) +// * :- 'SubqueryAlias T1 +// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#551, *] +// * : +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] +// * : +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) +// * : :- 'SubqueryAlias T1 +// * : : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] +// * : : +- 'Project ['name, 'age] +// * : : +- 'UnresolvedRelation [employees], [], false +// * : +- 'SubqueryAlias T2 +// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#549, *] +// * : +- 'DataFrameDropColumns ['m] +// * : +- 'Project [*, 1 AS m#547] +// * : +- 'Aggregate ['age AS age#546], ['COUNT(*) AS count()#545, 'age AS age#546] +// * : +- 'UnresolvedRelation [employees], [], false +// * +- 'SubqueryAlias T2 +// * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] +// * +- 'Project ['dept] +// * +- 'UnresolvedRelation [employees], [], false +// */ +// // @formatter:on +// test("test multiple AppendCol clauses") { +// val context = new CatalystPlanContext +// val logicalPlan = planTransformer.visit( +// plan( +// pplParser, +// "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), +// context) +// +// /* +// :- 'SubqueryAlias T1 +// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] +// : +- 'Project ['name, 'age] +// : +- 'UnresolvedRelation [employees], [], false +// */ +// val mainSearch = SubqueryAlias( +// "T1", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// Project( +// Seq(UnresolvedAttribute("name"), UnresolvedAttribute("age")), +// RELATION_EMPLOYEES))) +// +// /* +// +- 'SubqueryAlias T2 +// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] +// +- 'DataFrameDropColumns ['m] +// +- 'Project [*, 1 AS m#430] +// +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] +// +- 'UnresolvedRelation [employees], [], false +// */ +// val firstAppenCol = SubqueryAlias( +// "T2", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// DataFrameDropColumns( +// Seq(UnresolvedAttribute("m")), +// Project( +// Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), +// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) +// +// val joinWithFirstAppendCol = SubqueryAlias( +// "T1", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// DataFrameDropColumns( +// T12_COLUMNS_SEQ, +// Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) +// +// /* +// +- 'SubqueryAlias T2 +// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] +// +- 'Project ['dept] +// +- 'UnresolvedRelation [employees], [], false +// */ +// val secondAppendCol = SubqueryAlias( +// "T2", +// Project( +// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), +// Project(Seq(UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) +// +// val joinWithSecondAppendCol = Project( +// Seq(UnresolvedStar(None)), +// DataFrameDropColumns( +// T12_COLUMNS_SEQ, +// Join( +// joinWithFirstAppendCol, +// secondAppendCol, +// LeftOuter, +// Some(T12_JOIN_CONDITION), +// JoinHint.NONE))) +// +// comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) +// } } From cba664d369cc9817625394b40edd370663bd2477 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 14:20:22 -0800 Subject: [PATCH 22/37] Update override option Signed-off-by: Andy Kwok --- ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 | 2 +- .../src/main/java/org/opensearch/sql/ast/tree/AppendCol.java | 5 +++-- .../main/java/org/opensearch/sql/ppl/parser/AstBuilder.java | 4 +++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 index be769a9fb..04cc19828 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 @@ -278,7 +278,7 @@ trendlineType ; appendcolCommand - : APPENDCOL (OVERRIDE EQUAL booleanLiteral)? LT_SQR_PRTHS commands (PIPE commands)* RT_SQR_PRTHS + : APPENDCOL (OVERRIDE EQUAL override = booleanLiteral)? LT_SQR_PRTHS commands (PIPE commands)* RT_SQR_PRTHS ; kmeansCommand diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java index e52c37f11..fc6b774c0 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java @@ -22,13 +22,14 @@ @AllArgsConstructor public class AppendCol extends UnresolvedPlan { - public boolean override = true; + public boolean override; private UnresolvedPlan subSearch; private UnresolvedPlan child; - public AppendCol(UnresolvedPlan subSearch) { + public AppendCol(UnresolvedPlan subSearch, boolean override) { + this.override = override; this.subSearch = subSearch; } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java index dd9208abe..4312b9a7a 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/parser/AstBuilder.java @@ -422,8 +422,10 @@ public UnresolvedPlan visitAppendcolCommand(OpenSearchPPLParser.AppendcolCommand final Optional pplCmd = ctx.commands().stream() .map(this::visit) .reduce((r, e) -> e.attach(r)); + final boolean override = (ctx.override != null && + Boolean.parseBoolean(ctx.override.getText())); // ANTLR parser check guarantee pplCmd won't be null. - return new AppendCol(pplCmd.get()); + return new AppendCol(pplCmd.get(), override); } private Trendline.TrendlineComputation toTrendlineComputation(OpenSearchPPLParser.TrendlineClauseContext ctx) { From d790d20521df38b8099c0ce01f3c2aaa47e7fbd8 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 14:23:56 -0800 Subject: [PATCH 23/37] Enable override option Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 12 +- ...nAppendColCommandTranslatorTestSuite.scala | 580 +++++++++--------- 2 files changed, 297 insertions(+), 295 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 676a54264..c579234fc 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -305,13 +305,15 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - SparkSession sparkSession = SparkSession.getActiveSession().get(); + if (node.override) { + SparkSession sparkSession = SparkSession.getActiveSession().get(); - QueryExecution queryExecution = sparkSession.sessionState().executePlan(mainSearchWithRowNumber, CommandExecutionMode.ALL()); - QueryExecution queryExecutionSub = sparkSession.sessionState().executePlan(subSearchWithRowNumber, CommandExecutionMode.ALL()); + QueryExecution queryExecution = sparkSession.sessionState().executePlan(mainSearchWithRowNumber, CommandExecutionMode.ALL()); + QueryExecution queryExecutionSub = sparkSession.sessionState().executePlan(subSearchWithRowNumber, CommandExecutionMode.ALL()); - Seq outputMain = queryExecution.analyzed().output(); - Seq outputSub = queryExecutionSub.analyzed().output(); + Seq outputMain = queryExecution.analyzed().output(); + Seq outputSub = queryExecutionSub.analyzed().output(); + } // Composite the join clause LogicalPlan joinedQuery = join( diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index daae82225..37b992c00 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -47,295 +47,295 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite private val T12_COLUMNS_SEQ = Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) -// -// // @formatter:off -// /** -// * Expected: -// 'Project [*] -// +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] -// +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) -// :- 'SubqueryAlias T1 -// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#1, *] -// : +- 'UnresolvedRelation [employees], [], false -// +- 'SubqueryAlias T2 -// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#5, *] -// +- 'Aggregate ['age AS age#3], ['COUNT(*) AS count()#2, 'age AS age#3] -// +- 'UnresolvedRelation [employees], [], false -// */ -// // @formatter:on -// test("test AppendCol with NO transformation on main") { -// val context = new CatalystPlanContext -// val logicalPlan = planTransformer.visit( -// plan(pplParser, "source=employees | APPENDCOL [stats count() by age];"), -// context) -// -// /* -// :- 'SubqueryAlias T1 -// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] -// : +- 'UnresolvedRelation [relation], [], false -// */ -// val t1 = SubqueryAlias( -// "T1", -// Project(Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), RELATION_EMPLOYEES)) -// -// /* -// +- 'SubqueryAlias T2 -// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, -// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] -// +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] -// +- 'UnresolvedRelation [relation], [], false -// */ -// val t2 = SubqueryAlias( -// "T2", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) -// -// val result = Project( -// Seq(UnresolvedStar(None)), -// DataFrameDropColumns( -// T12_COLUMNS_SEQ, -// Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) -// -// // scalastyle:off -// println(logicalPlan) -// println(result) -// // scalastyle:on -// -// comparePlans(logicalPlan, result, checkAnalysis = false) -// } -// -// // @formatter:off -// /** -// * 'Project [*] -// * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] -// * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) -// * :- 'SubqueryAlias T1 -// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] -// * : +- 'Project ['age, 'dept, 'salary] -// * : +- 'UnresolvedRelation [relation], [], false -// * +- 'SubqueryAlias T2 -// * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#15, *] -// * +- 'Aggregate ['age AS age#13], ['COUNT(*) AS count()#12, 'age AS age#13] -// * +- 'UnresolvedRelation [relation], [], false -// */ -// // @formatter:on -// test("test AppendCol with transformation on main-search") { -// val context = new CatalystPlanContext -// val logicalPlan = planTransformer.visit( -// plan( -// pplParser, -// "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), -// context) -// -// /* -// :- 'SubqueryAlias T1 -// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, -// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] -// : +- 'Project ['age, 'dept, 'salary] -// : +- 'UnresolvedRelation [relation], [], false -// */ -// val t1 = SubqueryAlias( -// "T1", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// Project( -// Seq( -// UnresolvedAttribute("age"), -// UnresolvedAttribute("dept"), -// UnresolvedAttribute("salary")), -// RELATION_EMPLOYEES))) -// -// /* -// +- 'SubqueryAlias T2 -// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, -// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] -// +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] -// +- 'UnresolvedRelation [relation], [], false -// */ -// val t2 = SubqueryAlias( -// "T2", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) -// -// val result = Project( -// Seq(UnresolvedStar(None)), -// DataFrameDropColumns( -// T12_COLUMNS_SEQ, -// Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) -// -// comparePlans(logicalPlan, result, checkAnalysis = false) -// } -// -// // @formatter:off -// /** -// * 'Project [*] -// * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] -// * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) -// * :- 'SubqueryAlias T1 -// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#427, *] -// * : +- 'Project ['age, 'dept, 'salary] -// * : +- 'UnresolvedRelation [employees], [], false -// * +- 'SubqueryAlias T2 -// * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] -// * +- 'DataFrameDropColumns ['m] -// * +- 'Project [*, 1 AS m#430] -// * +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] -// * +- 'UnresolvedRelation [employees], [], false -// */ -// // @formatter:on -// test("test AppendCol with chained sub-search") { -// val context = new CatalystPlanContext -// val logicalPlan = planTransformer.visit( -// plan( -// pplParser, -// "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), -// context) -// -// /* -// :- 'SubqueryAlias T1 -// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, -// specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] -// : +- 'Project ['age, 'dept, 'salary] -// : +- 'UnresolvedRelation [relation], [], false -// */ -// val t1 = SubqueryAlias( -// "T1", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// Project( -// Seq( -// UnresolvedAttribute("age"), -// UnresolvedAttribute("dept"), -// UnresolvedAttribute("salary")), -// RELATION_EMPLOYEES))) -// -// /* -// +- 'SubqueryAlias T2 -// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] -// +- 'DataFrameDropColumns ['m] -// +- 'Project [*, 1 AS m#430] -// +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] -// +- 'UnresolvedRelation [employees], [], false -// */ -// val t2 = SubqueryAlias( -// "T2", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// DataFrameDropColumns( -// Seq(UnresolvedAttribute("m")), -// Project( -// Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), -// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) -// -// val result = Project( -// Seq(UnresolvedStar(None)), -// DataFrameDropColumns( -// T12_COLUMNS_SEQ, -// Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) -// -// comparePlans(logicalPlan, result, checkAnalysis = false) -// } -// -// // @formatter:off -// /** -// * == Parsed Logical Plan == -// * 'Project [*] -// * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] -// * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) -// * :- 'SubqueryAlias T1 -// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#551, *] -// * : +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] -// * : +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) -// * : :- 'SubqueryAlias T1 -// * : : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] -// * : : +- 'Project ['name, 'age] -// * : : +- 'UnresolvedRelation [employees], [], false -// * : +- 'SubqueryAlias T2 -// * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#549, *] -// * : +- 'DataFrameDropColumns ['m] -// * : +- 'Project [*, 1 AS m#547] -// * : +- 'Aggregate ['age AS age#546], ['COUNT(*) AS count()#545, 'age AS age#546] -// * : +- 'UnresolvedRelation [employees], [], false -// * +- 'SubqueryAlias T2 -// * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] -// * +- 'Project ['dept] -// * +- 'UnresolvedRelation [employees], [], false -// */ -// // @formatter:on -// test("test multiple AppendCol clauses") { -// val context = new CatalystPlanContext -// val logicalPlan = planTransformer.visit( -// plan( -// pplParser, -// "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), -// context) -// -// /* -// :- 'SubqueryAlias T1 -// : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] -// : +- 'Project ['name, 'age] -// : +- 'UnresolvedRelation [employees], [], false -// */ -// val mainSearch = SubqueryAlias( -// "T1", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// Project( -// Seq(UnresolvedAttribute("name"), UnresolvedAttribute("age")), -// RELATION_EMPLOYEES))) -// -// /* -// +- 'SubqueryAlias T2 -// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] -// +- 'DataFrameDropColumns ['m] -// +- 'Project [*, 1 AS m#430] -// +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] -// +- 'UnresolvedRelation [employees], [], false -// */ -// val firstAppenCol = SubqueryAlias( -// "T2", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// DataFrameDropColumns( -// Seq(UnresolvedAttribute("m")), -// Project( -// Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), -// Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) -// -// val joinWithFirstAppendCol = SubqueryAlias( -// "T1", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// DataFrameDropColumns( -// T12_COLUMNS_SEQ, -// Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) -// -// /* -// +- 'SubqueryAlias T2 -// +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] -// +- 'Project ['dept] -// +- 'UnresolvedRelation [employees], [], false -// */ -// val secondAppendCol = SubqueryAlias( -// "T2", -// Project( -// Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), -// Project(Seq(UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) -// -// val joinWithSecondAppendCol = Project( -// Seq(UnresolvedStar(None)), -// DataFrameDropColumns( -// T12_COLUMNS_SEQ, -// Join( -// joinWithFirstAppendCol, -// secondAppendCol, -// LeftOuter, -// Some(T12_JOIN_CONDITION), -// JoinHint.NONE))) -// -// comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) -// } + + // @formatter:off + /** + * Expected: + 'Project [*] + +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#1, *] + : +- 'UnresolvedRelation [employees], [], false + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#5, *] + +- 'Aggregate ['age AS age#3], ['COUNT(*) AS count()#2, 'age AS age#3] + +- 'UnresolvedRelation [employees], [], false + */ + // @formatter:on + test("test AppendCol with NO transformation on main") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan(pplParser, "source=employees | APPENDCOL [stats count() by age];"), + context) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project(Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), RELATION_EMPLOYEES)) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] + +- 'UnresolvedRelation [relation], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) + + val result = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + // scalastyle:off + println(logicalPlan) + println(result) + // scalastyle:on + + comparePlans(logicalPlan, result, checkAnalysis = false) + } + + // @formatter:off + /** + * 'Project [*] + * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * :- 'SubqueryAlias T1 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + * : +- 'Project ['age, 'dept, 'salary] + * : +- 'UnresolvedRelation [relation], [], false + * +- 'SubqueryAlias T2 + * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#15, *] + * +- 'Aggregate ['age AS age#13], ['COUNT(*) AS count()#12, 'age AS age#13] + * +- 'UnresolvedRelation [relation], [], false + */ + // @formatter:on + test("test AppendCol with transformation on main-search") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan( + pplParser, + "source=employees | FIELDS age, dept, salary | APPENDCOL [stats count() by age];"), + context) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['age, 'dept, 'salary] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("age"), + UnresolvedAttribute("dept"), + UnresolvedAttribute("salary")), + RELATION_EMPLOYEES))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] + +- 'UnresolvedRelation [relation], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) + + val result = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + comparePlans(logicalPlan, result, checkAnalysis = false) + } + + // @formatter:off + /** + * 'Project [*] + * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * :- 'SubqueryAlias T1 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#427, *] + * : +- 'Project ['age, 'dept, 'salary] + * : +- 'UnresolvedRelation [employees], [], false + * +- 'SubqueryAlias T2 + * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + * +- 'DataFrameDropColumns ['m] + * +- 'Project [*, 1 AS m#430] + * +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + * +- 'UnresolvedRelation [employees], [], false + */ + // @formatter:on + test("test AppendCol with chained sub-search") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan( + pplParser, + "source=employees | FIELDS age, dept, salary | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ];"), + context) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['age, 'dept, 'salary] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("age"), + UnresolvedAttribute("dept"), + UnresolvedAttribute("salary")), + RELATION_EMPLOYEES))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + +- 'DataFrameDropColumns ['m] + +- 'Project [*, 1 AS m#430] + +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + +- 'UnresolvedRelation [employees], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + Seq(UnresolvedAttribute("m")), + Project( + Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) + + val result = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + comparePlans(logicalPlan, result, checkAnalysis = false) + } + + // @formatter:off + /** + * == Parsed Logical Plan == + * 'Project [*] + * +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * :- 'SubqueryAlias T1 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#551, *] + * : +- 'DataFrameDropColumns ['T1._row_number_, 'T2._row_number_] + * : +- 'Join LeftOuter, ('T1._row_number_ = 'T2._row_number_) + * : :- 'SubqueryAlias T1 + * : : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] + * : : +- 'Project ['name, 'age] + * : : +- 'UnresolvedRelation [employees], [], false + * : +- 'SubqueryAlias T2 + * : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#549, *] + * : +- 'DataFrameDropColumns ['m] + * : +- 'Project [*, 1 AS m#547] + * : +- 'Aggregate ['age AS age#546], ['COUNT(*) AS count()#545, 'age AS age#546] + * : +- 'UnresolvedRelation [employees], [], false + * +- 'SubqueryAlias T2 + * +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] + * +- 'Project ['dept] + * +- 'UnresolvedRelation [employees], [], false + */ + // @formatter:on + test("test multiple AppendCol clauses") { + val context = new CatalystPlanContext + val logicalPlan = planTransformer.visit( + plan( + pplParser, + "source=employees | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS dept];"), + context) + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] + : +- 'Project ['name, 'age] + : +- 'UnresolvedRelation [employees], [], false + */ + val mainSearch = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq(UnresolvedAttribute("name"), UnresolvedAttribute("age")), + RELATION_EMPLOYEES))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + +- 'DataFrameDropColumns ['m] + +- 'Project [*, 1 AS m#430] + +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + +- 'UnresolvedRelation [employees], [], false + */ + val firstAppenCol = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + Seq(UnresolvedAttribute("m")), + Project( + Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) + + val joinWithFirstAppendCol = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] + +- 'Project ['dept] + +- 'UnresolvedRelation [employees], [], false + */ + val secondAppendCol = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq(UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) + + val joinWithSecondAppendCol = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join( + joinWithFirstAppendCol, + secondAppendCol, + LeftOuter, + Some(T12_JOIN_CONDITION), + JoinHint.NONE))) + + comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) + } } From 816f6d69499da11cc4c10c1d9b8fd29a24726cf6 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 16:06:54 -0800 Subject: [PATCH 24/37] Override impl Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 61 ++++++++++--------- 1 file changed, 33 insertions(+), 28 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index c579234fc..575537826 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -8,7 +8,6 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.TableIdentifier; import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$; import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction; import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation; import org.apache.spark.sql.catalyst.analysis.UnresolvedStar; @@ -17,12 +16,10 @@ import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.Descending$; import org.apache.spark.sql.catalyst.expressions.EqualTo; -import org.apache.spark.sql.catalyst.expressions.Equality$; import org.apache.spark.sql.catalyst.expressions.Explode; import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.GeneratorOuter; import org.apache.spark.sql.catalyst.expressions.NamedExpression; -import org.apache.spark.sql.catalyst.expressions.Predicate$; import org.apache.spark.sql.catalyst.expressions.SortDirection; import org.apache.spark.sql.catalyst.expressions.SortOrder; import org.apache.spark.sql.catalyst.plans.logical.Aggregate; @@ -48,15 +45,12 @@ import org.opensearch.sql.ast.Node; import org.opensearch.sql.ast.expression.Alias; import org.opensearch.sql.ast.expression.Argument; -import org.opensearch.sql.ast.expression.Compare; -import org.opensearch.sql.ast.expression.DataType; import org.opensearch.sql.ast.expression.Field; import org.opensearch.sql.ast.expression.Function; import org.opensearch.sql.ast.expression.In; import org.opensearch.sql.ast.expression.Let; import org.opensearch.sql.ast.expression.Literal; import org.opensearch.sql.ast.expression.ParseMethod; -import org.opensearch.sql.ast.expression.QualifiedName; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.ast.expression.WindowFunction; import org.opensearch.sql.ast.statement.Explain; @@ -104,7 +98,6 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; -import java.util.stream.Stream; import static java.util.Collections.emptyList; import static java.util.List.of; @@ -284,37 +277,32 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { final String TABLE_RHS = "T2"; final UnresolvedAttribute t1Attr = new UnresolvedAttribute(seq(TABLE_LHS, APPENDCOL_ID)); final UnresolvedAttribute t2Attr = new UnresolvedAttribute(seq(TABLE_RHS, APPENDCOL_ID)); - final Seq fieldsToRemove = seq(t1Attr, t2Attr); +// final Seq fieldsToRemove = seq(t1Attr, t2Attr); + final List fieldsToRemove = new ArrayList<>(List.of(t1Attr, t2Attr)); final Node mainSearchNode = node.getChild().get(0); - - // Add a new projection layer with * and ROW_NUMBER (Main-search) - LogicalPlan leftTemp = mainSearchNode.accept(this, context); - var mainSearchWithRowNumber = getRowNumStarProjection(context, leftTemp, TABLE_LHS); - context.withSubqueryAlias(mainSearchWithRowNumber); + final Node subSearchNode = node.getSubSearch(); // Traverse to look for relation clause then append it into the sub-search. Relation relation = retrieveRelationClause(mainSearchNode); appendRelationClause(node.getSubSearch(), relation); + // Add apply a dropColumns if override present, then add * with ROW_NUMBER + LogicalPlan leftTemp = mainSearchNode.accept(this, context); +// LogicalPlan mainSearch = (node.override) +// ? new DataFrameDropColumns(getoverridedlist(subSearch), leftTemp) +// : leftTemp; + var mainSearchWithRowNumber = getRowNumStarProjection(context, leftTemp, TABLE_LHS); + context.withSubqueryAlias(mainSearchWithRowNumber); + context.apply(left -> { // Add a new projection layer with * and ROW_NUMBER (Sub-search) - LogicalPlan subSearchNode = node.getSubSearch().accept(this, context); - var subSearchWithRowNumber = getRowNumStarProjection(context, subSearchNode, TABLE_RHS); + LogicalPlan subSearch = subSearchNode.accept(this, context); + var subSearchWithRowNumber = getRowNumStarProjection(context, subSearch, TABLE_RHS); context.withSubqueryAlias(subSearchWithRowNumber); context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - if (node.override) { - SparkSession sparkSession = SparkSession.getActiveSession().get(); - - QueryExecution queryExecution = sparkSession.sessionState().executePlan(mainSearchWithRowNumber, CommandExecutionMode.ALL()); - QueryExecution queryExecutionSub = sparkSession.sessionState().executePlan(subSearchWithRowNumber, CommandExecutionMode.ALL()); - - Seq outputMain = queryExecution.analyzed().output(); - Seq outputSub = queryExecutionSub.analyzed().output(); - } - // Composite the join clause LogicalPlan joinedQuery = join( mainSearchWithRowNumber, subSearchWithRowNumber, @@ -322,12 +310,15 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { Optional.of(new EqualTo(t1Attr, t2Attr)), new Join.JoinHint()); + // Remove the APPEND_ID - return new DataFrameDropColumns(fieldsToRemove, joinedQuery); + if (node.override) { + List getoverridedlist = getoverridedlist(subSearchWithRowNumber, TABLE_LHS); + fieldsToRemove.addAll(getoverridedlist); + } + return new DataFrameDropColumns(seq(fieldsToRemove), joinedQuery); }); - System.out.println("Attributes: "); - System.out.println(context.getPlan().output()); return context.getPlan(); } @@ -368,6 +359,20 @@ private static Relation retrieveRelationClause(Node node) { return null; } + private static List getoverridedlist(LogicalPlan lp, String tableName) { + // When override option present, extract fields to project from sub-search, + // then apply a dfDropColumns on main-search to avoid duplicate fields. + SparkSession sparkSession = SparkSession.getActiveSession().get(); + QueryExecution queryExecutionSub = sparkSession.sessionState() + .executePlan(lp, CommandExecutionMode.ALL()); + Seq output = queryExecutionSub.analyzed().output(); + List attributes = seqAsJavaList(output); + return attributes.stream() + .map(attr -> + new UnresolvedAttribute(seq(tableName, attr.name()))) + .collect(Collectors.toList()); + } + private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { SortOrder sortOrder = SortUtils.sortOrder( From f6e03dd5406173f34201976159cac22eddac1482 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 17:58:55 -0800 Subject: [PATCH 25/37] Minimise cmd permission Signed-off-by: Andy Kwok --- .../java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 575537826..67f1fdef9 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -364,7 +364,7 @@ private static List getoverridedlist(LogicalPlan lp, String tableNam // then apply a dfDropColumns on main-search to avoid duplicate fields. SparkSession sparkSession = SparkSession.getActiveSession().get(); QueryExecution queryExecutionSub = sparkSession.sessionState() - .executePlan(lp, CommandExecutionMode.ALL()); + .executePlan(lp, CommandExecutionMode.SKIP()); Seq output = queryExecutionSub.analyzed().output(); List attributes = seqAsJavaList(output); return attributes.stream() @@ -380,7 +380,6 @@ private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarP UTF8String.fromString("1"), DataTypes.StringType), false); NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); - List projectList = (context.getNamedParseExpressions().isEmpty()) ? List.of(appendCol, new UnresolvedStar(Option.empty())) : List.of(appendCol); From 83d621ece926923d5df19de4984e28ff6b4dcdf2 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 20:48:10 -0800 Subject: [PATCH 26/37] Refactor util class Signed-off-by: Andy Kwok --- .../sql/ppl/CatalystQueryPlanVisitor.java | 85 ++--------------- .../sql/ppl/utils/AppendColCatalystUtils.java | 92 +++++++++++++++++++ 2 files changed, 99 insertions(+), 78 deletions(-) create mode 100644 ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 67f1fdef9..912be3dee 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -83,6 +83,7 @@ import org.opensearch.sql.ast.tree.UnresolvedPlan; import org.opensearch.sql.ast.tree.Window; import org.opensearch.sql.common.antlr.SyntaxCheckException; +import org.opensearch.sql.ppl.utils.AppendColCatalystUtils; import org.opensearch.sql.ppl.utils.FieldSummaryTransformer; import org.opensearch.sql.ppl.utils.ParseTransformer; import org.opensearch.sql.ppl.utils.SortUtils; @@ -277,32 +278,27 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { final String TABLE_RHS = "T2"; final UnresolvedAttribute t1Attr = new UnresolvedAttribute(seq(TABLE_LHS, APPENDCOL_ID)); final UnresolvedAttribute t2Attr = new UnresolvedAttribute(seq(TABLE_RHS, APPENDCOL_ID)); -// final Seq fieldsToRemove = seq(t1Attr, t2Attr); final List fieldsToRemove = new ArrayList<>(List.of(t1Attr, t2Attr)); final Node mainSearchNode = node.getChild().get(0); final Node subSearchNode = node.getSubSearch(); // Traverse to look for relation clause then append it into the sub-search. - Relation relation = retrieveRelationClause(mainSearchNode); - appendRelationClause(node.getSubSearch(), relation); + Relation relation = AppendColCatalystUtils.retrieveRelationClause(mainSearchNode); + AppendColCatalystUtils.appendRelationClause(node.getSubSearch(), relation); // Add apply a dropColumns if override present, then add * with ROW_NUMBER LogicalPlan leftTemp = mainSearchNode.accept(this, context); -// LogicalPlan mainSearch = (node.override) -// ? new DataFrameDropColumns(getoverridedlist(subSearch), leftTemp) -// : leftTemp; - var mainSearchWithRowNumber = getRowNumStarProjection(context, leftTemp, TABLE_LHS); + var mainSearchWithRowNumber = AppendColCatalystUtils.getRowNumStarProjection(context, leftTemp, TABLE_LHS); context.withSubqueryAlias(mainSearchWithRowNumber); context.apply(left -> { // Add a new projection layer with * and ROW_NUMBER (Sub-search) LogicalPlan subSearch = subSearchNode.accept(this, context); - var subSearchWithRowNumber = getRowNumStarProjection(context, subSearch, TABLE_RHS); + var subSearchWithRowNumber = AppendColCatalystUtils.getRowNumStarProjection(context, subSearch, TABLE_RHS); context.withSubqueryAlias(subSearchWithRowNumber); context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - // Composite the join clause LogicalPlan joinedQuery = join( mainSearchWithRowNumber, subSearchWithRowNumber, @@ -310,10 +306,9 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { Optional.of(new EqualTo(t1Attr, t2Attr)), new Join.JoinHint()); - - // Remove the APPEND_ID + // Remove the APPEND_ID and duplicated field on T1 if override option is true. if (node.override) { - List getoverridedlist = getoverridedlist(subSearchWithRowNumber, TABLE_LHS); + List getoverridedlist = AppendColCatalystUtils.getoverridedlist(subSearchWithRowNumber, TABLE_LHS); fieldsToRemove.addAll(getoverridedlist); } return new DataFrameDropColumns(seq(fieldsToRemove), joinedQuery); @@ -322,72 +317,6 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { return context.getPlan(); } - private static void appendRelationClause(Node subSearch, Relation relation) { - - Relation table = new Relation(relation.getTableNames()); - // Replace it with a function to look up the search command and extract the index name. - while (subSearch != null) { - try { - subSearch = subSearch.getChild().get(0); - } catch (NullPointerException ex) { - System.out.println("Null when getting the child "); - ((UnresolvedPlan) subSearch).attach(table); - break; - } - } - } - - private static Relation retrieveRelationClause(Node node) { - while (node != null) { - if (node instanceof Relation) { - return (Relation) node; - } else { - try { - node = node.getChild().get(0); - } catch (NullPointerException ex) { - // NPE will be thrown by some node.getChild() call. - break; - } - /* - if (node == null || node.getChild() == null || node.getChild().isEmpty()) { - break; - } - node = node.getChild().get(0); - */ - } - } - return null; - } - - private static List getoverridedlist(LogicalPlan lp, String tableName) { - // When override option present, extract fields to project from sub-search, - // then apply a dfDropColumns on main-search to avoid duplicate fields. - SparkSession sparkSession = SparkSession.getActiveSession().get(); - QueryExecution queryExecutionSub = sparkSession.sessionState() - .executePlan(lp, CommandExecutionMode.SKIP()); - Seq output = queryExecutionSub.analyzed().output(); - List attributes = seqAsJavaList(output); - return attributes.stream() - .map(attr -> - new UnresolvedAttribute(seq(tableName, attr.name()))) - .collect(Collectors.toList()); - } - - private org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { - - SortOrder sortOrder = SortUtils.sortOrder( - new org.apache.spark.sql.catalyst.expressions.Literal( - UTF8String.fromString("1"), DataTypes.StringType), false); - - NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); - List projectList = (context.getNamedParseExpressions().isEmpty()) - ? List.of(appendCol, new UnresolvedStar(Option.empty())) - : List.of(appendCol); - - LogicalPlan lpWithProjection = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( - projectList), lp); - return SubqueryAlias$.MODULE$.apply(alias, lpWithProjection); - } @Override public LogicalPlan visitCorrelation(Correlation node, CatalystPlanContext context) { diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java new file mode 100644 index 000000000..3074b29c5 --- /dev/null +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java @@ -0,0 +1,92 @@ +package org.opensearch.sql.ppl.utils; + +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; +import org.apache.spark.sql.catalyst.analysis.UnresolvedStar; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.NamedExpression; +import org.apache.spark.sql.catalyst.expressions.SortOrder; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias; +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias$; +import org.apache.spark.sql.execution.CommandExecutionMode; +import org.apache.spark.sql.execution.QueryExecution; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.unsafe.types.UTF8String; +import org.opensearch.sql.ast.Node; +import org.opensearch.sql.ast.tree.Relation; +import org.opensearch.sql.ast.tree.UnresolvedPlan; +import org.opensearch.sql.ppl.CatalystPlanContext; +import scala.Option; +import scala.collection.Seq; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.opensearch.sql.ppl.utils.DataTypeTransformer.seq; +import static scala.collection.JavaConverters.seqAsJavaList; + +/** + * Util class to facilitate the logical plan composition for APPENDCOL command. + */ +public interface AppendColCatalystUtils { + + static void appendRelationClause(Node subSearch, Relation relation) { + Relation table = new Relation(relation.getTableNames()); + // Replace it with a function to look up the search command and extract the index name. + while (subSearch != null) { + try { + subSearch = subSearch.getChild().get(0); + } catch (NullPointerException ex) { + ((UnresolvedPlan) subSearch).attach(table); + break; + } + } + } + + static Relation retrieveRelationClause(Node node) { + while (node != null) { + if (node instanceof Relation) { + return (Relation) node; + } else { + try { + node = node.getChild().get(0); + } catch (NullPointerException ex) { + // NPE will be thrown by some node.getChild() call. + break; + } + } + } + return null; + } + + static List getoverridedlist(LogicalPlan lp, String tableName) { + // When override option present, extract fields to project from sub-search, + // then apply a dfDropColumns on main-search to avoid duplicate fields. + final SparkSession sparkSession = SparkSession.getActiveSession().get(); + final QueryExecution queryExecutionSub = sparkSession.sessionState() + .executePlan(lp, CommandExecutionMode.SKIP()); + final Seq output = queryExecutionSub.analyzed().output(); + final List attributes = seqAsJavaList(output); + return attributes.stream() + .map(attr -> + new UnresolvedAttribute(seq(tableName, attr.name()))) + .collect(Collectors.toList()); + } + + static SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { + final SortOrder sortOrder = SortUtils.sortOrder( + new org.apache.spark.sql.catalyst.expressions.Literal( + UTF8String.fromString("1"), DataTypes.StringType), false); + + final NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); + final List projectList = (context.getNamedParseExpressions().isEmpty()) + ? List.of(appendCol, new UnresolvedStar(Option.empty())) + : List.of(appendCol); + + final LogicalPlan lpWithProjection = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( + projectList), lp); + return SubqueryAlias$.MODULE$.apply(alias, lpWithProjection); + } +} From b6d5ca0e8562c4acc7fe9ba6096e94a6d6c11988 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 21:05:16 -0800 Subject: [PATCH 27/37] Java doc Signed-off-by: Andy Kwok --- .../sql/ppl/utils/AppendColCatalystUtils.java | 30 ++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java index 3074b29c5..699c05409 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java @@ -32,6 +32,12 @@ */ public interface AppendColCatalystUtils { + /** + * Response to traverse given subSearch Node till the last child, then append the Relation clause, + * in order to specify the data source || index. + * @param subSearch User provided sub-search from APPENDCOL command. + * @param relation Relation clause which represent the dataSource that this sub-search execute upon. + */ static void appendRelationClause(Node subSearch, Relation relation) { Relation table = new Relation(relation.getTableNames()); // Replace it with a function to look up the search command and extract the index name. @@ -45,6 +51,11 @@ static void appendRelationClause(Node subSearch, Relation relation) { } } + /** + * Util method to traverse a given Node object and return the first occurrence of a Relation clause. + * @param node The Node object that this util method search upon. + * @return The first occurrence of Relation object from the given Node. + */ static Relation retrieveRelationClause(Node node) { while (node != null) { if (node instanceof Relation) { @@ -53,7 +64,9 @@ static Relation retrieveRelationClause(Node node) { try { node = node.getChild().get(0); } catch (NullPointerException ex) { - // NPE will be thrown by some node.getChild() call. + // Base on the current implementation of Flint, + // NPE will be thrown by certain type of Node implementation, + // when node.getChild() being called. break; } } @@ -61,6 +74,14 @@ static Relation retrieveRelationClause(Node node) { return null; } + + /** + * Util method to perform analyzed() call against the given LogicalPlan to exact all fields + * that will be projected upon the execution in the form of Java List with user provided schema prefix. + * @param lp LogicalPlan instance to extract the projection fields from. + * @param tableName the table || schema name being appended as part of the returned fields. + * @return A list of Expression instances with alternated tableName || Schema information. + */ static List getoverridedlist(LogicalPlan lp, String tableName) { // When override option present, extract fields to project from sub-search, // then apply a dfDropColumns on main-search to avoid duplicate fields. @@ -75,6 +96,13 @@ static List getoverridedlist(LogicalPlan lp, String tableName) { .collect(Collectors.toList()); } + /** + * Helper method to first add an additional project clause to provide row_number, then wrap it SubqueryAlias and return. + * @param context Context object of the current Parser. + * @param lp The Logical Plan instance which contains the query. + * @param alias The name of the Alias clause. + * @return A subqeuryAlias instance which has row_number for natural ordering purpose. + */ static SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { final SortOrder sortOrder = SortUtils.sortOrder( new org.apache.spark.sql.catalyst.expressions.Literal( From c56e4e577cf0df409ada647166cbad8ef06349a0 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Fri, 13 Dec 2024 22:54:32 -0800 Subject: [PATCH 28/37] Integ test 1 2 Signed-off-by: Andy Kwok --- .../ppl/FlintSparkPPLAppendColITSuite.scala | 583 ++++++++++++++++++ ...nAppendColCommandTranslatorTestSuite.scala | 24 +- 2 files changed, 595 insertions(+), 12 deletions(-) create mode 100644 integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala new file mode 100644 index 000000000..436ef2b6e --- /dev/null +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala @@ -0,0 +1,583 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.spark.ppl + +import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq +import org.opensearch.sql.ppl.utils.SortUtils +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Ascending, CaseWhen, CurrentRow, Descending, Divide, EqualTo, Expression, LessThan, Literal, Multiply, RowFrame, RowNumber, SortOrder, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.LeftOuter +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.streaming.StreamTest + + + +class FlintSparkPPLAppendColITSuite + extends QueryTest + with LogicalPlanTestUtils + with FlintPPLSuite + with StreamTest { + + /** Test table and index name */ + private val testTable = "spark_catalog.default.flint_ppl_test" + + private val ROW_NUMBER_AGGREGATION = Alias( + WindowExpression( + RowNumber(), + WindowSpecDefinition( + Nil, + SortUtils.sortOrder(Literal("1"), false) :: Nil, + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), + "_row_number_")() + + private val COUNT_STAR = Alias( + UnresolvedFunction(Seq("COUNT"), Seq(UnresolvedStar(None)), isDistinct = false), + "count()")() + + private val AGE_ALIAS = Alias(UnresolvedAttribute("age"), "age")() + + private val RELATION_TEST_TABLE = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + + private val T12_JOIN_CONDITION = + EqualTo(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + + private val T12_COLUMNS_SEQ = + Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) + + + override def beforeAll(): Unit = { + super.beforeAll() + + // Create test table + createPartitionedStateCountryTable(testTable) + } + + protected override def afterEach(): Unit = { + super.afterEach() + // Stop all streaming jobs if any + spark.streams.active.foreach { job => + job.stop() + job.awaitTermination() + } + } + + test("test AppendCol with NO transformation on main") { + val frame = sql(s""" + | source = $testTable | APPENDCOL [stats count() by age] + | """.stripMargin) + + + assert( + frame.columns.sameElements( + Array("name", "age", "state", "country", "year", "month", "count()", "age"))) + // Retrieve the results + val results: Array[Row] = frame.collect() + val expectedResults: Array[Row] = + Array( + Row("Jake", 70, "California", "USA", 2023, 4, 1, 70), + Row("Hello", 30, "New York", "USA", 2023, 4, 1, 30), + Row("John", 25, "Ontario", "Canada", 2023, 4, 1, 25), + Row("Jane", 20, "Quebec", "Canada", 2023, 4, 1, 20)) + // Compare the results + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) + assert(results.sorted.sameElements(expectedResults.sorted)) + + // Retrieve the logical plan + val logicalPlan: LogicalPlan = frame.queryExecution.logical + + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#7, *] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project(Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), RELATION_TEST_TABLE)) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] + +- 'UnresolvedRelation [relation], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_TEST_TABLE))) + + val expectedPlan = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + + // scalastyle:off + println(logicalPlan) + println(expectedPlan) + // scalastyle:on + + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + + test("test AppendCol with transformation on main-search") { + val frame = sql(s""" + | source = $testTable | FIELDS name, age, state | APPENDCOL [stats count() by age] + | """.stripMargin) + + assert(frame.columns.sameElements(Array("name", "age", "state", "count()", "age"))) + // Retrieve the results + val results: Array[Row] = frame.collect() + val expectedResults: Array[Row] = + Array( + Row("Jake", 70, "California", 1, 70), + Row("Hello", 30, "New York", 1, 30), + Row("John", 25, "Ontario", 1, 25), + Row("Jane", 20, "Quebec", 1, 20)) + // Compare the results + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) + assert(results.sorted.sameElements(expectedResults.sorted)) + + // Retrieve the logical plan + val logicalPlan: LogicalPlan = frame.queryExecution.logical + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['name, 'age, 'state] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("name"), + UnresolvedAttribute("age"), + UnresolvedAttribute("state")), + RELATION_TEST_TABLE))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + +- 'Aggregate ['age AS age#9], ['COUNT(*) AS count()#8, 'age AS age#10] + +- 'UnresolvedRelation [relation], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_TEST_TABLE))) + + val expectedPlan = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } +// +// test("test multiple trendline sma commands") { +// val frame = sql(s""" +// | source = $testTable | trendline sort + age sma(2, age) as two_points_sma sma(3, age) as three_points_sma | fields name, age, two_points_sma, three_points_sma +// | """.stripMargin) +// +// assert(frame.columns.sameElements(Array("name", "age", "two_points_sma", "three_points_sma"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Jane", 20, null, null), +// Row("John", 25, 22.5, null), +// Row("Hello", 30, 27.5, 25.0), +// Row("Jake", 70, 50.0, 41.666666666666664)) +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// +// // Retrieve the logical plan +// val logicalPlan: LogicalPlan = frame.queryExecution.logical +// val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) +// val nameField = UnresolvedAttribute("name") +// val ageField = UnresolvedAttribute("age") +// val ageTwoPointsSmaField = UnresolvedAttribute("two_points_sma") +// val ageThreePointsSmaField = UnresolvedAttribute("three_points_sma") +// val sort = Sort(Seq(SortOrder(ageField, Ascending)), global = true, table) +// val twoPointsCountWindow = new WindowExpression( +// UnresolvedFunction("COUNT", Seq(Literal(1)), isDistinct = false), +// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) +// val twoPointsSmaWindow = WindowExpression( +// UnresolvedFunction("AVG", Seq(ageField), isDistinct = false), +// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) +// val threePointsCountWindow = new WindowExpression( +// UnresolvedFunction("COUNT", Seq(Literal(1)), isDistinct = false), +// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-2), CurrentRow))) +// val threePointsSmaWindow = WindowExpression( +// UnresolvedFunction("AVG", Seq(ageField), isDistinct = false), +// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-2), CurrentRow))) +// val twoPointsCaseWhen = CaseWhen( +// Seq((LessThan(twoPointsCountWindow, Literal(2)), Literal(null))), +// twoPointsSmaWindow) +// val threePointsCaseWhen = CaseWhen( +// Seq((LessThan(threePointsCountWindow, Literal(3)), Literal(null))), +// threePointsSmaWindow) +// val trendlineProjectList = Seq( +// UnresolvedStar(None), +// Alias(twoPointsCaseWhen, "two_points_sma")(), +// Alias(threePointsCaseWhen, "three_points_sma")()) +// val expectedPlan = Project( +// Seq(nameField, ageField, ageTwoPointsSmaField, ageThreePointsSmaField), +// Project(trendlineProjectList, sort)) +// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) +// } +// +// test("test trendline sma command on evaluated column") { +// val frame = sql(s""" +// | source = $testTable | eval doubled_age = age * 2 | trendline sort + age sma(2, doubled_age) as doubled_age_sma | fields name, doubled_age, doubled_age_sma +// | """.stripMargin) +// +// assert(frame.columns.sameElements(Array("name", "doubled_age", "doubled_age_sma"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Jane", 40, null), +// Row("John", 50, 45.0), +// Row("Hello", 60, 55.0), +// Row("Jake", 140, 100.0)) +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// +// // Retrieve the logical plan +// val logicalPlan: LogicalPlan = frame.queryExecution.logical +// val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) +// val nameField = UnresolvedAttribute("name") +// val ageField = UnresolvedAttribute("age") +// val doubledAgeField = UnresolvedAttribute("doubled_age") +// val doubledAgeSmaField = UnresolvedAttribute("doubled_age_sma") +// val evalProject = Project( +// Seq( +// UnresolvedStar(None), +// Alias( +// UnresolvedFunction("*", Seq(ageField, Literal(2)), isDistinct = false), +// "doubled_age")()), +// table) +// val sort = Sort(Seq(SortOrder(ageField, Ascending)), global = true, evalProject) +// val countWindow = new WindowExpression( +// UnresolvedFunction("COUNT", Seq(Literal(1)), isDistinct = false), +// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) +// val doubleAgeSmaWindow = WindowExpression( +// UnresolvedFunction("AVG", Seq(doubledAgeField), isDistinct = false), +// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) +// val caseWhen = +// CaseWhen(Seq((LessThan(countWindow, Literal(2)), Literal(null))), doubleAgeSmaWindow) +// val trendlineProjectList = +// Seq(UnresolvedStar(None), Alias(caseWhen, "doubled_age_sma")()) +// val expectedPlan = Project( +// Seq(nameField, doubledAgeField, doubledAgeSmaField), +// Project(trendlineProjectList, sort)) +// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) +// } +// +// test("test trendline sma command chaining") { +// val frame = sql(s""" +// | source = $testTable | eval age_1 = age, age_2 = age | trendline sort - age_1 sma(3, age_1) | trendline sort + age_2 sma(3, age_2) +// | """.stripMargin) +// +// assert( +// frame.columns.sameElements( +// Array( +// "name", +// "age", +// "state", +// "country", +// "year", +// "month", +// "age_1", +// "age_2", +// "age_1_trendline", +// "age_2_trendline"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Hello", 30, "New York", "USA", 2023, 4, 30, 30, null, 25.0), +// Row("Jake", 70, "California", "USA", 2023, 4, 70, 70, null, 41.666666666666664), +// Row("Jane", 20, "Quebec", "Canada", 2023, 4, 20, 20, 25.0, null), +// Row("John", 25, "Ontario", "Canada", 2023, 4, 25, 25, 41.666666666666664, null)) +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// } +// +// test("test trendline wma command with sort field and without alias") { +// val frame = sql(s""" +// | source = $testTable | trendline sort + age wma(3, age) +// | """.stripMargin) +// +// // Compare the headers +// assert( +// frame.columns.sameElements( +// Array("name", "age", "state", "country", "year", "month", "age_trendline"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Jane", 20, "Quebec", "Canada", 2023, 4, null), +// Row("John", 25, "Ontario", "Canada", 2023, 4, null), +// Row("Hello", 30, "New York", "USA", 2023, 4, 26.666666666666668), +// Row("Jake", 70, "California", "USA", 2023, 4, 49.166666666666664)) +// +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// +// // Compare the logical plans +// val logicalPlan: LogicalPlan = frame.queryExecution.logical +// val dividend = Add( +// Add( +// getNthValueAggregation("age", "age", 1, -2), +// getNthValueAggregation("age", "age", 2, -2)), +// getNthValueAggregation("age", "age", 3, -2)) +// val wmaExpression = Divide(dividend, Literal(6)) +// val trendlineProjectList = Seq(UnresolvedStar(None), Alias(wmaExpression, "age_trendline")()) +// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) +// val sortedTable = Sort( +// Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), +// global = true, +// unresolvedRelation) +// val expectedPlan = +// Project(Seq(UnresolvedStar(None)), Project(trendlineProjectList, sortedTable)) +// +// /** +// * Expected logical plan: 'Project [*] +- 'Project [*, ((( ('nth_value('age, 1) +// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, +// * currentrow$())) * 1) + ('nth_value('age, 2) windowspecdefinition('age ASC NULLS FIRST, +// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 2)) + ('nth_value('age, 3) +// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, +// * currentrow$())) * 3)) / 6) AS age_trendline#185] +- 'Sort ['age ASC NULLS FIRST], true +- +// * 'UnresolvedRelation [spark_catalog, default, flint_ppl_test], [], false +// */ +// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) +// } +// +// test("test trendline wma command with sort field and with alias") { +// val frame = sql(s""" +// | source = $testTable | trendline sort + age wma(3, age) as trendline_alias +// | """.stripMargin) +// +// // Compare the headers +// assert( +// frame.columns.sameElements( +// Array("name", "age", "state", "country", "year", "month", "trendline_alias"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Jane", 20, "Quebec", "Canada", 2023, 4, null), +// Row("John", 25, "Ontario", "Canada", 2023, 4, null), +// Row("Hello", 30, "New York", "USA", 2023, 4, 26.666666666666668), +// Row("Jake", 70, "California", "USA", 2023, 4, 49.166666666666664)) +// +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// +// // Compare the logical plans +// val logicalPlan: LogicalPlan = frame.queryExecution.logical +// val dividend = Add( +// Add( +// getNthValueAggregation("age", "age", 1, -2), +// getNthValueAggregation("age", "age", 2, -2)), +// getNthValueAggregation("age", "age", 3, -2)) +// val wmaExpression = Divide(dividend, Literal(6)) +// val trendlineProjectList = +// Seq(UnresolvedStar(None), Alias(wmaExpression, "trendline_alias")()) +// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) +// val sortedTable = Sort( +// Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), +// global = true, +// unresolvedRelation) +// val expectedPlan = +// Project(Seq(UnresolvedStar(None)), Project(trendlineProjectList, sortedTable)) +// +// /** +// * 'Project [*] +- 'Project [*, ((( ('nth_value('age, 1) windowspecdefinition('age ASC NULLS +// * FIRST, specifiedwindowframe(RowFrame, -2, currentrow$())) * 1) + ('nth_value('age, 2) +// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, +// * currentrow$())) * 2)) + ('nth_value('age, 3) windowspecdefinition('age ASC NULLS FIRST, +// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 3)) / 6) AS trendline_alias#185] +- +// * 'Sort ['age ASC NULLS FIRST], true +- 'UnresolvedRelation [spark_catalog, default, +// * flint_ppl_test], [], false +// */ +// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) +// } +// +// test("test multiple trendline wma commands") { +// val frame = sql(s""" +// | source = $testTable | trendline sort + age wma(2, age) as two_points_wma wma(3, age) as three_points_wma +// | """.stripMargin) +// +// // Compare the headers +// assert( +// frame.columns.sameElements( +// Array( +// "name", +// "age", +// "state", +// "country", +// "year", +// "month", +// "two_points_wma", +// "three_points_wma"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Jane", 20, "Quebec", "Canada", 2023, 4, null, null), +// Row("John", 25, "Ontario", "Canada", 2023, 4, 23.333333333333332, null), +// Row("Hello", 30, "New York", "USA", 2023, 4, 28.333333333333332, 26.666666666666668), +// Row("Jake", 70, "California", "USA", 2023, 4, 56.666666666666664, 49.166666666666664)) +// +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// +// // Compare the logical plans +// val logicalPlan: LogicalPlan = frame.queryExecution.logical +// +// val dividendTwo = Add( +// getNthValueAggregation("age", "age", 1, -1), +// getNthValueAggregation("age", "age", 2, -1)) +// val twoPointsExpression = Divide(dividendTwo, Literal(3)) +// +// val dividend = Add( +// Add( +// getNthValueAggregation("age", "age", 1, -2), +// getNthValueAggregation("age", "age", 2, -2)), +// getNthValueAggregation("age", "age", 3, -2)) +// val threePointsExpression = Divide(dividend, Literal(6)) +// +// val trendlineProjectList = Seq( +// UnresolvedStar(None), +// Alias(twoPointsExpression, "two_points_wma")(), +// Alias(threePointsExpression, "three_points_wma")()) +// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) +// val sortedTable = Sort( +// Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), +// global = true, +// unresolvedRelation) +// val expectedPlan = +// Project(Seq(UnresolvedStar(None)), Project(trendlineProjectList, sortedTable)) +// +// /** +// * 'Project [*] +- 'Project [*, (( ('nth_value('age, 1) windowspecdefinition('age ASC NULLS +// * FIRST, specifiedwindowframe(RowFrame, -1, currentrow$())) * 1) + ('nth_value('age, 2) +// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, +// * currentrow$())) * 2)) / 3) AS two_points_wma#247, +// * +// * ((( ('nth_value('age, 1) windowspecdefinition('age ASC NULLS FIRST, +// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 1) + ('nth_value('age, 2) +// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, +// * currentrow$())) * 2)) + ('nth_value('age, 3) windowspecdefinition('age ASC NULLS FIRST, +// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 3)) / 6) AS three_points_wma#248] +- +// * 'Sort ['age ASC NULLS FIRST], true +- 'UnresolvedRelation [spark_catalog, default, +// * flint_ppl_test], [], false +// */ +// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) +// } +// +// test("test trendline wma command on evaluated column") { +// val frame = sql(s""" +// | source = $testTable | eval doubled_age = age * 2 | trendline sort + age wma(2, doubled_age) as doubled_age_wma | fields name, doubled_age, doubled_age_wma +// | """.stripMargin) +// +// // Compare the headers +// assert(frame.columns.sameElements(Array("name", "doubled_age", "doubled_age_wma"))) +// // Retrieve the results +// val results: Array[Row] = frame.collect() +// val expectedResults: Array[Row] = +// Array( +// Row("Jane", 40, null), +// Row("John", 50, 46.666666666666664), +// Row("Hello", 60, 56.666666666666664), +// Row("Jake", 140, 113.33333333333333)) +// +// // Compare the results +// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) +// assert(results.sorted.sameElements(expectedResults.sorted)) +// +// // Compare the logical plans +// val logicalPlan: LogicalPlan = frame.queryExecution.logical +// val dividend = Add( +// getNthValueAggregation("doubled_age", "age", 1, -1), +// getNthValueAggregation("doubled_age", "age", 2, -1)) +// val wmaExpression = Divide(dividend, Literal(3)) +// val trendlineProjectList = +// Seq(UnresolvedStar(None), Alias(wmaExpression, "doubled_age_wma")()) +// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) +// val doubledAged = Alias( +// UnresolvedFunction( +// seq("*"), +// seq(UnresolvedAttribute("age"), Literal(2)), +// isDistinct = false), +// "doubled_age")() +// val doubleAgeProject = Project(seq(UnresolvedStar(None), doubledAged), unresolvedRelation) +// val sortedTable = +// Sort(Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), global = true, doubleAgeProject) +// val expectedPlan = Project( +// Seq( +// UnresolvedAttribute("name"), +// UnresolvedAttribute("doubled_age"), +// UnresolvedAttribute("doubled_age_wma")), +// Project(trendlineProjectList, sortedTable)) +// +// /** +// * 'Project ['name, 'doubled_age, 'doubled_age_wma] +- 'Project [*, (( +// * ('nth_value('doubled_age, 1) windowspecdefinition('age ASC NULLS FIRST, +// * specifiedwindowframe(RowFrame, -1, currentrow$())) * 1) + ('nth_value('doubled_age, 2) +// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, +// * currentrow$())) * 2)) / 3) AS doubled_age_wma#288] +- 'Sort ['age ASC NULLS FIRST], true +- +// * 'Project [*, '`*`('age, 2) AS doubled_age#287] +- 'UnresolvedRelation [spark_catalog, +// * default, flint_ppl_test], [], false +// */ +// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) +// +// } +// +// test("test invalid wma command with negative dataPoint value") { +// val exception = intercept[ParseException](sql(s""" +// | source = $testTable | trendline sort + age wma(-3, age) +// | """.stripMargin)) +// assert(exception.getMessage contains "[PARSE_SYNTAX_ERROR] Syntax error") +// } + + private def getNthValueAggregation( + dataField: String, + sortField: String, + lookBackPos: Int, + lookBackRange: Int): Expression = { + Multiply( + WindowExpression( + UnresolvedFunction( + "nth_value", + Seq(UnresolvedAttribute(dataField), Literal(lookBackPos)), + isDistinct = false), + WindowSpecDefinition( + Seq(), + seq(SortUtils.sortOrder(UnresolvedAttribute(sortField), true)), + SpecifiedWindowFrame(RowFrame, Literal(lookBackRange), CurrentRow))), + Literal(lookBackPos)) + } + +} diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index 37b992c00..c5ef8e2da 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -91,18 +91,18 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) - val result = Project( + val expectedPlan = Project( Seq(UnresolvedStar(None)), DataFrameDropColumns( T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - // scalastyle:off - println(logicalPlan) - println(result) - // scalastyle:on +// // scalastyle:off +// println(logicalPlan) +// println(expectedPlan) +// // scalastyle:on - comparePlans(logicalPlan, result, checkAnalysis = false) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } // @formatter:off @@ -159,13 +159,13 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))) - val result = Project( + val expectedPlan = Project( Seq(UnresolvedStar(None)), DataFrameDropColumns( T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - comparePlans(logicalPlan, result, checkAnalysis = false) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } // @formatter:off @@ -229,13 +229,13 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_EMPLOYEES))))) - val result = Project( + val expectedPlan = Project( Seq(UnresolvedStar(None)), DataFrameDropColumns( T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - comparePlans(logicalPlan, result, checkAnalysis = false) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } // @formatter:off @@ -324,7 +324,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), Project(Seq(UnresolvedAttribute("dept")), RELATION_EMPLOYEES))) - val joinWithSecondAppendCol = Project( + val expectedPlan = Project( Seq(UnresolvedStar(None)), DataFrameDropColumns( T12_COLUMNS_SEQ, @@ -335,7 +335,7 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite Some(T12_JOIN_CONDITION), JoinHint.NONE))) - comparePlans(logicalPlan, joinWithSecondAppendCol, checkAnalysis = false) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } } From da242f5f9e23cb86aa03364e64960df78fba8bd9 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Sat, 14 Dec 2024 14:06:04 -0800 Subject: [PATCH 29/37] Test cases 3 4 Signed-off-by: Andy Kwok --- .../ppl/FlintSparkPPLAppendColITSuite.scala | 544 +++++------------- 1 file changed, 150 insertions(+), 394 deletions(-) diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala index 436ef2b6e..ed1788f9c 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala @@ -126,7 +126,6 @@ class FlintSparkPPLAppendColITSuite comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } - test("test AppendCol with transformation on main-search") { val frame = sql(s""" | source = $testTable | FIELDS name, age, state | APPENDCOL [stats count() by age] @@ -185,399 +184,156 @@ class FlintSparkPPLAppendColITSuite Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } -// -// test("test multiple trendline sma commands") { -// val frame = sql(s""" -// | source = $testTable | trendline sort + age sma(2, age) as two_points_sma sma(3, age) as three_points_sma | fields name, age, two_points_sma, three_points_sma -// | """.stripMargin) -// -// assert(frame.columns.sameElements(Array("name", "age", "two_points_sma", "three_points_sma"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Jane", 20, null, null), -// Row("John", 25, 22.5, null), -// Row("Hello", 30, 27.5, 25.0), -// Row("Jake", 70, 50.0, 41.666666666666664)) -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// -// // Retrieve the logical plan -// val logicalPlan: LogicalPlan = frame.queryExecution.logical -// val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) -// val nameField = UnresolvedAttribute("name") -// val ageField = UnresolvedAttribute("age") -// val ageTwoPointsSmaField = UnresolvedAttribute("two_points_sma") -// val ageThreePointsSmaField = UnresolvedAttribute("three_points_sma") -// val sort = Sort(Seq(SortOrder(ageField, Ascending)), global = true, table) -// val twoPointsCountWindow = new WindowExpression( -// UnresolvedFunction("COUNT", Seq(Literal(1)), isDistinct = false), -// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) -// val twoPointsSmaWindow = WindowExpression( -// UnresolvedFunction("AVG", Seq(ageField), isDistinct = false), -// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) -// val threePointsCountWindow = new WindowExpression( -// UnresolvedFunction("COUNT", Seq(Literal(1)), isDistinct = false), -// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-2), CurrentRow))) -// val threePointsSmaWindow = WindowExpression( -// UnresolvedFunction("AVG", Seq(ageField), isDistinct = false), -// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-2), CurrentRow))) -// val twoPointsCaseWhen = CaseWhen( -// Seq((LessThan(twoPointsCountWindow, Literal(2)), Literal(null))), -// twoPointsSmaWindow) -// val threePointsCaseWhen = CaseWhen( -// Seq((LessThan(threePointsCountWindow, Literal(3)), Literal(null))), -// threePointsSmaWindow) -// val trendlineProjectList = Seq( -// UnresolvedStar(None), -// Alias(twoPointsCaseWhen, "two_points_sma")(), -// Alias(threePointsCaseWhen, "three_points_sma")()) -// val expectedPlan = Project( -// Seq(nameField, ageField, ageTwoPointsSmaField, ageThreePointsSmaField), -// Project(trendlineProjectList, sort)) -// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) -// } -// -// test("test trendline sma command on evaluated column") { -// val frame = sql(s""" -// | source = $testTable | eval doubled_age = age * 2 | trendline sort + age sma(2, doubled_age) as doubled_age_sma | fields name, doubled_age, doubled_age_sma -// | """.stripMargin) -// -// assert(frame.columns.sameElements(Array("name", "doubled_age", "doubled_age_sma"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Jane", 40, null), -// Row("John", 50, 45.0), -// Row("Hello", 60, 55.0), -// Row("Jake", 140, 100.0)) -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// -// // Retrieve the logical plan -// val logicalPlan: LogicalPlan = frame.queryExecution.logical -// val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) -// val nameField = UnresolvedAttribute("name") -// val ageField = UnresolvedAttribute("age") -// val doubledAgeField = UnresolvedAttribute("doubled_age") -// val doubledAgeSmaField = UnresolvedAttribute("doubled_age_sma") -// val evalProject = Project( -// Seq( -// UnresolvedStar(None), -// Alias( -// UnresolvedFunction("*", Seq(ageField, Literal(2)), isDistinct = false), -// "doubled_age")()), -// table) -// val sort = Sort(Seq(SortOrder(ageField, Ascending)), global = true, evalProject) -// val countWindow = new WindowExpression( -// UnresolvedFunction("COUNT", Seq(Literal(1)), isDistinct = false), -// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) -// val doubleAgeSmaWindow = WindowExpression( -// UnresolvedFunction("AVG", Seq(doubledAgeField), isDistinct = false), -// WindowSpecDefinition(Seq(), Seq(), SpecifiedWindowFrame(RowFrame, Literal(-1), CurrentRow))) -// val caseWhen = -// CaseWhen(Seq((LessThan(countWindow, Literal(2)), Literal(null))), doubleAgeSmaWindow) -// val trendlineProjectList = -// Seq(UnresolvedStar(None), Alias(caseWhen, "doubled_age_sma")()) -// val expectedPlan = Project( -// Seq(nameField, doubledAgeField, doubledAgeSmaField), -// Project(trendlineProjectList, sort)) -// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) -// } -// -// test("test trendline sma command chaining") { -// val frame = sql(s""" -// | source = $testTable | eval age_1 = age, age_2 = age | trendline sort - age_1 sma(3, age_1) | trendline sort + age_2 sma(3, age_2) -// | """.stripMargin) -// -// assert( -// frame.columns.sameElements( -// Array( -// "name", -// "age", -// "state", -// "country", -// "year", -// "month", -// "age_1", -// "age_2", -// "age_1_trendline", -// "age_2_trendline"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Hello", 30, "New York", "USA", 2023, 4, 30, 30, null, 25.0), -// Row("Jake", 70, "California", "USA", 2023, 4, 70, 70, null, 41.666666666666664), -// Row("Jane", 20, "Quebec", "Canada", 2023, 4, 20, 20, 25.0, null), -// Row("John", 25, "Ontario", "Canada", 2023, 4, 25, 25, 41.666666666666664, null)) -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// } -// -// test("test trendline wma command with sort field and without alias") { -// val frame = sql(s""" -// | source = $testTable | trendline sort + age wma(3, age) -// | """.stripMargin) -// -// // Compare the headers -// assert( -// frame.columns.sameElements( -// Array("name", "age", "state", "country", "year", "month", "age_trendline"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Jane", 20, "Quebec", "Canada", 2023, 4, null), -// Row("John", 25, "Ontario", "Canada", 2023, 4, null), -// Row("Hello", 30, "New York", "USA", 2023, 4, 26.666666666666668), -// Row("Jake", 70, "California", "USA", 2023, 4, 49.166666666666664)) -// -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// -// // Compare the logical plans -// val logicalPlan: LogicalPlan = frame.queryExecution.logical -// val dividend = Add( -// Add( -// getNthValueAggregation("age", "age", 1, -2), -// getNthValueAggregation("age", "age", 2, -2)), -// getNthValueAggregation("age", "age", 3, -2)) -// val wmaExpression = Divide(dividend, Literal(6)) -// val trendlineProjectList = Seq(UnresolvedStar(None), Alias(wmaExpression, "age_trendline")()) -// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) -// val sortedTable = Sort( -// Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), -// global = true, -// unresolvedRelation) -// val expectedPlan = -// Project(Seq(UnresolvedStar(None)), Project(trendlineProjectList, sortedTable)) -// -// /** -// * Expected logical plan: 'Project [*] +- 'Project [*, ((( ('nth_value('age, 1) -// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -// * currentrow$())) * 1) + ('nth_value('age, 2) windowspecdefinition('age ASC NULLS FIRST, -// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 2)) + ('nth_value('age, 3) -// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -// * currentrow$())) * 3)) / 6) AS age_trendline#185] +- 'Sort ['age ASC NULLS FIRST], true +- -// * 'UnresolvedRelation [spark_catalog, default, flint_ppl_test], [], false -// */ -// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) -// } -// -// test("test trendline wma command with sort field and with alias") { -// val frame = sql(s""" -// | source = $testTable | trendline sort + age wma(3, age) as trendline_alias -// | """.stripMargin) -// -// // Compare the headers -// assert( -// frame.columns.sameElements( -// Array("name", "age", "state", "country", "year", "month", "trendline_alias"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Jane", 20, "Quebec", "Canada", 2023, 4, null), -// Row("John", 25, "Ontario", "Canada", 2023, 4, null), -// Row("Hello", 30, "New York", "USA", 2023, 4, 26.666666666666668), -// Row("Jake", 70, "California", "USA", 2023, 4, 49.166666666666664)) -// -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// -// // Compare the logical plans -// val logicalPlan: LogicalPlan = frame.queryExecution.logical -// val dividend = Add( -// Add( -// getNthValueAggregation("age", "age", 1, -2), -// getNthValueAggregation("age", "age", 2, -2)), -// getNthValueAggregation("age", "age", 3, -2)) -// val wmaExpression = Divide(dividend, Literal(6)) -// val trendlineProjectList = -// Seq(UnresolvedStar(None), Alias(wmaExpression, "trendline_alias")()) -// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) -// val sortedTable = Sort( -// Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), -// global = true, -// unresolvedRelation) -// val expectedPlan = -// Project(Seq(UnresolvedStar(None)), Project(trendlineProjectList, sortedTable)) -// -// /** -// * 'Project [*] +- 'Project [*, ((( ('nth_value('age, 1) windowspecdefinition('age ASC NULLS -// * FIRST, specifiedwindowframe(RowFrame, -2, currentrow$())) * 1) + ('nth_value('age, 2) -// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -// * currentrow$())) * 2)) + ('nth_value('age, 3) windowspecdefinition('age ASC NULLS FIRST, -// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 3)) / 6) AS trendline_alias#185] +- -// * 'Sort ['age ASC NULLS FIRST], true +- 'UnresolvedRelation [spark_catalog, default, -// * flint_ppl_test], [], false -// */ -// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) -// } -// -// test("test multiple trendline wma commands") { -// val frame = sql(s""" -// | source = $testTable | trendline sort + age wma(2, age) as two_points_wma wma(3, age) as three_points_wma -// | """.stripMargin) -// -// // Compare the headers -// assert( -// frame.columns.sameElements( -// Array( -// "name", -// "age", -// "state", -// "country", -// "year", -// "month", -// "two_points_wma", -// "three_points_wma"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Jane", 20, "Quebec", "Canada", 2023, 4, null, null), -// Row("John", 25, "Ontario", "Canada", 2023, 4, 23.333333333333332, null), -// Row("Hello", 30, "New York", "USA", 2023, 4, 28.333333333333332, 26.666666666666668), -// Row("Jake", 70, "California", "USA", 2023, 4, 56.666666666666664, 49.166666666666664)) -// -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// -// // Compare the logical plans -// val logicalPlan: LogicalPlan = frame.queryExecution.logical -// -// val dividendTwo = Add( -// getNthValueAggregation("age", "age", 1, -1), -// getNthValueAggregation("age", "age", 2, -1)) -// val twoPointsExpression = Divide(dividendTwo, Literal(3)) -// -// val dividend = Add( -// Add( -// getNthValueAggregation("age", "age", 1, -2), -// getNthValueAggregation("age", "age", 2, -2)), -// getNthValueAggregation("age", "age", 3, -2)) -// val threePointsExpression = Divide(dividend, Literal(6)) -// -// val trendlineProjectList = Seq( -// UnresolvedStar(None), -// Alias(twoPointsExpression, "two_points_wma")(), -// Alias(threePointsExpression, "three_points_wma")()) -// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) -// val sortedTable = Sort( -// Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), -// global = true, -// unresolvedRelation) -// val expectedPlan = -// Project(Seq(UnresolvedStar(None)), Project(trendlineProjectList, sortedTable)) -// -// /** -// * 'Project [*] +- 'Project [*, (( ('nth_value('age, 1) windowspecdefinition('age ASC NULLS -// * FIRST, specifiedwindowframe(RowFrame, -1, currentrow$())) * 1) + ('nth_value('age, 2) -// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, -// * currentrow$())) * 2)) / 3) AS two_points_wma#247, -// * -// * ((( ('nth_value('age, 1) windowspecdefinition('age ASC NULLS FIRST, -// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 1) + ('nth_value('age, 2) -// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -// * currentrow$())) * 2)) + ('nth_value('age, 3) windowspecdefinition('age ASC NULLS FIRST, -// * specifiedwindowframe(RowFrame, -2, currentrow$())) * 3)) / 6) AS three_points_wma#248] +- -// * 'Sort ['age ASC NULLS FIRST], true +- 'UnresolvedRelation [spark_catalog, default, -// * flint_ppl_test], [], false -// */ -// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) -// } -// -// test("test trendline wma command on evaluated column") { -// val frame = sql(s""" -// | source = $testTable | eval doubled_age = age * 2 | trendline sort + age wma(2, doubled_age) as doubled_age_wma | fields name, doubled_age, doubled_age_wma -// | """.stripMargin) -// -// // Compare the headers -// assert(frame.columns.sameElements(Array("name", "doubled_age", "doubled_age_wma"))) -// // Retrieve the results -// val results: Array[Row] = frame.collect() -// val expectedResults: Array[Row] = -// Array( -// Row("Jane", 40, null), -// Row("John", 50, 46.666666666666664), -// Row("Hello", 60, 56.666666666666664), -// Row("Jake", 140, 113.33333333333333)) -// -// // Compare the results -// implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) -// assert(results.sorted.sameElements(expectedResults.sorted)) -// -// // Compare the logical plans -// val logicalPlan: LogicalPlan = frame.queryExecution.logical -// val dividend = Add( -// getNthValueAggregation("doubled_age", "age", 1, -1), -// getNthValueAggregation("doubled_age", "age", 2, -1)) -// val wmaExpression = Divide(dividend, Literal(3)) -// val trendlineProjectList = -// Seq(UnresolvedStar(None), Alias(wmaExpression, "doubled_age_wma")()) -// val unresolvedRelation = UnresolvedRelation(testTable.split("\\.").toSeq) -// val doubledAged = Alias( -// UnresolvedFunction( -// seq("*"), -// seq(UnresolvedAttribute("age"), Literal(2)), -// isDistinct = false), -// "doubled_age")() -// val doubleAgeProject = Project(seq(UnresolvedStar(None), doubledAged), unresolvedRelation) -// val sortedTable = -// Sort(Seq(SortOrder(UnresolvedAttribute("age"), Ascending)), global = true, doubleAgeProject) -// val expectedPlan = Project( -// Seq( -// UnresolvedAttribute("name"), -// UnresolvedAttribute("doubled_age"), -// UnresolvedAttribute("doubled_age_wma")), -// Project(trendlineProjectList, sortedTable)) -// -// /** -// * 'Project ['name, 'doubled_age, 'doubled_age_wma] +- 'Project [*, (( -// * ('nth_value('doubled_age, 1) windowspecdefinition('age ASC NULLS FIRST, -// * specifiedwindowframe(RowFrame, -1, currentrow$())) * 1) + ('nth_value('doubled_age, 2) -// * windowspecdefinition('age ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, -// * currentrow$())) * 2)) / 3) AS doubled_age_wma#288] +- 'Sort ['age ASC NULLS FIRST], true +- -// * 'Project [*, '`*`('age, 2) AS doubled_age#287] +- 'UnresolvedRelation [spark_catalog, -// * default, flint_ppl_test], [], false -// */ -// comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) -// -// } -// -// test("test invalid wma command with negative dataPoint value") { -// val exception = intercept[ParseException](sql(s""" -// | source = $testTable | trendline sort + age wma(-3, age) -// | """.stripMargin)) -// assert(exception.getMessage contains "[PARSE_SYNTAX_ERROR] Syntax error") -// } - - private def getNthValueAggregation( - dataField: String, - sortField: String, - lookBackPos: Int, - lookBackRange: Int): Expression = { - Multiply( - WindowExpression( - UnresolvedFunction( - "nth_value", - Seq(UnresolvedAttribute(dataField), Literal(lookBackPos)), - isDistinct = false), - WindowSpecDefinition( - Seq(), - seq(SortUtils.sortOrder(UnresolvedAttribute(sortField), true)), - SpecifiedWindowFrame(RowFrame, Literal(lookBackRange), CurrentRow))), - Literal(lookBackPos)) + + test("test AppendCol with chained sub-search") { + val frame = sql(s""" + | source = $testTable | FIELDS name, age, state | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] + | """.stripMargin) + + assert(frame.columns.sameElements( + Array("name", "age", "state", "count()", "age"))) + // Retrieve the results + val results: Array[Row] = frame.collect() + val expectedResults: Array[Row] = + Array( + Row("Jake", 70, "California", 1, 70), + Row("Hello", 30, "New York", 1, 30), + Row("John", 25, "Ontario", 1, 25), + Row("Jane", 20, "Quebec", 1, 20)) + // Compare the results + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) + assert(results.sorted.sameElements(expectedResults.sorted)) + + // Retrieve the logical plan + val logicalPlan: LogicalPlan = frame.queryExecution.logical + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['age, 'dept, 'salary] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("name"), + UnresolvedAttribute("age"), + UnresolvedAttribute("state")), + RELATION_TEST_TABLE))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + +- 'DataFrameDropColumns ['m] + +- 'Project [*, 1 AS m#430] + +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + +- 'UnresolvedRelation [employees], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + Seq(UnresolvedAttribute("m")), + Project( + Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_TEST_TABLE))))) + + val expectedPlan = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test multiple AppendCol clauses") { + val frame = sql(s""" + | source = $testTable | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS state] + | """.stripMargin) + + assert(frame.columns.sameElements( + Array("name", "age", "count()", "age", "state"))) + // Retrieve the results + val results: Array[Row] = frame.collect() + val expectedResults: Array[Row] = + Array( + Row("Jake", 70, 1, 70, "California"), + Row("Hello", 30, 1, 30, "New York"), + Row("John", 25, 1, 25, "Ontario"), + Row("Jane", 20, 1, 20, "Quebec")) + // Compare the results + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0)) + assert(results.sorted.sameElements(expectedResults.sorted)) + + // Retrieve the logical plan + val logicalPlan: LogicalPlan = frame.queryExecution.logical + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] + : +- 'Project ['name, 'age] + : +- 'UnresolvedRelation [employees], [], false + */ + val mainSearch = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq(UnresolvedAttribute("name"), UnresolvedAttribute("age")), + RELATION_TEST_TABLE))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#432, *] + +- 'DataFrameDropColumns ['m] + +- 'Project [*, 1 AS m#430] + +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] + +- 'UnresolvedRelation [employees], [], false + */ + val firstAppenCol = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + Seq(UnresolvedAttribute("m")), + Project( + Seq(UnresolvedStar(None), Alias(Literal(1), "m")()), + Aggregate(AGE_ALIAS :: Nil, Seq(COUNT_STAR, AGE_ALIAS), RELATION_TEST_TABLE))))) + + val joinWithFirstAppendCol = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join(mainSearch, firstAppenCol, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE)))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] + +- 'Project ['dept] + +- 'UnresolvedRelation [employees], [], false + */ + val secondAppendCol = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project(Seq(UnresolvedAttribute("state")), RELATION_TEST_TABLE))) + + val expectedPlan = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ, + Join( + joinWithFirstAppendCol, + secondAppendCol, + LeftOuter, + Some(T12_JOIN_CONDITION), + JoinHint.NONE))) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } } From 9194ca6052c30fee8e5360766fcc7e9f9f57674e Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 10:35:58 -0800 Subject: [PATCH 30/37] Test code comments Signed-off-by: Andy Kwok --- .../ppl/FlintSparkPPLAppendColITSuite.scala | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala index ed1788f9c..e2a8eb789 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala @@ -66,12 +66,15 @@ class FlintSparkPPLAppendColITSuite } } + /** + * The baseline test-case to make sure APPENDCOL( ) function works, + * when no transformation present on the main search, after the search command. + */ test("test AppendCol with NO transformation on main") { val frame = sql(s""" | source = $testTable | APPENDCOL [stats count() by age] | """.stripMargin) - assert( frame.columns.sameElements( Array("name", "age", "state", "country", "year", "month", "count()", "age"))) @@ -126,6 +129,10 @@ class FlintSparkPPLAppendColITSuite comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } + + /** + * To simulate the use-case when user attempt to attach an APPENDCOL command on a well established main search. + */ test("test AppendCol with transformation on main-search") { val frame = sql(s""" | source = $testTable | FIELDS name, age, state | APPENDCOL [stats count() by age] @@ -185,6 +192,9 @@ class FlintSparkPPLAppendColITSuite comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } + /** + * To simulate the situation when multiple PPL commands being applied on the sub-search. + */ test("test AppendCol with chained sub-search") { val frame = sql(s""" | source = $testTable | FIELDS name, age, state | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] @@ -230,7 +240,7 @@ class FlintSparkPPLAppendColITSuite +- 'DataFrameDropColumns ['m] +- 'Project [*, 1 AS m#430] +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] - +- 'UnresolvedRelation [employees], [], false + +- 'UnresolvedRelation [flint_ppl_test], [], false */ val t2 = SubqueryAlias( "T2", @@ -250,6 +260,10 @@ class FlintSparkPPLAppendColITSuite comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } + /** + * The use-case when user attempt to chain multiple APPENCOL command in a PPL, this is a common use case, + * when user prefer to show the statistic report alongside with the dataset. + */ test("test multiple AppendCol clauses") { val frame = sql(s""" | source = $testTable | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS state] @@ -275,7 +289,7 @@ class FlintSparkPPLAppendColITSuite :- 'SubqueryAlias T1 : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] : +- 'Project ['name, 'age] - : +- 'UnresolvedRelation [employees], [], false + : +- 'UnresolvedRelation [flint_ppl_test], [], false */ val mainSearch = SubqueryAlias( "T1", @@ -291,7 +305,7 @@ class FlintSparkPPLAppendColITSuite +- 'DataFrameDropColumns ['m] +- 'Project [*, 1 AS m#430] +- 'Aggregate ['age AS age#429], ['COUNT(*) AS count()#428, 'age AS age#429] - +- 'UnresolvedRelation [employees], [], false + +- 'UnresolvedRelation [flint_ppl_test], [], false */ val firstAppenCol = SubqueryAlias( "T2", @@ -315,7 +329,7 @@ class FlintSparkPPLAppendColITSuite +- 'SubqueryAlias T2 +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#553, *] +- 'Project ['dept] - +- 'UnresolvedRelation [employees], [], false + +- 'UnresolvedRelation [flint_ppl_test], [], false */ val secondAppendCol = SubqueryAlias( "T2", From 7e17ab97780ffe5e22a7a008f5e595ae21aef394 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 10:53:54 -0800 Subject: [PATCH 31/37] Code tidy Signed-off-by: Andy Kwok --- .../function/BuiltinFunctionName.java | 2 -- .../sql/ppl/CatalystQueryPlanVisitor.java | 15 ++++++++------- .../sql/ppl/utils/AppendColCatalystUtils.java | 19 ++++++++++--------- .../sql/ppl/utils/TrendlineCatalystUtils.java | 2 +- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java index b101e664c..47ed7da44 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java @@ -248,7 +248,6 @@ public enum BuiltinFunctionName { ISNULL(FunctionName.of("isnull")), ISPRESENT(FunctionName.of("ispresent")), - ROW_NUMBER(FunctionName.of("row_number")), RANK(FunctionName.of("rank")), DENSE_RANK(FunctionName.of("dense_rank")), @@ -333,7 +332,6 @@ public FunctionName getName() { .put("percentile", BuiltinFunctionName.PERCENTILE) .put("percentile_approx", BuiltinFunctionName.PERCENTILE_APPROX) .put("approx_count_distinct", BuiltinFunctionName.APPROX_COUNT_DISTINCT) - .put("row_number", BuiltinFunctionName.ROW_NUMBER) .build(); public static Optional of(String str) { diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 912be3dee..028340ec6 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -282,31 +282,32 @@ public LogicalPlan visitAppendCol(AppendCol node, CatalystPlanContext context) { final Node mainSearchNode = node.getChild().get(0); final Node subSearchNode = node.getSubSearch(); - // Traverse to look for relation clause then append it into the sub-search. + // Traverse to look for relation clause, then append it into the sub-search. Relation relation = AppendColCatalystUtils.retrieveRelationClause(mainSearchNode); AppendColCatalystUtils.appendRelationClause(node.getSubSearch(), relation); - // Add apply a dropColumns if override present, then add * with ROW_NUMBER - LogicalPlan leftTemp = mainSearchNode.accept(this, context); - var mainSearchWithRowNumber = AppendColCatalystUtils.getRowNumStarProjection(context, leftTemp, TABLE_LHS); + // Apply an additional projection layer on main-search to provide natural order. + LogicalPlan mainSearch = mainSearchNode.accept(this, context); + var mainSearchWithRowNumber = AppendColCatalystUtils.getRowNumStarProjection(context, mainSearch, TABLE_LHS); context.withSubqueryAlias(mainSearchWithRowNumber); context.apply(left -> { - // Add a new projection layer with * and ROW_NUMBER (Sub-search) + // Apply an additional projection layer on sub-search to provide natural order. LogicalPlan subSearch = subSearchNode.accept(this, context); var subSearchWithRowNumber = AppendColCatalystUtils.getRowNumStarProjection(context, subSearch, TABLE_RHS); context.withSubqueryAlias(subSearchWithRowNumber); context.retainAllNamedParseExpressions(p -> p); context.retainAllPlans(p -> p); - // Composite the join clause + + // Join both Main and Sub search with _ROW_NUMBER_ column LogicalPlan joinedQuery = join( mainSearchWithRowNumber, subSearchWithRowNumber, Join.JoinType.LEFT, Optional.of(new EqualTo(t1Attr, t2Attr)), new Join.JoinHint()); - // Remove the APPEND_ID and duplicated field on T1 if override option is true. + // Remove the APPEND_ID and duplicated field on T1 if override option present. if (node.override) { List getoverridedlist = AppendColCatalystUtils.getoverridedlist(subSearchWithRowNumber, TABLE_LHS); fieldsToRemove.addAll(getoverridedlist); diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java index 699c05409..8892052b8 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/AppendColCatalystUtils.java @@ -5,9 +5,11 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedStar; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.Literal; import org.apache.spark.sql.catalyst.expressions.NamedExpression; import org.apache.spark.sql.catalyst.expressions.SortOrder; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.plans.logical.Project; import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias; import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias$; import org.apache.spark.sql.execution.CommandExecutionMode; @@ -33,14 +35,13 @@ public interface AppendColCatalystUtils { /** - * Response to traverse given subSearch Node till the last child, then append the Relation clause, - * in order to specify the data source || index. + * Responsible to traverse given subSearch Node till the last child, then append the Relation clause, + * in order to specify the data source || index for the subSearch. * @param subSearch User provided sub-search from APPENDCOL command. * @param relation Relation clause which represent the dataSource that this sub-search execute upon. */ static void appendRelationClause(Node subSearch, Relation relation) { - Relation table = new Relation(relation.getTableNames()); - // Replace it with a function to look up the search command and extract the index name. + final Relation table = new Relation(relation.getTableNames()); while (subSearch != null) { try { subSearch = subSearch.getChild().get(0); @@ -76,7 +77,7 @@ static Relation retrieveRelationClause(Node node) { /** - * Util method to perform analyzed() call against the given LogicalPlan to exact all fields + * Util method to perform analyzed() call against the given LogicalPlan to extract all fields * that will be projected upon the execution in the form of Java List with user provided schema prefix. * @param lp LogicalPlan instance to extract the projection fields from. * @param tableName the table || schema name being appended as part of the returned fields. @@ -97,15 +98,15 @@ static List getoverridedlist(LogicalPlan lp, String tableName) { } /** - * Helper method to first add an additional project clause to provide row_number, then wrap it SubqueryAlias and return. + * Helper method to first add an additional projection clause to provide row_number, then wrap it SubqueryAlias and return. * @param context Context object of the current Parser. * @param lp The Logical Plan instance which contains the query. * @param alias The name of the Alias clause. - * @return A subqeuryAlias instance which has row_number for natural ordering purpose. + * @return A subqueryAlias instance which has row_number for natural ordering purpose. */ static SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, LogicalPlan lp, String alias) { final SortOrder sortOrder = SortUtils.sortOrder( - new org.apache.spark.sql.catalyst.expressions.Literal( + new Literal( UTF8String.fromString("1"), DataTypes.StringType), false); final NamedExpression appendCol = WindowSpecTransformer.buildRowNumber(seq(), seq(sortOrder)); @@ -113,7 +114,7 @@ static SubqueryAlias getRowNumStarProjection(CatalystPlanContext context, Logica ? List.of(appendCol, new UnresolvedStar(Option.empty())) : List.of(appendCol); - final LogicalPlan lpWithProjection = new org.apache.spark.sql.catalyst.plans.logical.Project(seq( + final LogicalPlan lpWithProjection = new Project(seq( projectList), lp); return SubqueryAlias$.MODULE$.apply(alias, lpWithProjection); } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java index 1315165e5..647f4542e 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/TrendlineCatalystUtils.java @@ -153,7 +153,7 @@ private static NamedExpression getWMAComputationExpression(CatalystExpressionVis * @param expression The expression which will be evaluated. * @return An Alias instance with logical plan representation of `expression AS name`. */ - public static NamedExpression getAlias(String name, Expression expression) { + private static NamedExpression getAlias(String name, Expression expression) { return org.apache.spark.sql.catalyst.expressions.Alias$.MODULE$.apply(expression, name, NamedExpression.newExprId(), From a3623e8e736c7f35b26cdffb1f0677db1560243d Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 11:07:53 -0800 Subject: [PATCH 32/37] Code refactor Signed-off-by: Andy Kwok --- .../main/java/org/opensearch/sql/ast/tree/AppendCol.java | 4 ++++ .../sql/expression/function/BuiltinFunctionName.java | 1 + .../org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java | 6 +----- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java index fc6b774c0..b98d08caa 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ast/tree/AppendCol.java @@ -1,3 +1,7 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ package org.opensearch.sql.ast.tree; import com.google.common.collect.ImmutableList; diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java index 47ed7da44..86970cefb 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/BuiltinFunctionName.java @@ -248,6 +248,7 @@ public enum BuiltinFunctionName { ISNULL(FunctionName.of("isnull")), ISPRESENT(FunctionName.of("ispresent")), + ROW_NUMBER(FunctionName.of("row_number")), RANK(FunctionName.of("rank")), DENSE_RANK(FunctionName.of("dense_rank")), diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java index 028340ec6..08d288874 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystQueryPlanVisitor.java @@ -146,11 +146,7 @@ public LogicalPlan visitFirstChild(Node node, CatalystPlanContext context) { @Override public LogicalPlan visitExplain(Explain node, CatalystPlanContext context) { node.getStatement().accept(this, context); - context.apply(p -> new ExplainCommand(p, ExplainMode.fromString(node.getExplainMode().name()))); - System.out.println(context.getPlan()); - Seq output = context.getPlan().output(); - System.out.println(output); - return context.getPlan(); + return context.apply(p -> new ExplainCommand(p, ExplainMode.fromString(node.getExplainMode().name()))); } @Override From 04af9adbb6be1b605a90616ca7a1fcb82b204c5b Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 11:09:40 -0800 Subject: [PATCH 33/37] ScalaFmt Signed-off-by: Andy Kwok --- .../ppl/FlintSparkPPLAppendColITSuite.scala | 39 +++++++++---------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala index e2a8eb789..5603599b5 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala @@ -7,6 +7,7 @@ package org.opensearch.flint.spark.ppl import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq import org.opensearch.sql.ppl.utils.SortUtils + import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Ascending, CaseWhen, CurrentRow, Descending, Divide, EqualTo, Expression, LessThan, Literal, Multiply, RowFrame, RowNumber, SortOrder, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition} @@ -15,8 +16,6 @@ import org.apache.spark.sql.catalyst.plans.LeftOuter import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.streaming.StreamTest - - class FlintSparkPPLAppendColITSuite extends QueryTest with LogicalPlanTestUtils @@ -41,7 +40,8 @@ class FlintSparkPPLAppendColITSuite private val AGE_ALIAS = Alias(UnresolvedAttribute("age"), "age")() - private val RELATION_TEST_TABLE = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + private val RELATION_TEST_TABLE = UnresolvedRelation( + Seq("spark_catalog", "default", "flint_ppl_test")) private val T12_JOIN_CONDITION = EqualTo(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) @@ -49,7 +49,6 @@ class FlintSparkPPLAppendColITSuite private val T12_COLUMNS_SEQ = Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T2._row_number_")) - override def beforeAll(): Unit = { super.beforeAll() @@ -67,8 +66,8 @@ class FlintSparkPPLAppendColITSuite } /** - * The baseline test-case to make sure APPENDCOL( ) function works, - * when no transformation present on the main search, after the search command. + * The baseline test-case to make sure APPENDCOL( ) function works, when no transformation + * present on the main search, after the search command. */ test("test AppendCol with NO transformation on main") { val frame = sql(s""" @@ -121,17 +120,17 @@ class FlintSparkPPLAppendColITSuite T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - // scalastyle:off - println(logicalPlan) - println(expectedPlan) - // scalastyle:on + // scalastyle:off + println(logicalPlan) + println(expectedPlan) + // scalastyle:on comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } - /** - * To simulate the use-case when user attempt to attach an APPENDCOL command on a well established main search. + * To simulate the use-case when user attempt to attach an APPENDCOL command on a well + * established main search. */ test("test AppendCol with transformation on main-search") { val frame = sql(s""" @@ -159,7 +158,7 @@ class FlintSparkPPLAppendColITSuite specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] : +- 'Project ['name, 'age, 'state] : +- 'UnresolvedRelation [relation], [], false - */ + */ val t1 = SubqueryAlias( "T1", Project( @@ -200,8 +199,7 @@ class FlintSparkPPLAppendColITSuite | source = $testTable | FIELDS name, age, state | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | """.stripMargin) - assert(frame.columns.sameElements( - Array("name", "age", "state", "count()", "age"))) + assert(frame.columns.sameElements(Array("name", "age", "state", "count()", "age"))) // Retrieve the results val results: Array[Row] = frame.collect() val expectedResults: Array[Row] = @@ -222,7 +220,7 @@ class FlintSparkPPLAppendColITSuite specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] : +- 'Project ['age, 'dept, 'salary] : +- 'UnresolvedRelation [relation], [], false - */ + */ val t1 = SubqueryAlias( "T1", Project( @@ -261,16 +259,15 @@ class FlintSparkPPLAppendColITSuite } /** - * The use-case when user attempt to chain multiple APPENCOL command in a PPL, this is a common use case, - * when user prefer to show the statistic report alongside with the dataset. + * The use-case when user attempt to chain multiple APPENCOL command in a PPL, this is a common + * use case, when user prefer to show the statistic report alongside with the dataset. */ test("test multiple AppendCol clauses") { val frame = sql(s""" | source = $testTable | FIELDS name, age | APPENDCOL [ stats count() by age | eval m = 1 | FIELDS -m ] | APPENDCOL [FIELDS state] | """.stripMargin) - assert(frame.columns.sameElements( - Array("name", "age", "count()", "age", "state"))) + assert(frame.columns.sameElements(Array("name", "age", "count()", "age", "state"))) // Retrieve the results val results: Array[Row] = frame.collect() val expectedResults: Array[Row] = @@ -290,7 +287,7 @@ class FlintSparkPPLAppendColITSuite : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#544, *] : +- 'Project ['name, 'age] : +- 'UnresolvedRelation [flint_ppl_test], [], false - */ + */ val mainSearch = SubqueryAlias( "T1", Project( From a58823635b5f04090a667377b24607f7a6c8f238 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 11:18:00 -0800 Subject: [PATCH 34/37] Remove sout Signed-off-by: Andy Kwok --- .../PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala index c5ef8e2da..8338c9e20 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanAppendColCommandTranslatorTestSuite.scala @@ -97,11 +97,6 @@ class PPLLogicalPlanAppendColCommandTranslatorTestSuite T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) -// // scalastyle:off -// println(logicalPlan) -// println(expectedPlan) -// // scalastyle:on - comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } From 8cf6f1a70ba9af583720e327c47fc7007c0ef818 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 12:42:46 -0800 Subject: [PATCH 35/37] Update doc Signed-off-by: Andy Kwok --- docs/ppl-lang/README.md | 2 + docs/ppl-lang/ppl-appendcol-command.md | 79 ++++++++++++++++++++++++++ 2 files changed, 81 insertions(+) create mode 100644 docs/ppl-lang/ppl-appendcol-command.md diff --git a/docs/ppl-lang/README.md b/docs/ppl-lang/README.md index 19e1a6ee0..bddf9a975 100644 --- a/docs/ppl-lang/README.md +++ b/docs/ppl-lang/README.md @@ -74,6 +74,8 @@ For additional examples see the next [documentation](PPL-Example-Commands.md). - [`expand commands`](ppl-expand-command.md) + - [`appendcol commands`](ppl-appendcol-command.md) + * **Functions** - [`Expressions`](functions/ppl-expressions.md) diff --git a/docs/ppl-lang/ppl-appendcol-command.md b/docs/ppl-lang/ppl-appendcol-command.md new file mode 100644 index 000000000..6636c8861 --- /dev/null +++ b/docs/ppl-lang/ppl-appendcol-command.md @@ -0,0 +1,79 @@ +## PPL `appendcol` command + +### Description +Using `appendcol` command append the result of a sub-search and attach it alongside with the input search results (The main search). + +### Syntax - APPENDCOL +`APPENDCOL [sub-search]...` + +* : optional boolean field to specify should result from main-result be overwritten in the case of column name conflict. +* sub-search: The PPL commands to execute for the secondary search. + + +#### Example 1: To append the result of `stats ave(age) as AVG_AGE` into existing search result + +The example append the result of sub-search `stats ave(age) as AVG_AGE` alongside with the main-search. + +PPL query: + + os> source=employees | FIELDS name, dept, age | APPENDCOL [ stats avg(age) as AVG_AGE ]; + fetched rows / total rows = 9/9 + +------+-------------+-----+------------------+ + | name | dept | age | AVG_AGE | + +------+-------------+-----+------------------+ + | Lisa | Sales------ | 35 | 31.2222222222222| + | Fred | Engineering | 28 | NULL | + | Paul | Engineering | 23 | NULL | + | Evan | Sales------ | 38 | NULL | + | Chloe| Engineering | 25 | NULL | + | Tom | Engineering | 33 | NULL | + | Alex | Sales | 33 | NULL | + | Jane | Marketing | 28 | NULL | + | Jeff | Marketing | 38 | NULL | + +------+-------------+-----+------------------+ + + +#### Example 2: Append multiple sub-search result + +The example demonstrate multiple APPENCOL commands can be chained to provide one comprehensive view for user. + +PPL query: + + os> source=employees | FIELDS name, dept, age | APPENDCOL [ stats avg(age) as AVG_AGE ] | APPENDCOL [ stats max(age) as MAX_AGE ]; + fetched rows / total rows = 9/9 + +------+-------------+-----+------------------+---------+ + | name | dept | age | AVG_AGE | MAX_AGE | + +------+-------------+-----+------------------+---------+ + | Lisa | Sales------ | 35 | 31.22222222222222| 38 | + | Fred | Engineering | 28 | NULL | NULL | + | Paul | Engineering | 23 | NULL | NULL | + | Evan | Sales------ | 38 | NULL | NULL | + | Chloe| Engineering | 25 | NULL | NULL | + | Tom | Engineering | 33 | NULL | NULL | + | Alex | Sales | 33 | NULL | NULL | + | Jane | Marketing | 28 | NULL | NULL | + | Jeff | Marketing | 38 | NULL | NULL | + +------+-------------+-----+------------------+---------+ + +#### Example 3: Over main-search in the case of column name conflict + +The example demonstrate the usage of `OVERRIDE` option to overwrite the `age` column from the main-search, +when the option is set to true and column with same name `age` present on sub-search. + +PPL query: + + os> source=employees | FIELDS name, dept, age | APPENDCOL OVERRIDE=true [ stats avg(age) as age ]; + fetched rows / total rows = 9/9 + +------+-------------+------------------+ + | name | dept | age | + +------+-------------+------------------+ + | Lisa | Sales------ | 31.22222222222222| + | Fred | Engineering | NULL | + | Paul | Engineering | NULL | + | Evan | Sales------ | NULL | + | Chloe| Engineering | NULL | + | Tom | Engineering | NULL | + | Alex | Sales | NULL | + | Jane | Marketing | NULL | + | Jeff | Marketing | NULL | + +------+-------------+------------------+ From 6269b5bd33045eea6cc52bdeda62992484863232 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 14:24:04 -0800 Subject: [PATCH 36/37] Override option test case Signed-off-by: Andy Kwok --- docs/ppl-lang/ppl-appendcol-command.md | 2 +- .../ppl/FlintSparkPPLAppendColITSuite.scala | 69 +++++++++++++++++-- 2 files changed, 65 insertions(+), 6 deletions(-) diff --git a/docs/ppl-lang/ppl-appendcol-command.md b/docs/ppl-lang/ppl-appendcol-command.md index 6636c8861..b025fcb1a 100644 --- a/docs/ppl-lang/ppl-appendcol-command.md +++ b/docs/ppl-lang/ppl-appendcol-command.md @@ -1,7 +1,7 @@ ## PPL `appendcol` command ### Description -Using `appendcol` command append the result of a sub-search and attach it alongside with the input search results (The main search). +Using `appendcol` command to append the result of a sub-search and attach it alongside with the input search results (The main search). ### Syntax - APPENDCOL `APPENDCOL [sub-search]...` diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala index 5603599b5..418dbc81c 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala @@ -120,11 +120,6 @@ class FlintSparkPPLAppendColITSuite T12_COLUMNS_SEQ, Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) - // scalastyle:off - println(logicalPlan) - println(expectedPlan) - // scalastyle:on - comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } @@ -347,4 +342,68 @@ class FlintSparkPPLAppendColITSuite comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) } + /** + * To simulate the use-case when column `age` present on both main and sub search, with option OVERRIDE=true. + */ + test("test AppendCol with OVERRIDE option") { + val frame = sql(s""" + | source = $testTable | FIELDS name, age, state | APPENDCOL OVERRIDE=true [stats count() as age] + | """.stripMargin) + + assert(frame.columns.sameElements(Array("name", "state", "age"))) + // Retrieve the results + val results: Array[Row] = frame.collect() + + /* + The sub-search result `APPENDCOL OVERRIDE=true [stats count() as age]` will be attached alongside with first row of main-search, + however given the non-deterministic natural of nature order, we cannot guarantee which specific data row will be returned from the primary search query. + Hence, only assert sub-search position but skipping the table content comparison. + */ + assert(results(0).get(2) == 4) + + // Retrieve the logical plan + val logicalPlan: LogicalPlan = frame.queryExecution.logical + /* + :- 'SubqueryAlias T1 + : +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#11, *] + : +- 'Project ['name, 'age, 'state] + : +- 'UnresolvedRelation [relation], [], false + */ + val t1 = SubqueryAlias( + "T1", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Project( + Seq( + UnresolvedAttribute("name"), + UnresolvedAttribute("age"), + UnresolvedAttribute("state")), + RELATION_TEST_TABLE))) + + /* + +- 'SubqueryAlias T2 + +- 'Project [row_number() windowspecdefinition(1 DESC NULLS LAST, + specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS _row_number_#216, *] + +- 'Aggregate ['COUNT(*) AS age#240] + +- 'UnresolvedRelation [flint_ppl_test], [], false + */ + val t2 = SubqueryAlias( + "T2", + Project( + Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), + Aggregate(Nil, Seq(Alias( + UnresolvedFunction(Seq("COUNT"), Seq(UnresolvedStar(None)), isDistinct = false), + "age")()), RELATION_TEST_TABLE))) + + val overrideFields = Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T1.age")) + + val expectedPlan = Project( + Seq(UnresolvedStar(None)), + DataFrameDropColumns( + T12_COLUMNS_SEQ ++ overrideFields, + Join(t1, t2, LeftOuter, Some(T12_JOIN_CONDITION), JoinHint.NONE))) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + } From 2847e5a9161be72b2f9dc4951136c05a31bee7d2 Mon Sep 17 00:00:00 2001 From: Andy Kwok Date: Mon, 16 Dec 2024 14:50:19 -0800 Subject: [PATCH 37/37] Code style Signed-off-by: Andy Kwok --- .../ppl/FlintSparkPPLAppendColITSuite.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala index 418dbc81c..0cb98c0e9 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLAppendColITSuite.scala @@ -343,7 +343,8 @@ class FlintSparkPPLAppendColITSuite } /** - * To simulate the use-case when column `age` present on both main and sub search, with option OVERRIDE=true. + * To simulate the use-case when column `age` present on both main and sub search, with option + * OVERRIDE=true. */ test("test AppendCol with OVERRIDE option") { val frame = sql(s""" @@ -392,11 +393,16 @@ class FlintSparkPPLAppendColITSuite "T2", Project( Seq(ROW_NUMBER_AGGREGATION, UnresolvedStar(None)), - Aggregate(Nil, Seq(Alias( - UnresolvedFunction(Seq("COUNT"), Seq(UnresolvedStar(None)), isDistinct = false), - "age")()), RELATION_TEST_TABLE))) + Aggregate( + Nil, + Seq( + Alias( + UnresolvedFunction(Seq("COUNT"), Seq(UnresolvedStar(None)), isDistinct = false), + "age")()), + RELATION_TEST_TABLE))) - val overrideFields = Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T1.age")) + val overrideFields = + Seq(UnresolvedAttribute("T1._row_number_"), UnresolvedAttribute("T1.age")) val expectedPlan = Project( Seq(UnresolvedStar(None)),