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 2020/03/06 22:52:31 UTC

[GitHub] [spark] maryannxue commented on a change in pull request #27833: [SPARK-31070][SQL] make skew join split skewed partitions more evenly

maryannxue commented on a change in pull request #27833: [SPARK-31070][SQL] make skew join split skewed partitions more evenly
URL: https://github.com/apache/spark/pull/27833#discussion_r389182488
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala
 ##########
 @@ -115,18 +115,32 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] {
     partitionStartIndices += 0
     var i = 0
     var postMapPartitionSize = 0L
+    var lastPackagedPartitionSize = -1L
     while (i < mapPartitionSizes.length) {
       val nextMapPartitionSize = mapPartitionSizes(i)
       if (i > 0 && postMapPartitionSize + nextMapPartitionSize > targetSize) {
         partitionStartIndices += i
+        lastPackagedPartitionSize = postMapPartitionSize
         postMapPartitionSize = nextMapPartitionSize
       } else {
         postMapPartitionSize += nextMapPartitionSize
       }
       i += 1
     }
 
-    partitionStartIndices.toArray
+    if (lastPackagedPartitionSize > -1) {
+      val lastPartitionDiff = math.abs(targetSize - postMapPartitionSize)
+      val diffIfMergeLastPartition = math.abs(
+        lastPackagedPartitionSize + postMapPartitionSize - targetSize)
+      // If the last partition is very small, we should merge it to the previous partition.
+      if (lastPartitionDiff > diffIfMergeLastPartition * 2) {
 
 Review comment:
   I get your point here, but using target size and this formula doesn't quite make sense sometimes, e.g.,
   ```
   targetSize = 7
   lastButOneSize = 4
   lastSize = 4
   ```
   You'd get:
   ```
   diffLastPartition = 3
   diffLastTwoPartitions = 1
   ```
   This would satisfy your "very small" condition, yet I'd argue that first of all the last partition is well over half the target size, so I wouldn't consider it too small; second, what if previous partitions are mostly around size of 4 too? It would more even not to merge them, right?
   
   A simple way is probably get the average size of all partitions except the last one and merge if `(avgSize - lastSize) > (lastSize + lastButOneSize - avgSize)`. 

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


With regards,
Apache Git Services

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