You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/08/15 17:00:09 UTC

[GitHub] [spark] peter-toth opened a new pull request, #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

peter-toth opened a new pull request, #37525:
URL: https://github.com/apache/spark/pull/37525

   ### What changes were proposed in this pull request?
   Currently `AliasAwareOutputPartitioning` takes only the last alias by aliased expressions into account. We could avoid more shuffles with better alias handling.
   
   ### Why are the changes needed?
   Performance improvement.
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   ### How was this patch tested?
   Added new UT.
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r946468953


##########
sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala:
##########
@@ -681,7 +681,7 @@ class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuit
         df.createTempView("df")
 
         val sqlText = "EXPLAIN CODEGEN SELECT key, MAX(value) FROM df GROUP BY key"
-        val expectedCodegenText = "Found 2 WholeStageCodegen subtrees."
+        val expectedCodegenText = "Found 1 WholeStageCodegen subtrees."

Review Comment:
   Explain codegen output (plan) changed from:
   ```
   Found 2 WholeStageCodegen subtrees.
   == Subtree 1 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:348(0.53% used); numInnerClasses:2) ==
   *(1) HashAggregate(keys=[key#xL], functions=[partial_max(value#xL)], output=[key#xL, max#xL])
   +- *(1) Project [id#xL AS key#xL, id#xL AS value#xL]
      +- *(1) Range (0, 5, step=1, splits=2)
   
   
   == Subtree 2 / 2 (maxMethodCodeSize:252; maxConstantPoolSize:214(0.33% used); numInnerClasses:0) ==
   *(2) HashAggregate(keys=[key#xL], functions=[max(value#xL)], output=[key#xL, max(value)#xL])
   +- AQEShuffleRead coalesced
      +- ShuffleQueryStage 0
         +- Exchange hashpartitioning(key#xL, 5), ENSURE_REQUIREMENTS, [plan_id=55]
            +- *(1) HashAggregate(keys=[key#xL], functions=[partial_max(value#xL)], output=[key#xL, max#xL])
               +- *(1) Project [id#xL AS key#xL, id#xL AS value#xL]
                  +- *(1) Range (0, 5, step=1, splits=2)
   ```
   to:
   ```
   Found 1 WholeStageCodegen subtrees.
   == Subtree 1 / 1 (maxMethodCodeSize:308; maxConstantPoolSize:374(0.57% used); numInnerClasses:2) ==
   *(1) HashAggregate(keys=[key#xL], functions=[max(value#xL)], output=[key#xL, max(value)#xL])
   +- *(1) HashAggregate(keys=[key#xL], functions=[partial_max(value#xL)], output=[key#xL, max#xL])
      +- *(1) Project [id#xL AS key#xL, id#xL AS value#xL]
         +- *(1) Range (0, 5, step=1, splits=2)
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081448596


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))

Review Comment:
   fixed



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080885277


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()

Review Comment:
   Oh, no you are right, the `mutable.ListBuffer[Attribute]` size can be limited.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090102038


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.

Review Comment:
   let's update the comment as well.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090102910


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,55 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait AliasAwareOutputPartitioning extends UnaryExecNode

Review Comment:
   `PartitioningPreservingUnaryExecNode`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090106043


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.children.size == 2)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to children of " +
+    "complex expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id as a", "id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    // (a + b) is the very same as (b + a) so keep only one
+    assert(partitionings.size == 3)

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.children.size == 2)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to children of " +
+    "complex expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id as a", "id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    // (a + b) is the very same as (b + a) so keep only one
+    assert(partitionings.size == 3)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id as a", "id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    // (a + b) is the very same as (b + a) so keep only one
+    assert(outputOrdering.head.children.size == 3)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions and to their children") {
+    val df2 = spark.range(2).repartition($"id" + $"id")
+      .selectExpr("id + id as aa", "id + id as bb", "id as a", "id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 5)

Review Comment:
   ditto



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090105484


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)

Review Comment:
   shall we check the ordering expression as well? We can call `.sql` and check the string.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090440615


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)

Review Comment:
   Ok.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)

Review Comment:
   Done.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091360793


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)

Review Comment:
   +1



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091732809


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   No, why would `c AS a` match on `case a: Attribute`? The foreach runs on a `Seq[Expression]` and not on an `Expression` recursively.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   No, why would `c AS a` match on `case a: Attribute`? This foreach runs on a `Seq[Expression]` and not on an `Expression` recursively.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070665499


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -70,53 +66,16 @@ trait AliasAwareOutputExpression extends SQLConfHelper {
   protected def normalizeExpression(
       expr: Expression,
       pruneFunc: (Expression, AttributeSet) => Option[Expression]): Seq[Expression] = {
-    val normalizedCandidates = new mutable.HashSet[Expression]()
-    normalizedCandidates.add(expr)
     val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
-
-    def pruneCandidate(candidate: Expression): Option[Expression] = {
+    expr.multiTransform {

Review Comment:
   @cloud-fan, @ulysses-you I've updated this PR. Now it is based on `multiTransform` and contains changes from both this PR and https://github.com/apache/spark/pull/39556 (see the description).
   
   `normalizeExpression()` becomes as simple as this with `multiTransform`.
   
   Please note that currently `pruneFunc` is used only for "after transformation filtering", but, as `multiTransform` does the mapping in "one run" (unlike the removed code which runs a `transform` for each alias) so it is much more efficient than the removed version if we have high number of aliases.
   
   Some early pruning would also be possible using `multiTransform`, I will show you that version a bit later.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070753260


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -70,53 +66,16 @@ trait AliasAwareOutputExpression extends SQLConfHelper {
   protected def normalizeExpression(
       expr: Expression,
       pruneFunc: (Expression, AttributeSet) => Option[Expression]): Seq[Expression] = {
-    val normalizedCandidates = new mutable.HashSet[Expression]()
-    normalizedCandidates.add(expr)
     val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
-
-    def pruneCandidate(candidate: Expression): Option[Expression] = {
+    expr.multiTransform {

Review Comment:
   According to my usage. The `multiTransform` should at least support 3 cases of pruning:
   1. the max limit size of returned result
   2. eagerly pruning func
       - prune the result whose references is not subset of output
       - prune intermediate result if the alias map does not contain any other sub-expression
       - prune sub-expression, e.g. `PartitionCollection(a, b)` -> `PartitionCollection(a)` if b is not subset of output
   
   If all this requirements can be matched, I think it's good to switch to multi-transofrm.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] EnricoMi commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070898729


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,15 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of out put expressions whose alias are replaced." +
+        " It can preserve the output partitioning and ordering." +
+        " Negative value means disable this optimization.")

Review Comment:
   ```suggestion
         .doc("The maximum number of candidates for output expressions whose aliases are replaced." +
           " This can preserve the output partitioning and ordering." +
           " Negative value means to disable this optimization.")
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070235022


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   So I think this comment is obsolate and https://github.com/apache/spark/pull/38034/files#r1070212796 explains that we need to call only one `multiTransform`. The rule (map) can define all mappings at once `a + b -> x`, `a -> x1`, `b -> x2`, `c -> x3` and the result will be the expected `Stream(x1 + x2 + x3, x + x3, x + x3)`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080774249


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))
+          case ps => Some(PartitioningCollection(ps))
+        }
+
+      // Filter `SortOrder` children similarly to `PartitioningCollection` elements
+      case s: SortOrder =>

Review Comment:
   Since `SortOrder` is always the root of an expression tree, is it better to handle it here? https://github.com/apache/spark/pull/37525/files#diff-2d06454bd3d4226cab8749376af5298599e0d5a1de175d9ba462608390d7d593R135



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090105747


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)

