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

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


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

Reply via email to