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 2021/02/19 21:44:16 UTC

[GitHub] [spark] mridulm commented on a change in pull request #31480: [SPARK-32384][CORE] repartitionAndSortWithinPartitions avoid shuffle with same partitioner

mridulm commented on a change in pull request #31480:
URL: https://github.com/apache/spark/pull/31480#discussion_r579492908



##########
File path: core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
##########
@@ -73,7 +75,23 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
    * because it can push the sorting down into the shuffle machinery.
    */
   def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = self.withScope {
-    new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering)
+    if (self.partitioner == Some(partitioner)) {
+      self.mapPartitions(iter => {
+        val context = TaskContext.get
+        val sorter = new ExternalSorter[K, V, V](context, None, None, Some(ordering))
+        sorter.insertAll(iter)
+        context.taskMetrics.incMemoryBytesSpilled(sorter.memoryBytesSpilled)
+        context.taskMetrics.incDiskBytesSpilled(sorter.diskBytesSpilled)
+        context.taskMetrics.incPeakExecutionMemory(sorter.peakMemoryUsedBytes)
+        // Use completion callback to stop sorter if task was finished/cancelled.
+        context.addTaskCompletionListener[Unit](_ => sorter.stop)
+        val outputIter = new InterruptibleIterator(context,
+          sorter.iterator.asInstanceOf[Iterator[(K, V)]])
+        CompletionIterator[(K, V), Iterator[(K, V)]](outputIter, sorter.stop)
+      }, preservesPartitioning = true)
+    } else {

Review comment:
       I probably mentioned this in the earlier version of this PR as well.
   Can we unify the code in this if block with what is in BlockStoreShuffleReader as well ?
   We are duplicating this.




----------------------------------------------------------------
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:
users@infra.apache.org



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