-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26418][SHUFFLE] Only OpenBlocks without any ChunkFetch for one stream will cause memory leak in ExternalShuffleService #23355
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
If ExternalShuffleService only received a OpenBlocks message, not received any ChunkFetchRequest message, associate StreamState is hold by streams. |
|
Let me explain my problem in detail. We use In current code, server creates I think two reasons will cause this:
Currently the |
vanzin
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This seems fine given the client code will always use the same client.
You could clean up application state in ExternalShuffleBlockHandler.applicationRemoved, but this is simpler.
| // Connection closed before any FetchChunk request received | ||
| streamManager.connectionTerminated(reverseClient.getChannel()); | ||
|
|
||
| assert streamManager.getStreamsSize() == 0; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use jUnit asserts.
| import org.apache.spark.network.shuffle.protocol.OpenBlocks; | ||
| import org.junit.Test; | ||
|
|
||
| import java.nio.ByteBuffer; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Java imports go first.
| package org.apache.spark.network.shuffle; | ||
|
|
||
| import io.netty.channel.Channel; | ||
| import org.apache.spark.network.buffer.ManagedBuffer; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Spark imports go into their own group. See existing code.
| } | ||
|
|
||
| @VisibleForTesting | ||
| public long getStreamsSize() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
getStreamCount
|
Also, please explain the fix, not the problem, in the PR. |
|
ok to test |
|
Test build #100344 has finished for PR 23355 at commit
|
|
I check |
|
@vanzin
I will add some explanation about the fix in the PR description. Thank you. |
|
Test build #100355 has finished for PR 23355 at commit
|
|
@vanzin |
beliefer
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This change is so clearly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree with @vanzin that change's fine since OpenBlocks and fetchChunk would always use the same client.
A minor concern is:
Since we always register the channel when we receive the FetchChunkRequest in the past, so the associated channel with the stream will aways be correct. And for this change, if user call sendRpc(OpenBlocks) and fetchChunk with different client, then, we may record the wrong channel(which is the OpenBlocks chanel, rather than fetchChunk channel) with the stream which is being fetched, which may fall into the same issue as this pr fixed. So, I think we should leave some comment above fetchChunk to notify user that this method must use the same client with the OpenBlocks, otherwise, there may be potential memory leak.
| ManagedBuffer buf; | ||
| try { | ||
| streamManager.checkAuthorization(client, msg.streamChunkId.streamId); | ||
| streamManager.registerChannel(channel, msg.streamChunkId.streamId); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think if you remove this, you should add it back to NettyBlockRpcServer for non external shuffle service. Otherwise, you'll introduce the bug you fixed with pr to the non external shuffle service.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch! Added it back to NettyBlockRpcServer.
| RpcResponseCallback callback = mock(RpcResponseCallback.class); | ||
|
|
||
| // Open blocks | ||
| handler.receive(reverseClient, openBlocks, callback); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Assert stream count after receiving an OpenBlocks message.
|
Hi @wangshuo128 , did you see the issue was resolved with this pr ? |
|
@Ngone51 Thanks a lot for comment and review!
I had the same concern too. I added some comment at TransportClient.fetchChunk . |
|
cc @viirya @cloud-fan @gatorsmile Could you give some advice about this PR? Thanks! |
|
is it same as #23521 ? |
|
Test build #101239 has finished for PR 23355 at commit
|
|
|
|
@Ngone51 @cloud-fan Yes, found that this pr is same as 23521 and 23521 is better. I will close this pr. Thank you. Also thanks @viirya ! |
|
@wangshuo128 ??? Why is better ? I can't get understand. #23521 does not intend to fix the issue you proposed here. And the time for registering a channel is obviously different between these two prs. |
|
oh, I miss the newest update, they're same now. Sorry, @wangshuo128 . |
What changes were proposed in this pull request?
In current code path,
OneForOneStreamManagerholdsStreamStatein a Map namedstreams.A
StreamStateis initialized and put intostreamswhenOpenBlocksrequest received.One specific
StreamStateis removed from streams in two scenarios below:ChunkFetchRequestis closedStreamStatewill never be clean up, ifOpenBlocksrequest is received without any followingChunkFetchRequest. This will cause memory leak in server side, which is harmful for long running service such asExternalShuffleService.This PR associates
StreamStatewith channel when handleOpenBlocksrequest, becauseOpenBlocksrequest and followingChunkFetchRequests for a specific stream are sent from the sameTransportClientinOneForOneBlockFetcher.How was this patch tested?
New test added.