otterc commented on a change in pull request #32287:
URL: https://github.com/apache/spark/pull/32287#discussion_r622721640



##########
File path: 
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -261,14 +282,36 @@ 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))
+        remainingBlocks -= blockId
+        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")
+            }
+            deferredBlocks += blockId

Review comment:
       Sorry @Ngone51, I made a mistake here. The updates to the 
`deferredBlocks` will be done by a single netty event-loop not different netty 
event-loop so moving it inside `ShuffleBlockFetcherIterator.this.synchronized` 
should not be necessary.  
   When `shuffleClient.fetchBlocks(....)`  is called, it creates a channel and 
at that time a netty event loop gets associated with this channel. This is the 
event loop which will always be the one invoking  
`blockFetchingListener.onFailure()` or `blockFetchingListener.onSuccess()` for 
all the blocks in the request. 
   
   In the `onBlockFetchSuccess()` it uses 
`ShuffleBlockFetcherIterator.this.synchronized` but that is because I think it 
is accessing the `var isZombie` which can also be modified by the task thread.
   cc @attilapiros 
   




-- 
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to