gengliangwang commented on code in PR #39508:
URL: https://github.com/apache/spark/pull/39508#discussion_r1080920518


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInAggregate.scala:
##########
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
Expression, GetStructField, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, AppendColumns, 
LogicalPlan}
+import 
org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE,
 UNRESOLVED_ATTRIBUTE}
+
+/**
+ * A virtual rule to resolve [[UnresolvedAttribute]] in [[Aggregate]]. It's 
only used by the real
+ * rule `ResolveReferences`. The column resolution order for [[Aggregate]] is:
+ * 1. Resolves the column to [[AttributeReference]] with the output of the 
child plan. This
+ *    includes metadata columns as well.
+ * 2. Resolves the column to a literal function which is allowed to be invoked 
without braces, e.g.
+ *    `SELECT col, current_date FROM t`.
+ * 3. If `Aggregate.aggregateExpressions` are all resolved, resolve GROUP BY 
alias and GROUP BY ALL
+ *    for `Aggregate.groupingExpressions`:
+ * 3.1. If the grouping expressions contain an unresolved column whose name 
matches an alias in the
+ *      SELECT list, resolves that unresolved column to the alias. This is to 
support SQL pattern
+ *      like `SELECT a + b AS c, max(col) FROM t GROUP BY c`.
+ * 3.2. If the grouping expressions only have one single unresolved column 
named 'ALL', expanded it
+ *      to include all non-aggregate columns in the SELECT list. This is to 
support SQL pattern like
+ *      `SELECT col1, col2, agg_expr(...) FROM t GROUP BY ALL`.
+ * 4. Resolves the column in `Aggregate.aggregateExpressions` to 
[[LateralColumnAliasReference]] if
+ *    it references the alias defined previously in the SELECT list. The rule
+ *    `ResolveLateralColumnAliasReference` will further resolve 
[[LateralColumnAliasReference]] and
+ *    rewrite the plan. This is to support SQL pattern like
+ *    `SELECT col1 + 1 AS x, x + 1 AS y, y + 1 AS z FROM t`.
+ * 5. Resolves the column to outer references with the outer plan if we are 
resolving subquery
+ *    expressions.
+ */
+object ResolveReferencesInAggregate extends SQLConfHelper with 
ColumnResolutionHelper {
+  def apply(a: Aggregate): Aggregate = {
+    val planForResolve = a.child match {
+      // SPARK-25942: Resolves aggregate expressions with `AppendColumns`'s 
children, instead of
+      // `AppendColumns`, because `AppendColumns`'s serializer might produce 
conflict attribute
+      // names leading to ambiguous references exception.
+      case appendColumns: AppendColumns => appendColumns
+      case _ => a
+    }
+
+    val resolvedGroupExprsNoOuter = a.groupingExpressions
+      .map(resolveExpressionByPlanChildren(_, planForResolve, allowOuter = 
false))
+      // SPARK-31670: Resolve Struct field in groupByExpressions and 
aggregateExpressions
+      // with CUBE/ROLLUP will be wrapped with alias like 
Alias(GetStructField, name) with
+      // different ExprId. This cause aggregateExpressions can't be replaced 
by expanded
+      // groupByExpressions in 
`ResolveGroupingAnalytics.constructAggregateExprs()`, we trim
+      // unnecessary alias of GetStructField here.
+      .map(trimTopLevelGetStructFieldAlias)
+    val resolvedAggExprsNoOuter = a.aggregateExpressions.map(
+      resolveExpressionByPlanChildren(_, planForResolve, allowOuter = false))
+    val resolvedAggExprsWithLCA = 
resolveLateralColumnAlias(resolvedAggExprsNoOuter)
+    val resolvedAggExprsWithOuter = 
resolvedAggExprsWithLCA.map(resolveOuterRef)
+      .map(_.asInstanceOf[NamedExpression])
+    // `groupingExpressions` may rely on `aggregateExpressions`, due to 
features like GROUP BY alias
+    // and GROUP BY ALL. We only do basic resolution for 
`groupingExpressions`, and will further
+    // resolve it after `aggregateExpressions` are all resolved. Note: the 
basic resolution is
+    // needed as `aggregateExpressions` may rely on `groupingExpressions` as 
well, for the session
+    // window feature. See the rule `SessionWindowing` for more details.
+    if (resolvedAggExprsWithOuter.forall(_.resolved)) {
+      // TODO: currently we don't support LCA in `groupingExpressions` yet.
+      if 
(resolvedAggExprsWithOuter.exists(_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)))
 {

Review Comment:
   Oh ok there are comments at the beginning



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to