Review Comment:
   ditto, let's check the partition expressions



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091710769


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   No, sorry, my example was about a `c, c AS a` so both the original attribute `c` and its alias `a` are in the projection. In that case we need to have `c` ->  `Seq(a, c)` alternatives in the map. If we had only `c` ->  `Seq(a)` then then `projectExpression()` (`multiTransform()`) would always transform any child ordering / partitionings with `c` to `a`.
   
   If we had only `c AS a` projection then we would need only `c` ->  `Seq(a)`. 
   If we had only `c` projection then we wouldn't need to add anything to the map because the `rule` partial function argument of `multiTransform()` simply doesn't not apply and carry on the transformation using the original expression (attriribute).
   
   The purpose of this 2nd foreach is to add `c` -> `c` alternatives if needed. 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091527242


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   If we have `c, c AS a` projection then we need to add both the original `c` attribute and `a` to the alternatives of `a`. But we don't need to add an attribute if it isn't aliased.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090439766


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala:
##########
@@ -230,12 +208,14 @@ object V1WritesUtils {
 
   def isOrderingMatched(
       requiredOrdering: Seq[Expression],
-      outputOrdering: Seq[Expression]): Boolean = {
+      outputOrdering: Seq[SortOrder]): Boolean = {
     if (requiredOrdering.length > outputOrdering.length) {
       false
     } else {
       requiredOrdering.zip(outputOrdering).forall {
-        case (requiredOrder, outputOrder) => requiredOrder.semanticEquals(outputOrder)
+        case (requiredOrder, outputOrder) =>
+          // Follow `SortOrder.satisfies` that respects `SortOrder.sameOrderExpressions`

Review Comment:
   Sonuds good, changed.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090571779


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()

Review Comment:
   can we limit the size of this map as well? Now it only limits the size of each map value.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r946471033


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala:
##########
@@ -339,12 +339,12 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite {
       //     ShuffleQueryStage 0
       //   ShuffleQueryStage 2
       //     ReusedQueryStage 0
-      val grouped = df.groupBy("key").agg(max("value").as("value"))
+      val grouped = df.groupBy((col("key") + 1).as("key")).agg(max("value").as("value"))

Review Comment:
   I had to modify the test because the fix modified the the explain plan of the original query from:
   ```
   Union
   :- *(5) HashAggregate(keys=[_groupingexpression#79L], functions=[max(value#38L)], output=[(key + 1)#44L, max(value)#45L])
   :  +- AQEShuffleRead coalesced
   :     +- ShuffleQueryStage 3
   :        +- Exchange hashpartitioning(_groupingexpression#79L, 5), ENSURE_REQUIREMENTS, [plan_id=693]
   :           +- *(3) HashAggregate(keys=[_groupingexpression#79L], functions=[partial_max(value#38L)], output=[_groupingexpression#79L, max#62L])
   :              +- *(3) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#79L])
   :                 +- AQEShuffleRead coalesced
   :                    +- ShuffleQueryStage 0
   :                       +- Exchange hashpartitioning(key#12L, 5), ENSURE_REQUIREMENTS, [plan_id=623]
   :                          +- *(1) HashAggregate(keys=[key#12L], functions=[partial_max(value#13L)], output=[key#12L, max#64L])
   :                             +- *(1) Project [id#10L AS key#12L, id#10L AS value#13L]
   :                                +- *(1) Range (0, 6, step=1, splits=10)
   +- *(6) HashAggregate(keys=[_groupingexpression#80L], functions=[max(value#38L)], output=[(key + 2)#51L, max(value)#52L])
      +- AQEShuffleRead coalesced
         +- ShuffleQueryStage 4
            +- Exchange hashpartitioning(_groupingexpression#80L, 5), ENSURE_REQUIREMENTS, [plan_id=719]
               +- *(4) HashAggregate(keys=[_groupingexpression#80L], functions=[partial_max(value#38L)], output=[_groupingexpression#80L, max#66L])
                  +- *(4) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#80L])
                     +- AQEShuffleRead coalesced
                        +- ShuffleQueryStage 2
                           +- ReusedExchange [key#12L, max#64L], Exchange hashpartitioning(key#12L, 5), ENSURE_REQUIREMENTS, [plan_id=623]
   ```
   to (1 less exchange):
   ```
   Union
   :- *(3) HashAggregate(keys=[_groupingexpression#75L], functions=[max(value#38L)], output=[(key + 1)#44L, max(value)#45L])
   :  +- AQEShuffleRead coalesced
   :     +- ShuffleQueryStage 0
   :        +- Exchange hashpartitioning(_groupingexpression#75L, 5), ENSURE_REQUIREMENTS, [plan_id=514]
   :           +- *(1) HashAggregate(keys=[_groupingexpression#75L], functions=[partial_max(value#38L)], output=[_groupingexpression#75L, max#62L])
   :              +- *(1) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#75L])
   :                 +- *(1) HashAggregate(keys=[key#12L], functions=[partial_max(value#13L)], output=[key#12L, max#64L])
   :                    +- *(1) Project [id#10L AS key#12L, id#10L AS value#13L]
   :                       +- *(1) Range (0, 6, step=1, splits=10)
   +- *(4) HashAggregate(keys=[_groupingexpression#76L], functions=[max(value#38L)], output=[(key + 2)#51L, max(value)#52L])
      +- AQEShuffleRead coalesced
         +- ShuffleQueryStage 1
            +- Exchange hashpartitioning(_groupingexpression#76L, 5), ENSURE_REQUIREMENTS, [plan_id=532]
               +- *(2) HashAggregate(keys=[_groupingexpression#76L], functions=[partial_max(value#38L)], output=[_groupingexpression#76L, max#66L])
                  +- *(2) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#76L])
                     +- *(2) HashAggregate(keys=[key#12L], functions=[partial_max(value#13L)], output=[key#12L, max#64L])
                        +- *(2) Project [id#55L AS key#12L, id#55L AS value#13L]
                           +- *(2) Range (0, 6, step=1, splits=10)
   ```
   and so the query didn't match the `test case 2` description.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080767904


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty

Review Comment:
   shall we check both the maps?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty

Review Comment:
   shall we check both maps?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1397176778

   I've rebased the PR on https://github.com/apache/spark/pull/39652, that is not yet merged, so there is an extra commit (https://github.com/apache/spark/pull/37525/commits/59646bbc26476ec957fd7bff8cbae317791dc228) in this PR that doesn't belong to here, but it will disappear once https://github.com/apache/spark/pull/39652 gets merged


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080887546


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }

Review Comment:
   Yeah, always traversing down should work as we prune the expressions with missing attributes from output set.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080885277


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()

Review Comment:
   Oh, you are right, the `mutable.ListBuffer[Attribute]` size can be limited.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070753260


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -70,53 +66,16 @@ trait AliasAwareOutputExpression extends SQLConfHelper {
   protected def normalizeExpression(
       expr: Expression,
       pruneFunc: (Expression, AttributeSet) => Option[Expression]): Seq[Expression] = {
-    val normalizedCandidates = new mutable.HashSet[Expression]()
-    normalizedCandidates.add(expr)
     val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
-
-    def pruneCandidate(candidate: Expression): Option[Expression] = {
+    expr.multiTransform {

Review Comment:
   According to my usage. The `multiTransform` should at least support 4 cases of pruning:
   - the max limit size of returned result
   - prune the result whose references is not subset of output
   - prune intermediate result if the alias map does not contain any other sub-expression
   - prune sub-expression, e.g. `PartitionCollection(a, b)` -> `PartitionCollection(a)` if b is not subset of output
   
   If all this requirements can be matched, I think it's good to switch to multi-transofrm.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090101865


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,55 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait AliasAwareOutputPartitioning extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
+        case p: PartitioningCollection =>

Review Comment:
   how can we still hit `PartitioningCollection` after `flattenPartitioning`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091351231


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()

Review Comment:
   The current config acutally limit the size of final preserved exprs. I think we should add one more config to limit the candidates size in `aliasMap` and it can be bigger by default. This `aliasMap` may harm driver memory for wide tables.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091527242


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   If we have `c, c AS a` projection then we need to add both the original `c` attribute and `a` to the alternatives of `c`. But we don't need to add an attribute if it isn't aliased.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091535346


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   what's the behavior of `c as a` ? this code seems to return both `c` and `a`. I think the right way should be
   `if AttributSet(outputExpressions).contains(a) => // add a to buffer`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091590456


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   why `c -> c` should be added ? the outputExpression only contains `c as a`, shall we only return `c -> a` ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090675484


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,42 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait PartitioningPreservingUnaryExecNode extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
         case e: Expression =>
-          normalizeExpression(e).asInstanceOf[Partitioning]
-        case other => other
+          // We need unique partitionings but if the input partitioning is
+          // `HashPartitioning(Seq(id + id))` and we have `id -> a` and `id -> b` aliases then after
+          // the projection we have 4 partitionings:
+          // `HashPartitioning(Seq(a + a))`, `HashPartitioning(Seq(a + b))`,
+          // `HashPartitioning(Seq(b + a))`, `HashPartitioning(Seq(b + b))`, but
+          // `HashPartitioning(Seq(a + b))` is the same as `HashPartitioning(Seq(b + a))`.
+          val partitioningSet = mutable.Set.empty[Expression]
+          projectExpression(e)
+            .filter(e => partitioningSet.add(e.canonicalized))
+            .take(aliasCandidateLimit)
+            .asInstanceOf[Stream[Partitioning]]

Review Comment:
   This cast is required to avoid a compile error as `projectExpression` returns `Stream[Expression]` but the `flatMap` requires `Seq[Partitioning]`.
   We could use `.asInstanceOf[Seq[Partitioning]]` here but I'm not sure it makes any difference.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] EnricoMi commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "EnricoMi (via GitHub)" <gi...@apache.org>.
EnricoMi commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090613525


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,16 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of output expressions whose alias are replaced." +
+        " It can preserve the output partitioning and ordering." +
+        " Negative value means disable this optimization.")
+      .internal()
+      .version("3.4.0")

Review Comment:
   This PR targets master, which is `3.5.0`. Is this going to be merged into `branch-3.4`, which is feature-freeze? If not, this line should be adjusted.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,42 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait PartitioningPreservingUnaryExecNode extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
         case e: Expression =>
-          normalizeExpression(e).asInstanceOf[Partitioning]
-        case other => other
+          // We need unique partitionings but if the input partitioning is
+          // `HashPartitioning(Seq(id + id))` and we have `id -> a` and `id -> b` aliases then after
+          // the projection we have 4 partitionings:
+          // `HashPartitioning(Seq(a + a))`, `HashPartitioning(Seq(a + b))`,
+          // `HashPartitioning(Seq(b + a))`, `HashPartitioning(Seq(b + b))`, but
+          // `HashPartitioning(Seq(a + b))` is the same as `HashPartitioning(Seq(b + a))`.
+          val partitioningSet = mutable.Set.empty[Expression]
+          projectExpression(e)
+            .filter(e => partitioningSet.add(e.canonicalized))
+            .take(aliasCandidateLimit)

Review Comment:
   Scala 2.13 allows to simplify this. Its a shame...
   ```suggestion
             projectExpression(e)
               .distinctBy(_.canonicalized)
               .take(aliasCandidateLimit)
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090434872


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,15 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of out put expressions whose alias are replaced." +

Review Comment:
   done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090435947


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala:
##########
@@ -158,7 +158,7 @@ object FileFormatWriter extends Logging {
     // Use the output ordering from the original plan before adding the empty2null projection.

Review Comment:
   removed



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090577009


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala:
##########
@@ -230,12 +208,14 @@ object V1WritesUtils {
 
   def isOrderingMatched(
       requiredOrdering: Seq[Expression],
-      outputOrdering: Seq[Expression]): Boolean = {
+      outputOrdering: Seq[SortOrder]): Boolean = {
     if (requiredOrdering.length > outputOrdering.length) {
       false
     } else {
       requiredOrdering.zip(outputOrdering).forall {
-        case (requiredOrder, outputOrder) => requiredOrder.semanticEquals(outputOrder)
+        case (requiredOrder, outputOrder) =>
+          // Follow `SortOrder.satisfies` that respects `SortOrder.sameOrderExpressions`

Review Comment:
   this comment is not needed, as we are not following `SortOrder.satisfies`, but using it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070211621


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   The basic idea is: Some parts in the input expression tree can be replaced by one or more alternatives, and we want to get all the combinations. `multiTransform` fits it quite well.
   
   However, there are corner cases, when the replacement is not orthogonal. Let's say the output ordering is `a + b + c`, and the project is `a + b as x, a as x1, b as x2, c as x3`. The projected output ordering should be `x1 + x2 + x3`, `x + x3`. IIUC this is hard to do with `multiTransform`, as it only transforms the tree once.
   
   This can be handled well if we do the expansion manually
   ```
   var result = Seq(input)
   for ((expr, alias) <- aliasMap) {
     result ++= result.flatMap(_.transform {
       case e: Expression if e.canonicalized == expr => alias
     })
     // Do some early pruning if the expression can't do alias replacement anymore.
   }
   ```
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070212013


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   Oh, maybe we can do `multiTransform` twice
   ```
   val attrAliasMap = ...
   val exprAliasMap = ...
   input.multiTransform {
     case e if exprAliasMap.contains...
     case e if attrAliasMap.contains...
   } ++ input.multiTransform {
     case e if attrAliasMap.contains...
   }
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080884436


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()

Review Comment:
   I'm not sure we can set a limit because we don't know in advance that which mapping will be used during `outputPartitioning` and `outputOrdering` computation to get `spark.sql.optimizer.expressionProjectionCandidateLimit` number of elements.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081449673


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala:
##########
@@ -1314,6 +1313,135 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper {
     assert(topKs.size == 1)
     assert(sorts.isEmpty)
   }
+
+  test("SPARK-40086: an attribute and its aliased version in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT id, max(value) FROM df GROUP BY id")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id").as("key"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT key, max(value) FROM df GROUP BY key")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute with complex required distribution " +
+    "should not introduce extra shuffle") {
+    withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
+      val df = spark.range(5)
+      val df1 = df.repartition($"id" + $"id")
+        .select($"id".as("key1"), $"id".as("value1"), ($"id" + $"id").as("idPlusId1"))
+      val df2 = df.repartition($"id" + $"id")
+        .select($"id".as("key2"), $"id".as("value2"), ($"id" + $"id").as("idPlusId2"))
+      val df3 = df1.join(df2, $"key1" + $"value1" === $"idPlusId2")
+
+      val planned = df3.queryExecution.executedPlan
+
+      val numShuffles = collect(planned) {
+        case e: ShuffleExchangeExec => e
+      }
+      // before: numShuffles is 4

Review Comment:
   done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080770646


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {

Review Comment:
   I'd call it `projectExpression`, as it not really normalization



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070235022


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   So I think this comment is obsolate and https://github.com/apache/spark/pull/38034/files#r1070212796 explains that we need to call only one `multiTransform`. The rule (map) can define all mappings once `a + b -> x`, `a -> x1`, `b -> x2`, `c -> x3` and the result will be the expected `Seq(x1 + x2 + x3, x + x3, x + x3)`.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   So I think this comment is obsolate and https://github.com/apache/spark/pull/38034/files#r1070212796 explains that we need to call only one `multiTransform`. The rule (map) can define all mappings once `a + b -> x`, `a -> x1`, `b -> x2`, `c -> x3` and the result will be the expected `Stream(x1 + x2 + x3, x + x3, x + x3)`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080769202


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {

Review Comment:
   shall we have better namings? it's not about expression v.s. attribute, but one contains the other. How about `dependentAliasMap` and `exprAliasMap`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080777673


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala:
##########
@@ -1314,6 +1313,135 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper {
     assert(topKs.size == 1)
     assert(sorts.isEmpty)
   }
+
+  test("SPARK-40086: an attribute and its aliased version in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT id, max(value) FROM df GROUP BY id")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id").as("key"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT key, max(value) FROM df GROUP BY key")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute with complex required distribution " +
+    "should not introduce extra shuffle") {
+    withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
+      val df = spark.range(5)
+      val df1 = df.repartition($"id" + $"id")
+        .select($"id".as("key1"), $"id".as("value1"), ($"id" + $"id").as("idPlusId1"))
+      val df2 = df.repartition($"id" + $"id")
+        .select($"id".as("key2"), $"id".as("value2"), ($"id" + $"id").as("idPlusId2"))
+      val df3 = df1.join(df2, $"key1" + $"value1" === $"idPlusId2")
+
+      val planned = df3.queryExecution.executedPlan
+
+      val numShuffles = collect(planned) {
+        case e: ShuffleExchangeExec => e
+      }
+      // before: numShuffles is 4

Review Comment:
   let's put the JIRA ticket id. otherwise people don't know what `before` means



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080775765


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -74,18 +73,4 @@ trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
   }
 }
 
-/**
- * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
- * satisfies ordering requirements.
- */
-trait AliasAwareOutputOrdering extends AliasAwareOutputExpression {
-  protected def orderingExpressions: Seq[SortOrder]
-
-  final override def outputOrdering: Seq[SortOrder] = {
-    if (hasAlias) {
-      orderingExpressions.map(normalizeExpression(_).asInstanceOf[SortOrder])
-    } else {
-      orderingExpressions
-    }
-  }
-}
+trait AliasAwareOutputOrdering extends UnaryExecNode with AliasAwareQueryOutputOrdering[SparkPlan]

Review Comment:
   shall we be consistent and name it `OrderPreservingUnaryExecNode`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081447485


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()

Review Comment:
   fixed in https://github.com/apache/spark/pull/37525/commits/a7955b58cb2cd0fe9325ca241453aca106f55af4



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070970038


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   I think `multiTransform` is very powerful and we should let the caller side have more control. I like the blow more to keep the API simple
   ```
   input.multiTransform {
     case e if exprAliasMap.contains(e) => (e +: exprAliasMap(e)) -> NEVER
     case e if attrAliasMap.contains(e) => attrAliasMap(e) -> NEVER
   }
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091722633


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   I see your point, but this code show me that, it will return both `c -> c` and `c -> a` even the outputExpression is `c as a`. no ?
   
   ```scala
   // outputExpressions is `c as a`
   outputExpressions.foreach {
     // aliases is `c -> a` in first foreach so it's true
     case a: Attribute if aliases.contains(a.canonicalized) =>
       val buffer = aliases(a.canonicalized)
       if (buffer.size < aliasCandidateLimit) {
         // c is added into buffer
         buffer += a
       }
   ```
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091732809


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   No, why would `c AS a` match on `case a: Attribute`? The foreach runs on a `Seq[Expression]` not on an `Expression`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091688659


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   Got it already thanks! I will submit a fix today.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091358506


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   is it correct ? if `orderingExpressions` is `c1, c2` and aliasMap is `c2 as x`, then the outputOrdering should be `c1, x` or empty rather than `x`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091781591


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   No. If `c1` is not aliased otherwise then we don't need to add it to the map. If the map doesn't contain any alias for an expression then the transformation does nothing with that `c1` attribute, just leaves it in the expression tree as it is...



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090103838


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala:
##########
@@ -230,12 +208,14 @@ object V1WritesUtils {
 
   def isOrderingMatched(
       requiredOrdering: Seq[Expression],
-      outputOrdering: Seq[Expression]): Boolean = {
+      outputOrdering: Seq[SortOrder]): Boolean = {
     if (requiredOrdering.length > outputOrdering.length) {
       false
     } else {
       requiredOrdering.zip(outputOrdering).forall {
-        case (requiredOrder, outputOrder) => requiredOrder.semanticEquals(outputOrder)
+        case (requiredOrder, outputOrder) =>
+          // Follow `SortOrder.satisfies` that respects `SortOrder.sameOrderExpressions`

Review Comment:
   shall we write `outputOrder.satisfy(outputOrder.copy(child = requiredOrder))`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #37525: [SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1216301109

   cc @cloud-fan 


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1385458033

   @ulysses-you, @cloud-fan, I've rebased this PR on top of `master`, that now includes `multiTransform()`:
   - The 1st commit of this PR is a cherry-pick of the first commit from https://github.com/apache/spark/pull/39556.
   - The 2nd commit is from the original content of this PR and switches to `multiTransform()`.
   - 3rd is the aliasMap split as distussed here: https://github.com/apache/spark/pull/37525#discussion_r1070970038 and required as `autoComtinue` feature was removed.
   - 4th adds early pruning, but this is a bit different to what we have in https://github.com/apache/spark/pull/39556. It is actually more efficient and we don't need "after transformation pruning".
   - 5th contains test fixes in `PlannerSuite` as some of the previous expected results seems to be wrong. 
   
   Let me know your thougths.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070235022


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   So I think this comment is obsolate and https://github.com/apache/spark/pull/38034/files#r1070212796 explains that we need to call only one `multiTransform`. The rule (map) can define all mappings at once `a + b -> x`, `a -> x1`, `b -> x2`, `c -> x3` and the result will be the expected `Stream(x1 + x2 + x3, x + x3)`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080772242


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))

Review Comment:
   ah, we want to keep the type as `Expression`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1397205575

   > I've rebased the PR on #39652, that is not yet merged, so there is an extra commit ([59646bb](https://github.com/apache/spark/commit/59646bbc26476ec957fd7bff8cbae317791dc228)) in this PR that doesn't belong to here, but it will disappear once #39652 gets merged
   
   https://github.com/apache/spark/pull/39652 got merged, so rebased this PR on latest `master`.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081450369


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala:
##########
@@ -1314,6 +1313,135 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper {
     assert(topKs.size == 1)
     assert(sorts.isEmpty)
   }
+
+  test("SPARK-40086: an attribute and its aliased version in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT id, max(value) FROM df GROUP BY id")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id").as("key"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT key, max(value) FROM df GROUP BY key")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute with complex required distribution " +
+    "should not introduce extra shuffle") {
+    withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
+      val df = spark.range(5)
+      val df1 = df.repartition($"id" + $"id")
+        .select($"id".as("key1"), $"id".as("value1"), ($"id" + $"id").as("idPlusId1"))
+      val df2 = df.repartition($"id" + $"id")
+        .select($"id".as("key2"), $"id".as("value2"), ($"id" + $"id").as("idPlusId2"))
+      val df3 = df1.join(df2, $"key1" + $"value1" === $"idPlusId2")
+
+      val planned = df3.queryExecution.executedPlan
+
+      val numShuffles = collect(planned) {
+        case e: ShuffleExchangeExec => e
+      }
+      // before: numShuffles is 4
+      assert(numShuffles.size == 2)
+      val numOutputPartitioning = collectFirst(planned) {
+        case e: SortMergeJoinExec => e.outputPartitioning match {
+          case PartitioningCollection(Seq(PartitioningCollection(l), PartitioningCollection(r))) =>
+            l ++ r
+          case _ => Seq.empty
+        }
+      }.get
+      assert(numOutputPartitioning.size == 8)
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            // the references of child output partitioning is not the subset of output,
+            // so it has been pruned
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {

Review Comment:
   moved to `ProjectedOrderingAndPartitioningSuite`, let me know if it needs a better name
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080770454


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }

Review Comment:
   this looks very expensive. what if we always traverse down to the children in `multiTransform`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091544557


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   Hmm, I might see the issue here... Let check it today.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   Hmm, I might see the issue here... Let me fix it today.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090627714


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,42 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait PartitioningPreservingUnaryExecNode extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
         case e: Expression =>
-          normalizeExpression(e).asInstanceOf[Partitioning]
-        case other => other
+          // We need unique partitionings but if the input partitioning is
+          // `HashPartitioning(Seq(id + id))` and we have `id -> a` and `id -> b` aliases then after
+          // the projection we have 4 partitionings:
+          // `HashPartitioning(Seq(a + a))`, `HashPartitioning(Seq(a + b))`,
+          // `HashPartitioning(Seq(b + a))`, `HashPartitioning(Seq(b + b))`, but
+          // `HashPartitioning(Seq(a + b))` is the same as `HashPartitioning(Seq(b + a))`.
+          val partitioningSet = mutable.Set.empty[Expression]
+          projectExpression(e)
+            .filter(e => partitioningSet.add(e.canonicalized))
+            .take(aliasCandidateLimit)

Review Comment:
   hmm I think we still need to support scala 2.12 for now?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091353269


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)

Review Comment:
   I prefer `strip(child).canonicalized`. I have not seen other code places that match a canonicalized expression.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1410581682

   Thanks for the review!


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091776030


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)

Review Comment:
   fixed in https://github.com/apache/spark/pull/37525/commits/1f1f093e646d1b45dfabc4aad07e8074fac87dcf



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan closed pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account
URL: https://github.com/apache/spark/pull/37525


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070949153


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   > @peter-toth do you agree that it's better to let the caller decide if the original expr should be included or not?
   
   Well, yes, that would make the code of `multiTransform` much simpler. But the construction of `exprAliasMap` would become the burden of the caller. (I.e. the caller need to identify if the `from` expression in a `from` -> `to` mapping contais any other `from` expression and put the `from -> to` and the `from -> from` mappings to the separate `exprAliasMap` map.)
   
   > It seems currently the framework always include the original expr.
   
   That's right, currently the original expr is always included and the traversal/transformation continues with its children. But if nothing is transformed under the original expression then the original is not included in the final result. Details here: https://github.com/apache/spark/pull/38034#discussion_r1070234296
   
   My initial idea behind of this "automatic traversal/transformation continuation" was that the `rule` is a partial function which can define complex mappings possibly with conditional logic, which a simple map can't do.
   So I wasn't sure that we can expect to caller to write `case` statements that need to return the original expression too if other `case` statements can transform any of the original expressions descendants...
   Although, this is not an issue in our case as we define the `rule` based on simple alias maps.
   
   If we want to keep that "automatic traversal/transformation continuation" logic then we could define the `rule` parameter as `PartialFunction[BaseType, (Stream[BaseType], Continuation)]` (or maybe as `PartialFunction[BaseType, Seq[(BaseType, Continuation)]]`) to let the caller to choose the from 3 different`Continuation` enum elements like `ALWAYS`, `AUTO` or `NEVER`.
   
   So we could either define our multitransform:
   ```
   input.multiTransform {
     case e if aliasMap.contains(e) => exprAliasMap(e) -> AUTO
   }
   ```
   or
   ```
   input.multiTransform {
     case e if exprAliasMap.contains(e) => exprAliasMap(e) -> ALWAYS
     case e if attrAliasMap.contains(e) => attrAliasMap(e) -> NEVER
   }
   ```
   or
   ```
   input.multiTransform {
     case e if exprAliasMap.contains(e) => (e +: exprAliasMap(e)) -> NEVER
     case e if attrAliasMap.contains(e) => attrAliasMap(e) -> NEVER
   }
   ```
   Seems like we need only `AUTO` and `NEVER` so probably we just need a boolean flag instead of the enum...
   
   Anyways, I hope you agree that `multiTransform` is useful and efficient helper function but I'm not sure how general purpose swiss army knife we want to make it.
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r951569244


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala:
##########
@@ -112,28 +113,12 @@ case class BroadcastHashJoinExec(
   // Seq("a", "b", "c"), Seq("a", "b", "y"), Seq("a", "x", "c"), Seq("a", "x", "y").
   // The expanded expressions are returned as PartitioningCollection.
   private def expandOutputPartitioning(partitioning: HashPartitioning): PartitioningCollection = {
-    val maxNumCombinations = conf.broadcastHashJoinOutputPartitioningExpandLimit
-    var currentNumCombinations = 0
-
-    def generateExprCombinations(
-        current: Seq[Expression],
-        accumulated: Seq[Expression]): Seq[Seq[Expression]] = {
-      if (currentNumCombinations >= maxNumCombinations) {
-        Nil
-      } else if (current.isEmpty) {
-        currentNumCombinations += 1
-        Seq(accumulated)
-      } else {
-        val buildKeysOpt = streamedKeyToBuildKeyMapping.get(current.head.canonicalized)
-        generateExprCombinations(current.tail, accumulated :+ current.head) ++
-          buildKeysOpt.map(_.flatMap(b => generateExprCombinations(current.tail, accumulated :+ b)))
-            .getOrElse(Nil)
-      }
-    }
-
-    PartitioningCollection(
-      generateExprCombinations(partitioning.expressions, Nil)
-        .map(HashPartitioning(_, partitioning.numPartitions)))
+    PartitioningCollection(partitioning.multiTransform {

Review Comment:
   I realized that the logic added in https://github.com/apache/spark/pull/28676 can be easily rewritten with the new  `multiTransform()`. But if the rewrite should be independent to this PR then I can revert this part (https://github.com/apache/spark/pull/37525/commits/765feffe2b3d371f57b66ea38bf89a7efe5721b8).



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080772456


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))

Review Comment:
   then why do we have this `case`? `case ps => Some(PartitioningCollection(ps))` should cover it already



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080774659


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))
+          case ps => Some(PartitioningCollection(ps))
+        }
+
+      // Filter `SortOrder` children similarly to `PartitioningCollection` elements
+      case s: SortOrder =>

Review Comment:
   The same to `PartitioningCollection`. It looks cleaner if the caller flattens `PartitioningCollection`, and then uses `multiTransform` to expand it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081446977


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {

Review Comment:
   as disussed, we need only one map 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090102386


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,15 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of out put expressions whose alias are replaced." +
+        " It can preserve the output partitioning and ordering." +
+        " Negative value means disable this optimization.")
+      .version("3.4.0")
+      .intConf

Review Comment:
   let's make it an internal conf. Ideally users shouldn't tune it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090576220


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,42 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait PartitioningPreservingUnaryExecNode extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
         case e: Expression =>
-          normalizeExpression(e).asInstanceOf[Partitioning]
-        case other => other
+          // We need unique partitionings but if the input partitioning is
+          // `HashPartitioning(Seq(id + id))` and we have `id -> a` and `id -> b` aliases then after
+          // the projection we have 4 partitionings:
+          // `HashPartitioning(Seq(a + a))`, `HashPartitioning(Seq(a + b))`,
+          // `HashPartitioning(Seq(b + a))`, `HashPartitioning(Seq(b + b))`, but
+          // `HashPartitioning(Seq(a + b))` is the same as `HashPartitioning(Seq(b + a))`.
+          val partitioningSet = mutable.Set.empty[Expression]
+          projectExpression(e)
+            .filter(e => partitioningSet.add(e.canonicalized))
+            .take(aliasCandidateLimit)
+            .asInstanceOf[Stream[Partitioning]]

Review Comment:
   do we need to cast to `Stream`? It will be pattern matched by `Seq(...)` immediately which will materialize it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091361465


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   Good point! Partitioning is fine as the entire `HashPartitioning` is one expression.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091767957


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   ah, I see it. how about this case: `c1, c2 as x` ? We also should add c1 into aliasMap right ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091795936


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   But the aliasMap is not empty due to `c2 as x`. for this case, how can we preserve c1 if c1 does not add into aliasMap ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091619933


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   If data is sorted by `a, b`, we can't say the data is sorted by `b`, but we can say it's sorted by `a`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090440871


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.children.size == 2)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to children of " +
+    "complex expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id as a", "id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    // (a + b) is the very same as (b + a) so keep only one
+    assert(partitionings.size == 3)

Review Comment:
   Done.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.children.size == 2)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to children of " +
+    "complex expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id as a", "id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    // (a + b) is the very same as (b + a) so keep only one
+    assert(partitionings.size == 3)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id as a", "id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    // (a + b) is the very same as (b + a) so keep only one
+    assert(outputOrdering.head.children.size == 3)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions and to their children") {
+    val df2 = spark.range(2).repartition($"id" + $"id")
+      .selectExpr("id + id as aa", "id + id as bb", "id as a", "id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 5)

Review Comment:
   Done.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r947860136


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,56 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.logical.LeafNode
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number (64) of alternatives to mapped `exp`.

Review Comment:
   sure, done in https://github.com/apache/spark/pull/37525/commits/7266347040cf6af77190c47d91854638aa8a046d, let me know if new conf name doesn't fit



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080888037


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))
+          case ps => Some(PartitioningCollection(ps))
+        }
+
+      // Filter `SortOrder` children similarly to `PartitioningCollection` elements
+      case s: SortOrder =>

Review Comment:
   Ok, I will fix this and other comments today.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080772152


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))

Review Comment:
   shall we just return `p` here?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081448247


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }

Review Comment:
   fixed in https://github.com/apache/spark/pull/37525/commits/a7955b58cb2cd0fe9325ca241453aca106f55af4 we now have 1 map



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {

Review Comment:
   done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091781591


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   No. If `c1` is not aliased otherwise then we don't need to add it to the map. If the map doesn't contain any alias for an expression then the transformation does nothing with that `c1` attribute, just leave it in the expression tree as it is...



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091798658


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   But the aliasMap is not empty due to c2 as x. for this case, how can we preserve c1 if c1 does not add into aliasMap ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091803283


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   oh I see it, nvm. @peter-toth  Thank you for the patience !



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070665499


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -70,53 +66,16 @@ trait AliasAwareOutputExpression extends SQLConfHelper {
   protected def normalizeExpression(
       expr: Expression,
       pruneFunc: (Expression, AttributeSet) => Option[Expression]): Seq[Expression] = {
-    val normalizedCandidates = new mutable.HashSet[Expression]()
-    normalizedCandidates.add(expr)
     val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
-
-    def pruneCandidate(candidate: Expression): Option[Expression] = {
+    expr.multiTransform {

Review Comment:
   @cloud-fan, @ulysses-you I've updated this PR. Now it is based on `multiTransform` and contains changes from both this PR and https://github.com/apache/spark/pull/39556 (see the description).
   
   `normalizeExpression()` becomes as simple as this with `multiTransform`.
   
   Please note that currently `pruneFunc` is used only for "after transformation filtering", but, as `multiTransform` does the mapping in "one run" (unlike the removed code which runs a `transform` for each alias) it is much more efficient than the removed version if we have high number of aliases.
   
   Some early pruning would also be possible using `multiTransform`, I will show you that version a bit later.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r946471033


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala:
##########
@@ -339,12 +339,12 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite {
       //     ShuffleQueryStage 0
       //   ShuffleQueryStage 2
       //     ReusedQueryStage 0
-      val grouped = df.groupBy("key").agg(max("value").as("value"))
+      val grouped = df.groupBy((col("key") + 1).as("key")).agg(max("value").as("value"))

Review Comment:
   I had to modify the test because the fix modified the explain plan of the original query from:
   ```
   Union
   :- *(5) HashAggregate(keys=[_groupingexpression#79L], functions=[max(value#38L)], output=[(key + 1)#44L, max(value)#45L])
   :  +- AQEShuffleRead coalesced
   :     +- ShuffleQueryStage 3
   :        +- Exchange hashpartitioning(_groupingexpression#79L, 5), ENSURE_REQUIREMENTS, [plan_id=693]
   :           +- *(3) HashAggregate(keys=[_groupingexpression#79L], functions=[partial_max(value#38L)], output=[_groupingexpression#79L, max#62L])
   :              +- *(3) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#79L])
   :                 +- AQEShuffleRead coalesced
   :                    +- ShuffleQueryStage 0
   :                       +- Exchange hashpartitioning(key#12L, 5), ENSURE_REQUIREMENTS, [plan_id=623]
   :                          +- *(1) HashAggregate(keys=[key#12L], functions=[partial_max(value#13L)], output=[key#12L, max#64L])
   :                             +- *(1) Project [id#10L AS key#12L, id#10L AS value#13L]
   :                                +- *(1) Range (0, 6, step=1, splits=10)
   +- *(6) HashAggregate(keys=[_groupingexpression#80L], functions=[max(value#38L)], output=[(key + 2)#51L, max(value)#52L])
      +- AQEShuffleRead coalesced
         +- ShuffleQueryStage 4
            +- Exchange hashpartitioning(_groupingexpression#80L, 5), ENSURE_REQUIREMENTS, [plan_id=719]
               +- *(4) HashAggregate(keys=[_groupingexpression#80L], functions=[partial_max(value#38L)], output=[_groupingexpression#80L, max#66L])
                  +- *(4) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#80L])
                     +- AQEShuffleRead coalesced
                        +- ShuffleQueryStage 2
                           +- ReusedExchange [key#12L, max#64L], Exchange hashpartitioning(key#12L, 5), ENSURE_REQUIREMENTS, [plan_id=623]
   ```
   to (1 less exchange):
   ```
   Union
   :- *(3) HashAggregate(keys=[_groupingexpression#75L], functions=[max(value#38L)], output=[(key + 1)#44L, max(value)#45L])
   :  +- AQEShuffleRead coalesced
   :     +- ShuffleQueryStage 0
   :        +- Exchange hashpartitioning(_groupingexpression#75L, 5), ENSURE_REQUIREMENTS, [plan_id=514]
   :           +- *(1) HashAggregate(keys=[_groupingexpression#75L], functions=[partial_max(value#38L)], output=[_groupingexpression#75L, max#62L])
   :              +- *(1) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#75L])
   :                 +- *(1) HashAggregate(keys=[key#12L], functions=[partial_max(value#13L)], output=[key#12L, max#64L])
   :                    +- *(1) Project [id#10L AS key#12L, id#10L AS value#13L]
   :                       +- *(1) Range (0, 6, step=1, splits=10)
   +- *(4) HashAggregate(keys=[_groupingexpression#76L], functions=[max(value#38L)], output=[(key + 2)#51L, max(value)#52L])
      +- AQEShuffleRead coalesced
         +- ShuffleQueryStage 1
            +- Exchange hashpartitioning(_groupingexpression#76L, 5), ENSURE_REQUIREMENTS, [plan_id=532]
               +- *(2) HashAggregate(keys=[_groupingexpression#76L], functions=[partial_max(value#38L)], output=[_groupingexpression#76L, max#66L])
                  +- *(2) HashAggregate(keys=[key#12L], functions=[max(value#13L)], output=[value#38L, _groupingexpression#76L])
                     +- *(2) HashAggregate(keys=[key#12L], functions=[partial_max(value#13L)], output=[key#12L, max#64L])
                        +- *(2) Project [id#55L AS key#12L, id#55L AS value#13L]
                           +- *(2) Range (0, 6, step=1, splits=10)
   ```
   and so the query didn't match the `test case 2` description.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090434607


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090435629


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,15 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of out put expressions whose alias are replaced." +
+        " It can preserve the output partitioning and ordering." +
+        " Negative value means disable this optimization.")
+      .version("3.4.0")
+      .intConf

Review Comment:
   ok, done.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,55 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait AliasAwareOutputPartitioning extends UnaryExecNode

Review Comment:
   Renamed



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090677106


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala:
##########
@@ -230,12 +208,14 @@ object V1WritesUtils {
 
   def isOrderingMatched(
       requiredOrdering: Seq[Expression],
-      outputOrdering: Seq[Expression]): Boolean = {
+      outputOrdering: Seq[SortOrder]): Boolean = {
     if (requiredOrdering.length > outputOrdering.length) {
       false
     } else {
       requiredOrdering.zip(outputOrdering).forall {
-        case (requiredOrder, outputOrder) => requiredOrder.semanticEquals(outputOrder)
+        case (requiredOrder, outputOrder) =>
+          // Follow `SortOrder.satisfies` that respects `SortOrder.sameOrderExpressions`

Review Comment:
   indeed, fixed in https://github.com/apache/spark/pull/37525/commits/733ecb555ca9ed5e520fbff764b5245a1cefbcf1



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] EnricoMi commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "EnricoMi (via GitHub)" <gi...@apache.org>.
EnricoMi commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090631316


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,42 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait PartitioningPreservingUnaryExecNode extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
         case e: Expression =>
-          normalizeExpression(e).asInstanceOf[Partitioning]
-        case other => other
+          // We need unique partitionings but if the input partitioning is
+          // `HashPartitioning(Seq(id + id))` and we have `id -> a` and `id -> b` aliases then after
+          // the projection we have 4 partitionings:
+          // `HashPartitioning(Seq(a + a))`, `HashPartitioning(Seq(a + b))`,
+          // `HashPartitioning(Seq(b + a))`, `HashPartitioning(Seq(b + b))`, but
+          // `HashPartitioning(Seq(a + b))` is the same as `HashPartitioning(Seq(b + a))`.
+          val partitioningSet = mutable.Set.empty[Expression]
+          projectExpression(e)
+            .filter(e => partitioningSet.add(e.canonicalized))
+            .take(aliasCandidateLimit)

Review Comment:
   sure, that's the shame bit



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090626809


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,16 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of output expressions whose alias are replaced." +
+        " It can preserve the output partitioning and ordering." +
+        " Negative value means disable this optimization.")
+      .internal()
+      .version("3.4.0")

Review Comment:
   I'd like to merge it to 3.4 as it fixes a bug in planned write, which is a new feature in 3.4.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #37525: [SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1239098567

   @cloud-fan, @imback82, can you please help to review this PR?


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070800149


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   Yea the key is that `multiTransform` allows you to return the original expr so that it can be expanded again with other mappings.
   ```
   input.multiTransform {
     case e if exprAliasMap.contains(e) => e +: exprAliasMap(e)
     case e if attrAliasMap(e) => attrAliasMap(e)
   }
   ```
   @peter-toth do you agree that it's better to let the caller decide if the original expr should be included or not? It seems currently the framework always include the original expr.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081449349


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -74,18 +73,4 @@ trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
   }
 }
 
-/**
- * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
- * satisfies ordering requirements.
- */
-trait AliasAwareOutputOrdering extends AliasAwareOutputExpression {
-  protected def orderingExpressions: Seq[SortOrder]
-
-  final override def outputOrdering: Seq[SortOrder] = {
-    if (hasAlias) {
-      orderingExpressions.map(normalizeExpression(_).asInstanceOf[SortOrder])
-    } else {
-      orderingExpressions
-    }
-  }
-}
+trait AliasAwareOutputOrdering extends UnaryExecNode with AliasAwareQueryOutputOrdering[SparkPlan]

Review Comment:
   ok, renamed



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala:
##########
@@ -53,6 +53,11 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
   @transient
   lazy val outputSet: AttributeSet = AttributeSet(output)
 
+  /**
+   * Returns the output ordering that this plan generates.

Review Comment:
   done in https://github.com/apache/spark/pull/37525/commits/a7955b58cb2cd0fe9325ca241453aca106f55af4



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080769527


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()

Review Comment:
   shall we also set a limitation on the size of this map? It can be super slow for wide tables/relations.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080776282


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala:
##########
@@ -53,6 +53,11 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
   @transient
   lazy val outputSet: AttributeSet = AttributeSet(output)
 
+  /**
+   * Returns the output ordering that this plan generates.

Review Comment:
   Can we highlight the semantic difference of it when being used in logical plan and physical plan? It physical plan it means ordering in each partition



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080768235


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty

Review Comment:
   nvm, it will never happen that `attrAliasMap` is empty but `exprAliasMap` is not.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090103143


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala:
##########
@@ -158,7 +158,7 @@ object FileFormatWriter extends Logging {
     // Use the output ordering from the original plan before adding the empty2null projection.

Review Comment:
   We can remove this comment now



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090659709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()

Review Comment:
   I'm not sure we should as it might happen that we reach the map size limit but the added aliases doesn't project the child's partitioning / ordering. E.g. if child partitioning is `HashPartitioning(c)` and the projection is `c1 as c1a, ..., c100 as c100a, c as ca` then `c as ca` is not added to the map so we would end up with `UnknownPartitioning`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091775796


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   Here is the fix and test: https://github.com/apache/spark/pull/37525/commits/1f1f093e646d1b45dfabc4aad07e8074fac87dcf



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090659709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()

Review Comment:
   I'm not sure we should as it might happen that we reach the map size limit but the added aliases don't project the child's partitioning / ordering. E.g. if child partitioning is `HashPartitioning(c)` and the projection is `c1 as c1a, ..., c100 as c100a, c as ca` then `c as ca` is not added to the map so we would end up with `UnknownPartitioning`.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] ulysses-you commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "ulysses-you (via GitHub)" <gi...@apache.org>.
ulysses-you commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091351690


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   What does this do ? do you mean  `!aliases.contains(a.canonicalized)` ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091585555


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   This case doesn't match `c AS a`. This case makes sure that if `c` -> `a` has been added to the map by the previous forach then `c` -> `c` should be added too.
   
   I think your code would add all attributes, but that is not needed.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091775796


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   Here is the fix and a new test: https://github.com/apache/spark/pull/37525/commits/1f1f093e646d1b45dfabc4aad07e8074fac87dcf



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091817593


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>

Review Comment:
   Np, thanks for reviewing this PR @ulysses-you!



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090440338


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)

Review Comment:
   Sure, fixed.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")

Review Comment:
   Ok.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091361589


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   let's also add tests for it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r951569244


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala:
##########
@@ -112,28 +113,12 @@ case class BroadcastHashJoinExec(
   // Seq("a", "b", "c"), Seq("a", "b", "y"), Seq("a", "x", "c"), Seq("a", "x", "y").
   // The expanded expressions are returned as PartitioningCollection.
   private def expandOutputPartitioning(partitioning: HashPartitioning): PartitioningCollection = {
-    val maxNumCombinations = conf.broadcastHashJoinOutputPartitioningExpandLimit
-    var currentNumCombinations = 0
-
-    def generateExprCombinations(
-        current: Seq[Expression],
-        accumulated: Seq[Expression]): Seq[Seq[Expression]] = {
-      if (currentNumCombinations >= maxNumCombinations) {
-        Nil
-      } else if (current.isEmpty) {
-        currentNumCombinations += 1
-        Seq(accumulated)
-      } else {
-        val buildKeysOpt = streamedKeyToBuildKeyMapping.get(current.head.canonicalized)
-        generateExprCombinations(current.tail, accumulated :+ current.head) ++
-          buildKeysOpt.map(_.flatMap(b => generateExprCombinations(current.tail, accumulated :+ b)))
-            .getOrElse(Nil)
-      }
-    }
-
-    PartitioningCollection(
-      generateExprCombinations(partitioning.expressions, Nil)
-        .map(HashPartitioning(_, partitioning.numPartitions)))
+    PartitioningCollection(partitioning.multiTransform {

Review Comment:
   I realized that the logic added in https://github.com/apache/spark/pull/28676 can be easily rewritten with the new  `multiTransform()`. But if the rewrite should be independent to this PR then I can revert this part (https://github.com/apache/spark/pull/37525/commits/765feffe2b3d371f57b66ea38bf89a7efe5721b8).



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SQL] Improve AliasAwareOutputPartitioning to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r946757509


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,56 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.logical.LeafNode
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number (64) of alternatives to mapped `exp`.

Review Comment:
   shall we make this configurable?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1080778360


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala:
##########
@@ -1314,6 +1313,135 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper {
     assert(topKs.size == 1)
     assert(sorts.isEmpty)
   }
+
+  test("SPARK-40086: an attribute and its aliased version in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT id, max(value) FROM df GROUP BY id")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute in aggregate expressions should not " +
+    "introduce extra shuffle") {
+    withTempView("df") {
+      spark.range(5).select(col("id").as("key"), col("id").as("value")).createTempView("df")
+      val df = sql("SELECT key, max(value) FROM df GROUP BY key")
+
+      val planned = df.queryExecution.executedPlan
+
+      assert(collect(planned) { case h: HashAggregateExec => h }.nonEmpty)
+
+      val exchanges = collect(planned) { case s: ShuffleExchangeExec => s }
+      assert(exchanges.size == 0)
+    }
+  }
+
+  test("SPARK-40086: multiple aliases to the same attribute with complex required distribution " +
+    "should not introduce extra shuffle") {
+    withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
+      val df = spark.range(5)
+      val df1 = df.repartition($"id" + $"id")
+        .select($"id".as("key1"), $"id".as("value1"), ($"id" + $"id").as("idPlusId1"))
+      val df2 = df.repartition($"id" + $"id")
+        .select($"id".as("key2"), $"id".as("value2"), ($"id" + $"id").as("idPlusId2"))
+      val df3 = df1.join(df2, $"key1" + $"value1" === $"idPlusId2")
+
+      val planned = df3.queryExecution.executedPlan
+
+      val numShuffles = collect(planned) {
+        case e: ShuffleExchangeExec => e
+      }
+      // before: numShuffles is 4
+      assert(numShuffles.size == 2)
+      val numOutputPartitioning = collectFirst(planned) {
+        case e: SortMergeJoinExec => e.outputPartitioning match {
+          case PartitioningCollection(Seq(PartitioningCollection(l), PartitioningCollection(r))) =>
+            l ++ r
+          case _ => Seq.empty
+        }
+      }.get
+      assert(numOutputPartitioning.size == 8)
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            // the references of child output partitioning is not the subset of output,
+            // so it has been pruned
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {

Review Comment:
   some tests here are not really about the planner. Shall we add a test suite for output ordering and partitioning?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1081449075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
+import org.apache.spark.sql.catalyst.trees.MultiTransformHelper
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper with MultiTransformHelper {
+  private val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Split the alias map into 2 maps, the first contains `Expression` -> `Attribute` mappings where
+  // any children of the `Expression` contains any other mapping. This because during
+  // `normalizeExpression()` we will need to handle those maps separately and don't stop generating
+  // alternatives at the `Expression` but we also need to traverse down to its children.
+  private lazy val (exprAliasMap, attrAliasMap) = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior and give precedence
+        // the last Alias during `normalizeExpression()` to avoid any kind of regression.
+        a.toAttribute +=:
+          aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+
+    aliases.partition { case (expr, _) => expr.children.exists(_.exists(aliases.contains)) }
+  }
+
+  protected def hasAlias: Boolean = attrAliasMap.nonEmpty
+
+  /**
+   * Return a set of Expression which normalize the original expression to the aliased.
+   */
+  protected def normalizeExpression(expr: Expression): Seq[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+
+    def f: PartialFunction[Expression, Stream[Expression]] = {
+      // Mapping with aliases
+      case e: Expression if exprAliasMap.contains(e.canonicalized) =>
+        (exprAliasMap(e.canonicalized) :+ e).toStream
+      case e: Expression if attrAliasMap.contains(e.canonicalized) =>
+        attrAliasMap(e.canonicalized).toStream
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Stream.empty
+
+      // Remove `PartitioningCollection` elements that are expressions and contain an attribute that
+      // can't be mapped and the node's output set doesn't contain the attribute.
+      // To achieve this we need to "restart" `multiTransformDown()` for each expression child and
+      // filter out empty streams due to the above attribute pruning case.
+      // The child streams can be then combined using `generateChildrenSeq()` into one stream as
+      // `multiTransformDown()` would also do (but without filtering empty streams).
+      case p: PartitioningCollection =>
+        val childrenStreams = p.partitionings.map {
+          case e: Expression => e.multiTransformDown(f).asInstanceOf[Stream[Partitioning]]
+          case o => Stream(o)
+        }.filter(_.nonEmpty)
+        generateChildrenSeq(childrenStreams).flatMap {
+          case Nil => None
+          // We might have an expression type partitioning that doesn't need
+          // `PartitioningCollection`
+          case (p: Expression) :: Nil => Some(p)
+          case p :: Nil => Some(PartitioningCollection(Seq(p)))
+          case ps => Some(PartitioningCollection(ps))
+        }
+
+      // Filter `SortOrder` children similarly to `PartitioningCollection` elements
+      case s: SortOrder =>

Review Comment:
   ok, fixed in https://github.com/apache/spark/pull/37525/commits/a7955b58cb2cd0fe9325ca241453aca106f55af4



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090105820


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")
+      case _ => fail(s"Unexpected $outputPartitioning")
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " +
+    "expressions") {
+    val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputPartitioning = stripAQEPlan(df2.queryExecution.executedPlan).outputPartitioning
+    val partitionings = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+    assert(partitionings.size == 2)
+
+    val df = spark.range(2).orderBy($"id" + $"id").selectExpr("id + id as a", "id + id as b")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)

Review Comment:
   ditto



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090280063


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,52 +16,55 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
 
-/**
- * A trait that provides functionality to handle aliases in the `outputExpressions`.
- */
-trait AliasAwareOutputExpression extends UnaryExecNode {
-  protected def outputExpressions: Seq[NamedExpression]
-
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
-
-  protected def hasAlias: Boolean = aliasMap.nonEmpty
-
-  protected def normalizeExpression(exp: Expression): Expression = {
-    exp.transformDown {
-      case e: Expression => aliasMap.getOrElse(e.canonicalized, e)
-    }
-  }
-}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.{AliasAwareOutputExpression, AliasAwareQueryOutputOrdering}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
 
 /**
  * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` that
  * satisfies distribution requirements.
  */
-trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression {
+trait AliasAwareOutputPartitioning extends UnaryExecNode
+  with AliasAwareOutputExpression {
   final override def outputPartitioning: Partitioning = {
-    val normalizedOutputPartitioning = if (hasAlias) {
-      child.outputPartitioning match {
+    if (hasAlias) {
+      flattenPartitioning(child.outputPartitioning).flatMap {
+        case p: PartitioningCollection =>

Review Comment:
   You are right. This case is not needed.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091525727


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)

Review Comment:
   Ok, thanks. I will change it today.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091525383


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()

Review Comment:
   I'm ok to add a new config but `aliasMap` will never be bigger than the projection (`outputExpressions`) so is this a real concern?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091617777


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()

Review Comment:
   Let's leave it for now. Having less `Attribute` in map values may generate fewer alternatives, but having less map entries may stop the entire projection.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1091531169


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.plans
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, Empty2Null, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A trait that provides functionality to handle aliases in the `outputExpressions`.
+ */
+trait AliasAwareOutputExpression extends SQLConfHelper {
+  protected val aliasCandidateLimit = conf.getConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT)
+  protected def outputExpressions: Seq[NamedExpression]
+  /**
+   * This method can be used to strip expression which does not affect the result, for example:
+   * strip the expression which is ordering agnostic for output ordering.
+   */
+  protected def strip(expr: Expression): Expression = expr
+
+  // Build an `Expression` -> `Attribute` alias map.
+  // There can be multiple alias defined for the same expressions but it doesn't make sense to store
+  // more than `aliasCandidateLimit` attributes for an expression. In those cases the old logic
+  // handled only the last alias so we need to make sure that we give precedence to that.
+  // If the `outputExpressions` contain simple attributes we need to add those too to the map.
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ArrayBuffer[Attribute]]()
+    outputExpressions.reverse.foreach {
+      case a @ Alias(child, _) =>
+        val buffer = aliases.getOrElseUpdate(strip(child.canonicalized), mutable.ArrayBuffer.empty)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a.toAttribute
+        }
+      case _ =>
+    }
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) =>
+        val buffer = aliases(a.canonicalized)
+        if (buffer.size < aliasCandidateLimit) {
+          buffer += a
+        }
+      case _ =>
+    }
+    aliases
+  }
+
+  protected def hasAlias: Boolean = aliasMap.nonEmpty
+
+  /**
+   * Return a stream of expressions in which the original expression is projected with `aliasMap`.
+   */
+  protected def projectExpression(expr: Expression): Stream[Expression] = {
+    val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
+    expr.multiTransformDown {
+      // Mapping with aliases
+      case e: Expression if aliasMap.contains(e.canonicalized) =>
+        aliasMap(e.canonicalized).toSeq ++ (if (e.containsChild.nonEmpty) Seq(e) else Seq.empty)
+
+      // Prune if we encounter an attribute that we can't map and it is not in output set.
+      // This prune will go up to the closest `multiTransformDown()` call and returns `Stream.empty`
+      // there.
+      case a: Attribute if !outputSet.contains(a) => Seq.empty
+    }
+  }
+}
+
+/**
+ * A trait that handles aliases in the `orderingExpressions` to produce `outputOrdering` that
+ * satisfies ordering requirements.
+ */
+trait AliasAwareQueryOutputOrdering[T <: QueryPlan[T]]
+  extends AliasAwareOutputExpression { self: QueryPlan[T] =>
+  protected def orderingExpressions: Seq[SortOrder]
+
+  override protected def strip(expr: Expression): Expression = expr match {
+    case e: Empty2Null => strip(e.child)
+    case _ => expr
+  }
+
+  override final def outputOrdering: Seq[SortOrder] = {
+    if (hasAlias) {
+      orderingExpressions.flatMap { sortOrder =>
+        val orderingSet = mutable.Set.empty[Expression]
+        val sameOrderings = sortOrder.children.toStream
+          .flatMap(projectExpression)
+          .filter(e => orderingSet.add(e.canonicalized))
+          .take(aliasCandidateLimit)
+        if (sameOrderings.nonEmpty) {
+          Some(sortOrder.copy(child = sameOrderings.head,
+            sameOrderExpressions = sameOrderings.tail))
+        } else {
+          None
+        }

Review Comment:
   I'm not sure I get this. If we have 2 orderings  in `orderingExpressions ` (`SortOrder (c1), SortOrder(c2)`) and projection is `c2 as x` then why we don't expect `SortOrder(x)`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1410531037

   thanks, merging to master/3.4 (as it fixes a bug in planned write)!


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #37525:
URL: https://github.com/apache/spark/pull/37525#issuecomment-1409625294

   @peter-toth can you retrigger the tests? The pyspark failures may be flaky.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090105630


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.catalyst.expressions.{Add, Attribute}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ProjectedOrderingAndPartitioningSuite
+  extends SharedSparkSession with AdaptiveSparkPlanHelper {
+  import testImplicits._
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).orderBy($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+        limit match {
+          case 5 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 2)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 2 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 1 =>
+            assert(outputOrdering.size == 1)
+            assert(outputOrdering.head.sameOrderExpressions.size == 0)
+          case 0 =>
+            assert(outputOrdering.size == 0)
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-alias") {
+    Seq(0, 1, 2, 5).foreach { limit =>
+      withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> limit.toString) {
+        val df = spark.range(2).repartition($"id").selectExpr("id as x", "id as y", "id as z")
+        val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+        limit match {
+          case 5 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 3)
+            assert(p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet == Set("x", "y", "z"))
+          case 2 =>
+            val p = outputPartitioning.asInstanceOf[PartitioningCollection].partitionings
+            assert(p.size == 2)
+            p.flatMap(_.asInstanceOf[HashPartitioning].expressions
+              .map(_.asInstanceOf[Attribute].name)).toSet.subsetOf(Set("x", "y", "z"))
+          case 1 =>
+            val p = outputPartitioning.asInstanceOf[HashPartitioning]
+            assert(p.expressions.size == 1)
+            assert(p.expressions.map(_.asInstanceOf[Attribute].name)
+              .toSet.subsetOf(Set("x", "y", "z")))
+          case 0 =>
+            assert(outputPartitioning.isInstanceOf[UnknownPartitioning])
+        }
+      }
+    }
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - ordering - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .orderBy($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputOrdering = df.queryExecution.optimizedPlan.outputOrdering
+    assert(outputOrdering.size == 1)
+    assert(outputOrdering.head.sameOrderExpressions.size == 0)
+  }
+
+  test("SPARK-42049: Improve AliasAwareOutputExpression - partitioning - multi-references") {
+    val df = spark.range(2).selectExpr("id as a", "id as b")
+      .repartition($"a" + $"b").selectExpr("a as x", "b as y")
+    val outputPartitioning = stripAQEPlan(df.queryExecution.executedPlan).outputPartitioning
+    // (a + b), (a + y), (x + b) are pruned since their references are not the subset of output
+    outputPartitioning match {
+      case HashPartitioning(Seq(Add(l: Attribute, r: Attribute, _)), _) =>
+        assert(l.name == "x" && r.name == "y")

Review Comment:
   ditto, we can check `.sql`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #37525: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1090102096


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -435,6 +435,15 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val EXPRESSION_PROJECTION_CANDIDATE_LIMIT =
+    buildConf("spark.sql.optimizer.expressionProjectionCandidateLimit")
+      .doc("The maximum number of the candidate of out put expressions whose alias are replaced." +

Review Comment:
   ```suggestion
         .doc("The maximum number of the candidate of output expressions whose alias are replaced." +
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070665499


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala:
##########
@@ -70,53 +66,16 @@ trait AliasAwareOutputExpression extends SQLConfHelper {
   protected def normalizeExpression(
       expr: Expression,
       pruneFunc: (Expression, AttributeSet) => Option[Expression]): Seq[Expression] = {
-    val normalizedCandidates = new mutable.HashSet[Expression]()
-    normalizedCandidates.add(expr)
     val outputSet = AttributeSet(outputExpressions.map(_.toAttribute))
-
-    def pruneCandidate(candidate: Expression): Option[Expression] = {
+    expr.multiTransform {

Review Comment:
   @cloud-fan, @ulysses-you I've updated this PR. Now it is based on `multiTransform` and contains changes from both this PR and https://github.com/apache/spark/pull/39556.
   
   `normalizeExpression()` becomes as simple as this with `multiTransform`.
   
   Please note that currently `pruneFunc` is used only for "after transformation filtering", but, as `multiTransform` does the mapping in "one run" (unlike the removed code which runs a `transform` for each alias) it is much more efficient than the removed version if we have high number of aliases.
   
   Some early pruning would also be possible using `multiTransform`, I will show you that version a bit later.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #37525: [WIP][SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

Posted by GitBox <gi...@apache.org>.
peter-toth commented on code in PR #37525:
URL: https://github.com/apache/spark/pull/37525#discussion_r1070949153


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala:
##########
@@ -16,24 +16,53 @@
  */
 package org.apache.spark.sql.execution
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning}
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A trait that provides functionality to handle aliases in the `outputExpressions`.
  */
 trait AliasAwareOutputExpression extends UnaryExecNode {
   protected def outputExpressions: Seq[NamedExpression]
 
-  private lazy val aliasMap = outputExpressions.collect {
-    case a @ Alias(child, _) => child.canonicalized -> a.toAttribute
-  }.toMap
+  private lazy val aliasMap = {
+    val aliases = mutable.Map[Expression, mutable.ListBuffer[Attribute]]()
+    // Add aliases to the map. If multiple alias is defined for a source attribute then add all.
+    outputExpressions.foreach {
+      case a @ Alias(child, _) =>
+        // This prepend is needed to make the first element of the `ListBuffer` point to the last
+        // occurrence of an aliased child. This is to keep the previous behavior:
+        // - when we return `Partitioning`s in `outputPartitioning()`, the first should be same as
+        //   it was previously
+        // - when we return a `SortOrder` in `outputOrdering()`, it should be should be same as
+        //   previously
+        a.toAttribute +=: aliases.getOrElseUpdate(child.canonicalized, mutable.ListBuffer.empty)
+      case _ =>
+    }
+    // Append identity mapping of an attribute to the map if both the attribute and its aliased
+    // version can be found in `outputExpressions`.
+    outputExpressions.foreach {
+      case a: Attribute if aliases.contains(a.canonicalized) => aliases(a.canonicalized) += a
+      case _ =>
+    }
+    aliases
+  }
 
   protected def hasAlias: Boolean = aliasMap.nonEmpty
 
-  protected def normalizeExpression(exp: Expression): Expression = {
+  // This function returns a limited number of alternatives to mapped `exp`.
+  protected def normalizeExpression(exp: Expression): Seq[Expression] = {
+    exp.multiTransform {
+      case e: Expression if aliasMap.contains(e.canonicalized) => aliasMap(e.canonicalized).toSeq

Review Comment:
   > @peter-toth do you agree that it's better to let the caller decide if the original expr should be included or not?
   
   Well, yes, that would make the code of `multiTransform` much simpler. But the construction of `exprAliasMap` would become the burden of the caller. (I.e. the caller need to identify if the `from` expression in a `from` -> `to` mapping contais any other `from` expression and put the `from -> to` and the `from -> from` mappings to the separate `exprAliasMap` map.)
   
   > It seems currently the framework always include the original expr.
   
   That's right, currently the original expr is always included and the traversal/transformation continues with its children. But if nothing is transformed under the original expression then the original is not included in the final result. Details here: https://github.com/apache/spark/pull/38034#discussion_r1070234296
   
   My initial idea behind of this "automatic traversal/transformation continuation" was that the `rule` is a partial function which can define complex mappings possibly with conditional logic, which a simple map can't do.
   So I wasn't sure that we can expect to caller to write `case` statements that need to return the original expression too if other `case` statements can transform any of the original expressions descendants...
   Although, this is not an issue in our case as we define the `rule` based on simple alias maps.
   
   If we want to keep that "automatic traversal/transformation continuation" logic then we could define the `rule` parameter as `PartialFunction[BaseType, (Seq[BaseType], Continuation)]` (or maybe as `PartialFunction[BaseType, Seq[(BaseType, Continuation)]]`) to let the caller to choose the from 3 different`Continuation` enum elements like `ALWAYS`, `AUTO` or `NEVER`.
   
   So we could either define our multitransform:
   ```
   input.multiTransform {
     case e if aliasMap.contains(e) => exprAliasMap(e) -> AUTO
   }
   ```
   or
   ```
   input.multiTransform {
     case e if exprAliasMap.contains(e) => exprAliasMap(e) -> ALWAYS
     case e if attrAliasMap.contains(e) => attrAliasMap(e) -> NEVER
   }
   ```
   or
   ```
   input.multiTransform {
     case e if exprAliasMap.contains(e) => (e +: exprAliasMap(e)) -> NEVER
     case e if attrAliasMap.contains(e) => attrAliasMap(e) -> NEVER
   }
   ```
   Seems like we need only `AUTO` and `NEVER` so probably we just need a boolean flag instead of the enum...
   
   Anyways, I hope you agree that `multiTransform` is useful and efficient helper function but I'm not sure how general purpose swiss army knife we want to make it.
   



-- 
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-unsubscribe@spark.apache.org

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


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