You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Jack Hu (JIRA)" <ji...@apache.org> on 2015/04/28 06:11:06 UTC

[jira] [Created] (SPARK-7183) Memory leak with netty shuffle with spark standalone cluster

Jack Hu created SPARK-7183:
------------------------------

             Summary: Memory leak with netty shuffle with spark standalone cluster
                 Key: SPARK-7183
                 URL: https://issues.apache.org/jira/browse/SPARK-7183
             Project: Spark
          Issue Type: Bug
          Components: Shuffle
    Affects Versions: 1.3.0
            Reporter: Jack Hu


There is slow leak in netty shuffle with spark cluster in {{TransportRequestHandler.streamIds}}

In spark cluster, there are some reusable netty connections between two block managers to get/send blocks between worker/drivers. These connections are handled by the {{org.apache.spark.network.server.TransportRequestHandler}} in server side. This handler keep tracking all the streamids negotiate by RPC when shuffle data need transform in these two block managers and the streamid is keeping increasing, and never get a chance to be deleted exception this connection is dropped (seems never happen in normal running).

Here are some detail logs of this  {{TransportRequestHandler}} (Note: we add a log a print the total size of {{TransportRequestHandler.streamIds}}, the log is "Current set size is N of org.apache.spark.network.server.TransportRequestHandler@ADDRESS", this set size is keeping increasing in our test)
{quote}
15/04/22 21:00:16 DEBUG TransportServer: Shuffle server started on port :46288
15/04/22 21:00:16 INFO NettyBlockTransferService: Server created on 46288
15/04/22 21:00:31 INFO TransportRequestHandler: Created TransportRequestHandler org.apache.spark.network.server.TransportRequestHandler@29a4f3e7
15/04/22 21:00:32 TRACE MessageDecoder: Received message RpcRequest: RpcRequest\{requestId=6655045571437304938, message=[B@59778678\}
15/04/22 21:00:32 TRACE NettyBlockRpcServer: Received request: OpenBlocks\{appId=app-20150422210016-0000, execId=<driver>, blockIds=[broadcast_1_piece0]}
15/04/22 21:00:32 TRACE NettyBlockRpcServer: Registered streamId 1387459488000 with 1 buffers
15/04/22 21:00:33 TRACE TransportRequestHandler: Sent result RpcResponse\{requestId=6655045571437304938, response=[B@d2840b\} to client /10.111.7.150:33802
15/04/22 21:00:33 TRACE MessageDecoder: Received message ChunkFetchRequest: ChunkFetchRequest\{streamChunkId=StreamChunkId\{streamId=1387459488000, chunkIndex=0}}
15/04/22 21:00:33 TRACE TransportRequestHandler: Received req from /10.111.7.150:33802 to fetch block StreamChunkId\{streamId=1387459488000, chunkIndex=0\}
15/04/22 21:00:33 INFO TransportRequestHandler: Current set size is 1 of org.apache.spark.network.server.TransportRequestHandler@29a4f3e7
15/04/22 21:00:33 TRACE OneForOneStreamManager: Removing stream id 1387459488000
15/04/22 21:00:33 TRACE TransportRequestHandler: Sent result ChunkFetchSuccess\{streamChunkId=StreamChunkId\{streamId=1387459488000, chunkIndex=0}, buffer=NioManagedBuffer\{buf=java.nio.HeapByteBuffer[pos=0 lim=3839 cap=3839]}} to client /10.111.7.150:33802
15/04/22 21:00:34 TRACE MessageDecoder: Received message RpcRequest: RpcRequest\{requestId=6660601528868866371, message=[B@42bed1b8\}
15/04/22 21:00:34 TRACE NettyBlockRpcServer: Received request: OpenBlocks\{appId=app-20150422210016-0000, execId=<driver>, blockIds=[broadcast_3_piece0]}
15/04/22 21:00:34 TRACE NettyBlockRpcServer: Registered streamId 1387459488001 with 1 buffers
15/04/22 21:00:34 TRACE TransportRequestHandler: Sent result RpcResponse\{requestId=6660601528868866371, response=[B@7fa3fb60\} to client /10.111.7.150:33802
15/04/22 21:00:34 TRACE MessageDecoder: Received message ChunkFetchRequest: ChunkFetchRequest\{streamChunkId=StreamChunkId\{streamId=1387459488001, chunkIndex=0}}
15/04/22 21:00:34 TRACE TransportRequestHandler: Received req from /10.111.7.150:33802 to fetch block StreamChunkId\{streamId=1387459488001, chunkIndex=0\}
15/04/22 21:00:34 INFO TransportRequestHandler: Current set size is 2 of org.apache.spark.network.server.TransportRequestHandler@29a4f3e7
15/04/22 21:00:34 TRACE OneForOneStreamManager: Removing stream id 1387459488001
15/04/22 21:00:34 TRACE TransportRequestHandler: Sent result ChunkFetchSuccess\{streamChunkId=StreamChunkId\{streamId=1387459488001, chunkIndex=0}, buffer=NioManagedBuffer\{buf=java.nio.HeapByteBuffer[pos=0 lim=4277 cap=4277]}} to client /10.111.7.150:33802
15/04/22 21:00:34 TRACE MessageDecoder: Received message RpcRequest: RpcRequest\{requestId=8454597410163901330, message=[B@19c673d1\}
15/04/22 21:00:34 TRACE NettyBlockRpcServer: Received request: OpenBlocks\{appId=app-20150422210016-0000, execId=<driver>, blockIds=[broadcast_2_piece0]}
15/04/22 21:00:34 TRACE NettyBlockRpcServer: Registered streamId 1387459488002 with 1 buffers
15/04/22 21:00:34 TRACE TransportRequestHandler: Sent result RpcResponse\{requestId=8454597410163901330, response=[B@35dbdac2\} to client /10.111.7.150:33802
15/04/22 21:00:34 TRACE MessageDecoder: Received message ChunkFetchRequest: ChunkFetchRequest\{streamChunkId=StreamChunkId\{streamId=1387459488002, chunkIndex=0}}
15/04/22 21:00:34 TRACE TransportRequestHandler: Received req from /10.111.7.150:33802 to fetch block StreamChunkId\{streamId=1387459488002, chunkIndex=0\}
15/04/22 21:00:34 INFO TransportRequestHandler: Current set size is 3 of org.apache.spark.network.server.TransportRequestHandler@29a4f3e7
15/04/22 21:00:34 TRACE OneForOneStreamManager: Removing stream id 1387459488002
......
15/04/22 23:59:50 TRACE MessageDecoder: Received message RpcRequest: RpcRequest\{requestId=5718124278216696100, message=[B@7ade3ea3\}
15/04/22 23:59:50 TRACE NettyBlockRpcServer: Received request: OpenBlocks\{appId=app-20150422210016-0000, execId=<driver>, blockIds=[broadcast_14679_piece0]}
15/04/22 23:59:50 TRACE NettyBlockRpcServer: Registered streamId 1387459501252 with 1 buffers
15/04/22 23:59:50 TRACE TransportRequestHandler: Sent result RpcResponse\{requestId=5718124278216696100, response=[B@40c07a63\} to client /10.111.7.150:33802
15/04/22 23:59:50 TRACE MessageDecoder: Received message ChunkFetchRequest: ChunkFetchRequest\{streamChunkId=StreamChunkId\{streamId=1387459501252, chunkIndex=0}}
15/04/22 23:59:50 TRACE TransportRequestHandler: Received req from /10.111.7.150:33802 to fetch block StreamChunkId\{streamId=1387459501252, chunkIndex=0\}
15/04/22 23:59:50 INFO TransportRequestHandler: Current set size is 13253 of org.apache.spark.network.server.TransportRequestHandler@29a4f3e7
15/04/22 23:59:50 TRACE OneForOneStreamManager: Removing stream id 1387459501252
15/04/22 23:59:50 TRACE TransportRequestHandler: Sent result ChunkFetchSuccess\{streamChunkId=StreamChunkId\{streamId=1387459501252, chunkIndex=0}, buffer=NioManagedBuffer\{buf=java.nio.HeapByteBuffer[pos=0 lim=31474 cap=31474]}} to client /10.111.7.150:33802
15/04/22 23:59:50 TRACE MessageDecoder: Received message RpcRequest: RpcRequest\{requestId=8663805364150028136, message=[B@5974f9b4\}
15/04/22 23:59:50 TRACE NettyBlockRpcServer: Received request: OpenBlocks\{appId=app-20150422210016-0000, execId=<driver>, blockIds=[broadcast_14688_piece0]}
15/04/22 23:59:50 TRACE NettyBlockRpcServer: Registered streamId 1387459501253 with 1 buffers
15/04/22 23:59:50 TRACE TransportRequestHandler: Sent result RpcResponse\{requestId=8663805364150028136, response=[B@122023c6\} to client /10.111.7.150:33802
15/04/22 23:59:50 TRACE MessageDecoder: Received message ChunkFetchRequest: ChunkFetchRequest\{streamChunkId=StreamChunkId\{streamId=1387459501253, chunkIndex=0}}
15/04/22 23:59:50 TRACE TransportRequestHandler: Received req from /10.111.7.150:33802 to fetch block StreamChunkId\{streamId=1387459501253, chunkIndex=0\}
15/04/22 23:59:50 INFO TransportRequestHandler: Current set size is 13254 of org.apache.spark.network.server.TransportRequestHandler@29a4f3e7
15/04/22 23:59:50 TRACE OneForOneStreamManager: Removing stream id 1387459501253
15/04/22 23:59:50 TRACE TransportRequestHandler: Sent result ChunkFetchSuccess\{streamChunkId=StreamChunkId\{streamId=1387459501253, chunkIndex=0}, buffer=NioManagedBuffer\{buf=java.nio.HeapByteBuffer[pos=0 lim=4047 cap=4047]}} to client /10.111.7.150:33802
{quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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