[ 
https://issues.apache.org/jira/browse/SPARK-26418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16725649#comment-16725649
 ] 

Wang Shuo commented on SPARK-26418:
-----------------------------------

We use YarnShuffleService as aux service of NodeManager in our cluster. Full GC 
happens in some NodeManagers. We dump the heap memory, found that the Map in 
OneForOneStreamManager is 3G, almost 80% of heap size.  Some applications have 
finished, but the StreamState is still in StreamManager.

 

Two reasons will cause this:
 # OpenBlocks request is receivedand StreamState is initialized in server side. 
Then Client lost and no ChunkFetch request is sent to server for the stream.
 # OpenBlocks request is received and StreamState is initialized in server 
side. ChunkFetch request for the stream is sent to server. But server is under 
heavy pressure and not able to handle the ChunkFetch request before timeout. 
Then client close connection.

 

Could we call StreamManager#registerChannel to associate the channel with 
StreamState when server handles OpenBlocks request?

> Only OpenBlocks without any ChunkFetch for one stream will cause memory leak 
> in ExternalShuffleService
> ------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-26418
>                 URL: https://issues.apache.org/jira/browse/SPARK-26418
>             Project: Spark
>          Issue Type: Bug
>          Components: Shuffle
>    Affects Versions: 2.4.0
>            Reporter: Wang Shuo
>            Priority: Major
>
> In current code path,  OneForOneStreamManager holds StreamState in a Map 
> named streams. 
> A StreamState is initialized and put into streams when OpenBlocks request 
> received.
> One specific StreamState is removed from streams in two scenarios below:
>  # The last chunk of a stream is fetched
>  # The connection of ChunkFetch is closed
> StreamState will never be clean up, if OpenBlocks request is received without 
> and following  ChunkFetch request. This will cause memory leak in server 
> side, which is harmful for long running service such as 
> ExternalShuffleService.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to