You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "mridulm (via GitHub)" <gi...@apache.org> on 2024/03/01 00:42:14 UTC

Re: [PR] [SPARK-39771][CORE] Add a warning msg in `Dependency` when a too large number of shuffle blocks is to be created. [spark]

mridulm commented on code in PR #45266:
URL: https://github.com/apache/spark/pull/45266#discussion_r1508350378


##########
core/src/main/scala/org/apache/spark/Dependency.scala:
##########
@@ -206,6 +206,21 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag](
     finalizeTask = Option(task)
   }
 
+  // Set the threshold to 1 billion which leads to an 128MB bitmap and
+  // the actual size of `HighlyCompressedMapStatus` can be much larger than 128MB.
+  // This may crash the driver with an OOM error.
+  private val SHUFFLE_BLOCK_NUMBER_WARNING_THRESHOLD: Long = 1L << 30
+  private val numberOfShuffleBlocks = numPartitions.toLong * partitioner.numPartitions.toLong
+
+  if (numberOfShuffleBlocks > SHUFFLE_BLOCK_NUMBER_WARNING_THRESHOLD) {
+    logWarning(
+      s"The number of shuffle blocks (${numberOfShuffleBlocks}) for shuffleId ${shuffleId} " +
+        s"for ${_rdd} with ${numPartitions} partitions is possibly too large, " +
+        "which could cause the driver to crash with an out-of-memory error. " +
+        "Consider decreasing the number of partitions in this shuffle stage."
+    )
+  }
+

Review Comment:
   
   We can avoid the instance variables here.
   Move `SHUFFLE_BLOCK_NUMBER_WARNING_THRESHOLD` to the companion object, and
   
   ```suggestion
     {
       val numberOfShuffleBlocks = numPartitions.toLong * partitioner.numPartitions.toLong
   
       if (numberOfShuffleBlocks > SHUFFLE_BLOCK_NUMBER_WARNING_THRESHOLD) {
         logWarning(
           s"The number of shuffle blocks (${numberOfShuffleBlocks}) for shuffleId ${shuffleId} " +
             s"for ${_rdd} with ${numPartitions} partitions is possibly too large, " +
             "which could cause the driver to crash with an out-of-memory error. " +
             "Consider decreasing the number of partitions in this shuffle stage."
         )
       }
     }
   ```



-- 
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-unsubscribe@spark.apache.org

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