dtenedor commented on code in PR #48649:
URL: https://github.com/apache/spark/pull/48649#discussion_r1819915404


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1037,6 +1037,11 @@ class AstBuilder extends DataTypeAstBuilder
         throw 
QueryParsingErrors.multipleQueryResultClausesWithPipeOperatorsUnsupportedError(
           ctx, clause, PipeOperators.offsetClause)
       }
+      // WINDOW and OFFSET are not supported at the same time
+      if (forPipeOperators && windowClause != null) {

Review Comment:
   optional: should this just be
   
   ```
         if (forPipeOperators) {
           if (clause.nonEmpty) {
             throw QueryParsingErrors...
           } else if (windowClause != null) {
             throw QueryParsingErrors...
           }
         }
   ```
   
   same for L1057 below.



##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,28 @@ table t
 table t
 |> order by x sort by x;
 
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
 table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
 |> select cate, sum(val) over w;
 
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;
+
+-- The WINDOW clause: negative tests.
+-------------------------------------
 -- WINDOW and LIMIT are not supported at the same time.
 table windowTestData
 |> window w as (partition by cate order by val) limit 5;
 
+-- multiple WINDOW clause are not supported.

Review Comment:
   ```suggestion
   -- Multiple consecutive WINDOW clauses are not supported.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5855,51 +5865,79 @@ class AstBuilder extends DataTypeAstBuilder
     if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
       operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
     }
-    // This helper function adds a table subquery boundary between the new 
operator to be added
-    // (such as a filter or sort) and the input plan if one does not already 
exist. This helps the
-    // analyzer behave as if we had added the corresponding SQL clause after a 
table subquery
-    // containing the input plan.
-    def withSubqueryAlias(): LogicalPlan = left match {
-      case s: SubqueryAlias =>
-        s
-      case u: UnresolvedRelation =>
-        u
-      case _ =>
-        SubqueryAlias(SubqueryAlias.generateSubqueryName(), left)
-    }
-    Option(ctx.selectClause).map { c =>
-      withSelectQuerySpecification(
-        ctx = ctx,
-        selectClause = c,
-        lateralView = new java.util.ArrayList[LateralViewContext](),
-        whereClause = null,
-        aggregationClause = null,
-        havingClause = null,
-        windowClause = null,
-        relation = left,
-        isPipeOperatorSelect = true)
-    }.getOrElse(Option(ctx.whereClause).map { c =>
-      withWhereClause(c, withSubqueryAlias())
-    }.getOrElse(Option(ctx.pivotClause()).map { c =>
-      if (ctx.unpivotClause() != null) {
-        throw 
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+    // Helper function to extract window definitions from the plan
+    def extractWithWindowDefinitions(

Review Comment:
   this is only called once on L5883 below. Should we just inline the 
implementation instead?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5855,51 +5865,79 @@ class AstBuilder extends DataTypeAstBuilder
     if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
       operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
     }
-    // This helper function adds a table subquery boundary between the new 
operator to be added
-    // (such as a filter or sort) and the input plan if one does not already 
exist. This helps the
-    // analyzer behave as if we had added the corresponding SQL clause after a 
table subquery
-    // containing the input plan.
-    def withSubqueryAlias(): LogicalPlan = left match {
-      case s: SubqueryAlias =>
-        s
-      case u: UnresolvedRelation =>
-        u
-      case _ =>
-        SubqueryAlias(SubqueryAlias.generateSubqueryName(), left)
-    }
-    Option(ctx.selectClause).map { c =>
-      withSelectQuerySpecification(
-        ctx = ctx,
-        selectClause = c,
-        lateralView = new java.util.ArrayList[LateralViewContext](),
-        whereClause = null,
-        aggregationClause = null,
-        havingClause = null,
-        windowClause = null,
-        relation = left,
-        isPipeOperatorSelect = true)
-    }.getOrElse(Option(ctx.whereClause).map { c =>
-      withWhereClause(c, withSubqueryAlias())
-    }.getOrElse(Option(ctx.pivotClause()).map { c =>
-      if (ctx.unpivotClause() != null) {
-        throw 
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+    // Helper function to extract window definitions from the plan
+    def extractWithWindowDefinitions(
+        plan: LogicalPlan): (LogicalPlan, List[WithWindowDefinition]) = {
+      var currentPlan = plan
+      var withWindowDefinitions = List.empty[WithWindowDefinition]
+      while (currentPlan.isInstanceOf[WithWindowDefinition]) {
+        val wwd = currentPlan.asInstanceOf[WithWindowDefinition]
+        withWindowDefinitions = withWindowDefinitions :+ wwd
+        currentPlan = wwd.child
       }
-      withPivot(c, left)
-    }.getOrElse(Option(ctx.unpivotClause()).map { c =>
-      if (ctx.pivotClause() != null) {
-        throw 
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+      (currentPlan, withWindowDefinitions)
+    }
+
+    // Extract the base child and the list of WithWindowDefinition nodes
+    val (baseChild, withWindowDefinitions) = extractWithWindowDefinitions(left)
+
+    // Helper function to process the right side of the operator pipe
+    def processRightSide(relation: LogicalPlan): LogicalPlan = {

Review Comment:
   same, this is only called once on L5936. We can just assign it to a `val` 
instad.



##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,28 @@ table t
 table t
 |> order by x sort by x;
 
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
 table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
 |> select cate, sum(val) over w;
 
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;

Review Comment:
   Other test ideas:
   
   * `|> window ...` with RANGE BETWEEN as part of the window definition
   * what happens if we add a `|> window` operator, but then we do not refer to 
it in the following `|> select` operator (using `over`)?
   * `|> window ...` with other operators after it, e.g. `|> where` or `|> 
order by`
   * `|> window ...` creating window names that shadow other valid column 
names. Is this allowed?
   * `|> window ...` using table aliases to refer to columns, like L193-207 
above
   * `|> window ...` with nothing after it at all
   * `|> window ...` and OFFSET at the same time
   * `|> window ...` and ORDER BY at the same time, but outside the parentheses 
(e.g. `|> window w1 as (partition by val) order by cate`)
   * `|> select ...` with inline windows not defined in a preceding `|> window 
...` operator



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to