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


##########
sql/core/src/test/resources/sql-tests/inputs/group-by.sql:
##########
@@ -45,6 +45,15 @@ SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 
1 AS a, 2 AS b, 3 AS
 SELECT a AS k, COUNT(b) FROM testData GROUP BY k;
 SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1;
 
+-- GROUP BY alias is not triggered if SELECT list has lateral column alias.
+SELECT 1 AS x, x + 1 AS k FROM testData GROUP BY k;

Review Comment:
   Also add test for group by all:
   ```sql
   SELECT 1 AS x, x + 1 AS k FROM testData GROUP BY all;
   ```



##########
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)))
 {
+        a.copy(resolvedGroupExprsNoOuter.map(resolveOuterRef), 
resolvedAggExprsWithOuter, a.child)

Review Comment:
   Can we add extra handling to report user-friendly error message, showing 
that queries of group by all and group by alias are not supported when there is 
LCA in the select clause? For example, check if the grouping expression is 
`all` or alias, throw exception immediately.
   
   Currently I don't have a clear solution how to support the above case with 
the LCA design. So this unsupported situation may last for some time, and we 
better provide clean error messages.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInSort.scala:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.SortOrder
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Sort}
+
+/**
+ * A virtual rule to resolve [[UnresolvedAttribute]] in [[Sort]]. It's only 
used by the real
+ * rule `ResolveReferences`. The column resolution order for [[Sort]] 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 the child plan is Aggregate, resolves the column to 
[[TempResolvedColumn]] with the output
+ *    of Aggregate's child plan. This is to allow Sort to host grouping 
expressions and aggregate
+ *    functions, which can be pushed down to the Aggregate later. For example,
+ *    `SELECT max(a) FROM t GROUP BY b ORDER BY min(a)`.
+ * 4. Resolves the column to [[AttributeReference]] with the output of a 
descendant plan node.
+ *    Spark will propagate the missing attributes from the descendant plan 
node to the Sort node.
+ *    This is to allow users to ORDER BY columns that are not in the SELECT 
clause, which is
+ *    widely supported in other SQL dialects. For example, `SELECT a FROM t 
ORDER BY b`.
+ * 5. If the order by expressions only have one single unresolved column named 
ALL, expanded it to
+ *    include all columns in the SELECT list. This is to support SQL pattern 
like
+ *    `SELECT col1, col2 FROM t ORDER BY ALL`. This should also support 
specifying asc/desc, and
+ *    nulls first/last.
+ * 6. Resolves the column to outer references with the outer plan if we are 
resolving subquery
+ *    expressions.
+ */
+object ResolveReferencesInSort extends SQLConfHelper with 
ColumnResolutionHelper {
+
+  def apply(s: Sort): LogicalPlan = {
+    val resolvedNoOuter = s.order.map(resolveExpressionByPlanOutput(_, 
s.child))
+    val resolvedWithAgg = resolvedNoOuter.map(resolveColWithAgg(_, s.child))
+    val (missingAttrResolved, newChild) = 
resolveExprsAndAddMissingAttrs(resolvedWithAgg, s.child)
+    val orderByAllResolved = resolveOrderByAll(
+      s.global, newChild, missingAttrResolved.map(_.asInstanceOf[SortOrder]))
+    val finalOrdering = orderByAllResolved.map(e => 
resolveOuterRef(e).asInstanceOf[SortOrder])

Review Comment:
   Curious, do we require the child of Sort to be resolved? Or is it implied 
somewhere? Any risk of not doing so (the outer reference thing)? Though i can't 
think of any negative case.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInSort.scala:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.SortOrder
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Sort}
+
+/**
+ * A virtual rule to resolve [[UnresolvedAttribute]] in [[Sort]]. It's only 
used by the real
+ * rule `ResolveReferences`. The column resolution order for [[Sort]] 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 the child plan is Aggregate, resolves the column to 
[[TempResolvedColumn]] with the output
+ *    of Aggregate's child plan. This is to allow Sort to host grouping 
expressions and aggregate
+ *    functions, which can be pushed down to the Aggregate later. For example,
+ *    `SELECT max(a) FROM t GROUP BY b ORDER BY min(a)`.
+ * 4. Resolves the column to [[AttributeReference]] with the output of a 
descendant plan node.
+ *    Spark will propagate the missing attributes from the descendant plan 
node to the Sort node.
+ *    This is to allow users to ORDER BY columns that are not in the SELECT 
clause, which is
+ *    widely supported in other SQL dialects. For example, `SELECT a FROM t 
ORDER BY b`.
+ * 5. If the order by expressions only have one single unresolved column named 
ALL, expanded it to
+ *    include all columns in the SELECT list. This is to support SQL pattern 
like
+ *    `SELECT col1, col2 FROM t ORDER BY ALL`. This should also support 
specifying asc/desc, and
+ *    nulls first/last.
+ * 6. Resolves the column to outer references with the outer plan if we are 
resolving subquery
+ *    expressions.
+ */
+object ResolveReferencesInSort extends SQLConfHelper with 
ColumnResolutionHelper {
+
+  def apply(s: Sort): LogicalPlan = {
+    val resolvedNoOuter = s.order.map(resolveExpressionByPlanOutput(_, 
s.child))
+    val resolvedWithAgg = resolvedNoOuter.map(resolveColWithAgg(_, s.child))
+    val (missingAttrResolved, newChild) = 
resolveExprsAndAddMissingAttrs(resolvedWithAgg, s.child)
+    val orderByAllResolved = resolveOrderByAll(

Review Comment:
   Any tests covering this order: resolveColWithAgg  > resolveOrderByAll ?



##########
sql/core/src/test/resources/sql-tests/inputs/group-by.sql:
##########
@@ -45,6 +45,15 @@ SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 
1 AS a, 2 AS b, 3 AS
 SELECT a AS k, COUNT(b) FROM testData GROUP BY k;
 SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1;
 
+-- GROUP BY alias is not triggered if SELECT list has lateral column alias.
+SELECT 1 AS x, x + 1 AS k FROM testData GROUP BY k;
+
+-- GROUP BY alias is not triggered if SELECT list has outer reference.
+SELECT * FROM testData WHERE a = 1 AND EXISTS (SELECT a AS k GROUP BY k);

Review Comment:
   Actually reading the code I think group by alias is triggered, because the 
whole `resolvedAggExprsWithOuter` are resolved (to outer reference)..



##########
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)) {

Review Comment:
   +1, even use single-layer if, because all conditions are on one variable 
`resolvedAggExprsWithOuter`



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