cloud-fan commented on a change in pull request #32875:
URL: https://github.com/apache/spark/pull/32875#discussion_r769696976



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -352,3 +350,142 @@ case class BroadcastPartitioning(mode: BroadcastMode) 
extends Partitioning {
     case _ => false
   }
 }
+
+/**
+ * This is used in the scenario where an operator has multiple children (e.g., 
join) and one or more
+ * of which have their own requirement regarding whether its data can be 
considered as
+ * co-partitioned from others. This offers APIs for:
+ *
+ *   1. Comparing with specs from other children of the operator and check if 
they are compatible.
+ *      When two specs are compatible, we can say their data are 
co-partitioned, and Spark will
+ *      potentially able to eliminate shuffle if necessary.
+ *   1. Creating a partitioning that can be used to re-partition another 
child, so that to make it
+ *      having a compatible partitioning as this node.
+ */
+trait ShuffleSpec {
+  /**
+   * Returns the number of partitions of this shuffle spec
+   */
+  def numPartitions: Int
+
+  /**
+   * Returns true iff this spec is compatible with the provided shuffle spec.
+   *
+   * A true return value means that the data partitioning from this spec can 
be seen as
+   * co-partitioned with the `other`, and therefore no shuffle is required 
when joining the two
+   * sides.
+   */
+  def isCompatibleWith(other: ShuffleSpec): Boolean
+
+  /**
+   * Whether this shuffle spec can be used to create partitionings for the 
other children.
+   */
+  def canCreatePartitioning: Boolean = false
+
+  /**
+   * Creates a partitioning that can be used to re-partitioned the other side 
with the given
+   * clustering expressions.
+   *
+   * This will only be called when:
+   *  - [[canCreatePartitioning]] returns true.
+   *  - [[isCompatibleWith]] returns false on the side where the `clustering` 
is from.
+   */
+  def createPartitioning(clustering: Seq[Expression]): Partitioning =
+    throw new UnsupportedOperationException("Operation unsupported for " +
+        s"${getClass.getCanonicalName}")
+}
+
+case object SinglePartitionShuffleSpec extends ShuffleSpec {
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = {
+    other.numPartitions == numPartitions
+  }
+
+  override def canCreatePartitioning: Boolean = true
+
+  override def createPartitioning(clustering: Seq[Expression]): Partitioning =
+    SinglePartition
+
+  override def numPartitions: Int = 1
+}
+
+case class RangeShuffleSpec(
+    numPartitions: Int,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    case SinglePartitionShuffleSpec => numPartitions == 1
+    case ShuffleSpecCollection(specs) => specs.exists(isCompatibleWith)
+    case _ => false
+  }
+}
+
+case class HashShuffleSpec(
+    partitioning: HashPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+  lazy val hashKeyPositions =
+    createHashKeyPositions(distribution.clustering, partitioning.expressions)
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    case SinglePartitionShuffleSpec =>
+      partitioning.numPartitions == 1
+    case otherHashSpec @ HashShuffleSpec(otherPartitioning, otherDistribution) 
=>
+      // we need to check:
+      //  1. both partitioning have the same number of partitions
+      //  2. both partitioning have the same number of expressions
+      //  3. each pair of expression from both has overlapping positions in 
their
+      //     corresponding distributions.
+      partitioning.numPartitions == otherPartitioning.numPartitions &&
+      partitioning.expressions.length == otherPartitioning.expressions.length 
&& {
+        val otherHashKeyPositions = otherHashSpec.hashKeyPositions
+        hashKeyPositions.zip(otherHashKeyPositions).forall { case (left, 
right) =>
+          left.intersect(right).nonEmpty
+        }
+      }
+    case ShuffleSpecCollection(specs) =>
+      specs.exists(isCompatibleWith)
+    case _ =>
+      false
+  }
+
+  override def canCreatePartitioning: Boolean = true
+
+  override def createPartitioning(clustering: Seq[Expression]): Partitioning = 
{
+    val exprs = hashKeyPositions.map(v => clustering(v.head))
+    HashPartitioning(exprs, partitioning.numPartitions)
+  }
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  /**
+   * Returns a sequence where each element is a set of positions of the key in 
`hashKeys` to its
+   * positions in `requiredClusterKeys`. For instance, if 
`requiredClusterKeys` is [a, b, b] and
+   * `hashKeys` is [a, b], the result will be [(0), (1, 2)].
+   */
+  private def createHashKeyPositions(
+      requiredClusterKeys: Seq[Expression],
+      hashKeys: Seq[Expression]): Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    requiredClusterKeys.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, 
mutable.BitSet.empty).add(distKeyPos)
+    }
+
+    hashKeys.map(k => distKeyToPos(k.canonicalized))
+  }
+}
+
+case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec {
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = {
+    specs.exists(_.isCompatibleWith(other))
+  }
+
+  override def canCreatePartitioning: Boolean =
+    specs.forall(_.canCreatePartitioning)
+
+  override def createPartitioning(clustering: Seq[Expression]): Partitioning = 
{
+    // as we only consider # of partitions as the cost now, it doesn't matter 
which one we choose
+    // since they should all have the same # of partitions.
+    specs.head.createPartitioning(clustering)
+  }
+
+  override def numPartitions: Int = specs.head.numPartitions

Review comment:
       I think it's guaranteed, but we can add an assert in this class.




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