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 2019/02/01 02:44:21 UTC

[GitHub] cloud-fan commented on a change in pull request #22957: [SPARK-25951][SQL] Ignore aliases for distributions and orderings

cloud-fan commented on a change in pull request #22957: [SPARK-25951][SQL] Ignore aliases for distributions and orderings
URL: https://github.com/apache/spark/pull/22957#discussion_r252917970
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 ##########
 @@ -186,6 +187,69 @@ trait Partitioning {
   }
 }
 
+object Partitioning {
+  /**
+   * Gets as input the `Partitioning` of an expressions and returns the valid `Partitioning`s for
+   * the node w.r.t its output and its expressions.
+   */
+  def updatePartitioningWithNewOutput(
+      inputPartitioning: Partitioning,
+      expressions: Seq[NamedExpression],
+      outputSet: AttributeSet): Partitioning = {
+    inputPartitioning match {
+      case partitioning: Expression =>
+        val exprToEquiv = partitioning.references.map { attr =>
+          attr -> expressions.filter(e =>
+            CleanupAliases.trimAliases(e).semanticEquals(attr))
+        }.filterNot { case (attr, exprs) =>
+          exprs.size == 1 && exprs.forall(_ == attr)
+        }
+        val initValue = partitioning match {
+          case PartitioningCollection(partitionings) => partitionings
+          case other => Seq(other)
+        }
+        val validPartitionings = exprToEquiv.foldLeft(initValue) {
+          case (partitionings, (toReplace, equivalents)) =>
+            if (equivalents.isEmpty) {
+              partitionings.map {
+                case hp: HashPartitioning if hp.references.contains(toReplace) =>
+                  UnknownPartitioning(hp.numPartitions)
+                case rp: RangePartitioning if rp.references.contains(toReplace) =>
 
 Review comment:
   since we need to handle different partitioning differently, shall we add a method to `Partitioning`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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