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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5874,53 +5886,72 @@ 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)
-      }
-      withPivot(c, left)
-    }.getOrElse(Option(ctx.unpivotClause()).map { c =>
-      if (ctx.pivotClause() != null) {
-        throw 
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+    // Extract the base child and a list of WithWindowDefinition nodes from 
the plan
+    var baseChild = left
+    var withWindowDefinitions = List.empty[WithWindowDefinition]
+    while (baseChild.isInstanceOf[WithWindowDefinition]) {
+      val wwd = baseChild.asInstanceOf[WithWindowDefinition]
+      withWindowDefinitions = withWindowDefinitions :+ wwd
+      baseChild = wwd.child
+    }
+
+    // Process the base child
+    val newChild = {
+      // 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 = baseChild match {
+        case s: SubqueryAlias =>
+          s
+        case u: UnresolvedRelation =>
+          u
+        case _ =>
+          SubqueryAlias(SubqueryAlias.generateSubqueryName(), baseChild)
       }
-      withUnpivot(c, left)
-    }.getOrElse(Option(ctx.sample).map { c =>
-      withSample(c, left)
-    }.getOrElse(Option(ctx.joinRelation()).map { c =>
-      withJoinRelation(c, left)
-    }.getOrElse(Option(ctx.operator).map { c =>
-      val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
-      visitSetOperationImpl(left, plan(ctx.right), all, c.getType)
-    }.getOrElse(Option(ctx.queryOrganization).map { c =>
-      withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true)
-    }.getOrElse(
-      visitOperatorPipeAggregate(ctx, 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 = baseChild,
+          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)
+        }
+        withPivot(c, baseChild)
+      }.getOrElse(Option(ctx.unpivotClause()).map { c =>
+        if (ctx.pivotClause() != null) {
+          throw 
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+        }
+        withUnpivot(c, baseChild)
+      }.getOrElse(Option(ctx.sample).map { c =>
+        withSample(c, baseChild)
+      }.getOrElse(Option(ctx.joinRelation()).map { c =>
+        withJoinRelation(c, baseChild)
+      }.getOrElse(Option(ctx.operator).map { c =>
+        val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
+        visitSetOperationImpl(baseChild, plan(ctx.right), all, c.getType)
+      }.getOrElse(Option(ctx.queryOrganization).map { c =>
+        withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true)
+      }.getOrElse(
+        visitOperatorPipeAggregate(ctx, left)
+      ))))))))
+    }
+
+    // Reconstruct the WithWindowDefinition nodes on top of the new child

Review Comment:
   @dtenedor  That's very clear—thanks for explaining that. I had a close 
feeling that the `WINDOW` operator representing `WithWindowDefinition` is 
different from others because it is "stateful" and can affect other operators 
(dependent). But I did not realize it violates the design. Now it is very clear 
why we shall remove the WINDOW clause (`WithWindowDefinition`) as a separate 
operator!
   
   No problem! I will update it soon.



-- 
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