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/04/29 06:42:35 UTC

[GitHub] [spark] cloud-fan commented on a change in pull request #32287: [SPARK-27991][CORE] Defer the fetch request on Netty OOM

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



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -261,14 +283,38 @@ final class ShuffleBlockFetcherIterator(
             results.put(new SuccessFetchResult(BlockId(blockId), infoMap(blockId)._2,
               address, infoMap(blockId)._1, buf, remainingBlocks.isEmpty))
             logDebug("remainingBlocks: " + remainingBlocks)
+            enqueueDeferredFetchRequestIfNecessary()
           }
         }
         logTrace(s"Got remote block $blockId after ${Utils.getUsedTimeNs(startTimeNs)}")
       }
 
       override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
         logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
-        results.put(new FailureFetchResult(BlockId(blockId), infoMap(blockId)._2, address, e))
+        val (size, mapIndex) = infoMap(blockId)
+        e match {
+          // Catching OOM and do something based on it is only a workaround for handling the
+          // Netty OOM issue, which is not the best way towards memory management. We can
+          // get rid of it when we find a way to manage Netty's memory precisely.
+
+          // Ensure the Netty memory is at least enough for serving only one block to avoid
+          // the endless retry. And since the Netty memory is shared among shuffle, rpc, etc,
+          // modules, we use "1.5" for the overhead concern.
+          case _: OutOfDirectMemoryError if PlatformDependent.maxDirectMemory() > ( 1.5 * size) =>
+            if (isNettyOOMOnShuffle.compareAndSet(false, true)) {
+              // The fetcher can fail remaining blocks in batch for the same error. So we only
+              // log the warning once to avoid flooding the logs.
+              logWarning(s"Netty OOM happens, will retry the failed blocks")
+            }
+            ShuffleBlockFetcherIterator.this.synchronized {
+              remainingBlocks -= blockId

Review comment:
       shall we add `if (!isZombie) `?




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