spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jan Filipiak (Jira)" <j...@apache.org>
Subject [jira] [Comment Edited] (SPARK-30246) Spark on Yarn External Shuffle Service Memory Leak
Date Fri, 20 Dec 2019 09:40:00 GMT

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

Jan Filipiak edited comment on SPARK-30246 at 12/20/19 9:39 AM:
----------------------------------------------------------------

Hi, Feel free to send the PR along later i can double check it.

 

You can clearly see the connection is still hold on to by Netty, that probably indicates that
this connection doesn't time out properly.

Looks like (in your versin 2.4.3)

org.apache.spark.network.yarn.YarnShuffleService.serviceInit(Configuration)#L185

initializes TransportContext like this

TransportContext transportContext = new TransportContext(transportConf, blockHandler);

leaving closeIdleConnections set to false. and hence your idle connection wouldn't get closed

 


was (Author: jfilipiak):
Hi, Feel free to send the PR along later i can double check it.

 

You can clearly see the connection is still hold on to by Netty, that probably indicates that
this connection doesn't time out properly.

Looks like 

org.apache.spark.network.yarn.YarnShuffleService.serviceInit(Configuration)#L185

initializes TransportContext like this

TransportContext transportContext = new TransportContext(transportConf, blockHandler);

leaving closeIdleConnections set to false. and hence your idle connection wouldn't get closed

 

> Spark on Yarn External Shuffle Service Memory Leak
> --------------------------------------------------
>
>                 Key: SPARK-30246
>                 URL: https://issues.apache.org/jira/browse/SPARK-30246
>             Project: Spark
>          Issue Type: Bug
>          Components: Shuffle, Spark Core
>    Affects Versions: 2.4.3
>         Environment: hadoop 2.7.3
> spark 2.4.3
> jdk 1.8.0_60
>            Reporter: huangweiyi
>            Priority: Major
>
> In our large busy yarn cluster which deploy Spark external shuffle service as part of
YARN NM aux service, we encountered OOM in some NMs.
> after i dump the heap memory and found there are some StremState objects still in heap,
but the app which the StreamState belongs to is already finished.
> Here is some relate Figures:
> !https://raw.githubusercontent.com/012huang/public_source/master/SparkPRFigures/nm_oom.png|width=100%!
> The heap dump below shows that the memory consumption mainly consists of two parts:
> *(1) OneForOneStreamManager (4,429,796,424 (77.11%) bytes)*
> *(2) PoolChunk(occupy 1,059,201,712 (18.44%) bytes. )*
> !https://raw.githubusercontent.com/012huang/public_source/master/SparkPRFigures/nm_heap_overview.png|width=100%!
> dig into the OneForOneStreamManager, there are some StreaStates still remained :
> !https://raw.githubusercontent.com/012huang/public_source/master/SparkPRFigures/streamState.png|width=100%!
> incomming references to StreamState::associatedChannel: 
> !https://raw.githubusercontent.com/012huang/public_source/master/SparkPRFigures/associatedChannel_incomming_reference.png|width=100%!



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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


Mime
View raw message