tanelk commented on a change in pull request #31595:
URL: https://github.com/apache/spark/pull/31595#discussion_r579705619



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##########
@@ -492,12 +492,43 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
  * Remove no-op operators from the query plan that do not make any 
modifications.
  */
 object RemoveNoopOperators extends Rule[LogicalPlan] {
+  private def removeAliasOnlyProject(plan: LogicalPlan): LogicalPlan = plan 
match {
+    case p @ Project(projectList, child) =>
+      val originalOutputs = projectList.collect {
+        case Alias(a: Attribute, _) => a
+        case a: Attribute => a
+      }
+      if (child.outputSet.nonEmpty && child.outputSet == 
AttributeSet(originalOutputs)) {
+        child
+      } else {
+        p
+      }
+    case _ => plan
+  }
+
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     // Eliminate no-op Projects
     case p @ Project(_, child) if child.sameOutput(p) => child
 
     // Eliminate no-op Window
     case w: Window if w.windowExpressions.isEmpty => w.child
+
+    // Eliminate no-op Union
+    case d @ Distinct(u: Union) =>
+      val unionChildren = u.children.map(c => removeAliasOnlyProject(c))
+      if (unionChildren.tail.forall(unionChildren.head.sameResult(_))) {
+        d.withNewChildren(Seq(u.children.head))
+      } else {
+        d
+      }
+
+    case d @ Deduplicate(keys: Seq[Attribute], u: Union) if AttributeSet(keys) 
== u.outputSet =>

Review comment:
       Is this condition needed? it seems that it would also work if the keys 
is a subset of the outputSet.




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

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