-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-30246][CORE] OneForOneStreamManager might leak memory in connectionTerminated #27064
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
OneForOneStreamManager may not remove all StreamStates from memory map when a connection is terminated. A RuntimeException might be thrown in StreamState$buffers.next() by one of ExternalShuffleBlockResolver$getBlockData/getRddBlock... breaking the loop through streams.entrySet() This commit removes all StreamStates from memory map first and, after that, releases all buffers from each removed StreamState
common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java
Outdated
Show resolved
Hide resolved
…re's any exception on releasing any state.
|
cc. @viirya |
| // Release all remaining buffers. | ||
| try { | ||
| while (state.buffers.hasNext()) { | ||
| ManagedBuffer buffer = state.buffers.next(); |
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.
Once we got RuntimeException, don't we just fail? Is there memory leak?
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.
Lines 87 to 97 in 1b0570c
| @Override | |
| public void channelInactive() { | |
| if (streamManager != null) { | |
| try { | |
| streamManager.connectionTerminated(channel); | |
| } catch (RuntimeException e) { | |
| logger.error("StreamManager connectionTerminated() callback failed.", e); | |
| } | |
| } | |
| rpcHandler.channelInactive(reverseClient); | |
| } |
TransportRequestHandler.channelInactive will log error message and swallow the exception, so no, we don't fail because of exception thrown here.
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.
oh I see. Then this change may also swallow the RuntimeException so channelInactive cannot log as before? Should we rethrow the same exception?
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.
So I think there're two options here with achieving goal;
-
store any exception between exceptions (as we'll try to release from all the streams) and rethrow, then channelInactive will log it. other exceptions should be logged as well but maybe at here (or a new exception containing all exceptions and throw it instead).
-
catch them, and log and swallow at here.
Technically the change from logging side would be logger name which may not a big deal, but it's also valid concern if we think channelInactive is the one to decide how to handle the exception.
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.
yea, I think it might make the caller (channelInactive or others if any) thinks everything is fine inside connectionTerminated. Maybe not big deal for now, but sounds a potential concern.
We might have multiple exception during releasing buffers. We can rethrow a RuntimeException if any exception happens during releasing buffers.
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.
OK thanks for the input. @hensg Could you follow up the comment above? Thanks!
|
ok to test. |
|
Test build #116058 has finished for PR 27064 at commit
|
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.
there exists an case, if the application has already finished(failed), executors' info of the app has been cleaned, but the connectionTerminated be called delay, when execute this code, it will look up the buffer with appId and execId , so the buffer cannot be got and released. this happend in spark 2.4.3 as I report and I also make a pr, pls help review(#27060), thanks
|
@012huang I feel it's orthogonal to this. Could you please cc. to some committers modified around the code having bug in your PR? |
HeartSaVioR
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.
LGTM. cc. @viirya Could you please take a look again? Thanks in advance.
|
Test build #116383 has finished for PR 27064 at commit
|
[ERROR] src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java:[123] (sizes) LineLength: Line is longer than 100 characters (found 106).
|
Test build #116384 has finished for PR 27064 at commit
|
|
retest this, please |
|
Test build #116398 has finished for PR 27064 at commit
|
|
retest this please |
|
Test build #116414 has finished for PR 27064 at commit
|
|
looks like a valid failure |
|
@viirya The test fails from other PRs as well - please refer #26201 (comment) Looks like other PRs succeed to build because the pyspark test seems run conditionally. |
|
@HeartSaVioR hmm, I just ran the test locally and it passed. |
|
Yeah same for me as well, but I don't have pypy. Installing now. EDIT: and no luck. It works with my local. Maybe there's something with Jenkins worker machine? |
|
EDIT: It seems to fail intermittently, so not able to reproduce the original failure.
|
### What changes were proposed in this pull request? This patch increases the memory limit in the test 'test_memory_limit' from 1m to 8m. Credit to srowen and HyukjinKwon to provide the idea of suspicion and guide how to fix. ### Why are the changes needed? We observed consistent Pyspark test failures on multiple PRs (apache#26955, apache#26201, apache#27064) which block the PR builds whenever the test is included. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Jenkins builds passed in WIP PR (apache#27159) Closes apache#27162 from HeartSaVioR/SPARK-30480. Authored-by: Jungtaek Lim (HeartSaVioR) <[email protected]> Signed-off-by: HyukjinKwon <[email protected]>
|
retest this please |
|
Test build #116453 has finished for PR 27064 at commit
|
|
retest this, please |
|
Test build #116472 has finished for PR 27064 at commit
|
|
Test build #116556 has finished for PR 27064 at commit
|
|
Test build #116663 has finished for PR 27064 at commit
|
viirya
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.
looks good. cc @cloud-fan
| buffer.release(); | ||
| } | ||
| } | ||
| } catch (RuntimeException e) { |
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.
Is this the only exception type we need to deal with here?
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.
Other kind of exception would be Error which TransportRequestHandler.channelInactive is also not catching and let the process fail.
common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java
Outdated
Show resolved
Hide resolved
|
Test build #116727 has finished for PR 27064 at commit
|
|
cc @vanzin , too. |
|
Merging to master / 2.4. |
…ctionTerminated Ensure that all StreamStates are removed from OneForOneStreamManager memory map even if there's an error trying to release buffers OneForOneStreamManager may not remove all StreamStates from memory map when a connection is terminated. A RuntimeException might be thrown in StreamState$buffers.next() by one of ExternalShuffleBlockResolver$getBlockData... **breaking the loop through streams.entrySet(), keeping StreamStates in memory forever leaking memory.** That may happen when an application is terminated abruptly and executors removed before the connection is terminated or if shuffleIndexCache fails to get ShuffleIndexInformation References: https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java#L319 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java#L357 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L195 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L208 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L330 No Unit test added Closes #27064 from hensg/SPARK-30246. Lead-authored-by: Henrique Goulart <[email protected]> Co-authored-by: Henrique Goulart <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]> (cherry picked from commit d42cf45) Signed-off-by: Marcelo Vanzin <[email protected]>
|
FYI: fixed a trivial conflict in 2.4, and had to add a couple of things to the test:
|
|
Thank you all! |
What changes were proposed in this pull request?
Ensure that all StreamStates are removed from OneForOneStreamManager memory map even if there's an error trying to release buffers
Why are the changes needed?
OneForOneStreamManager may not remove all StreamStates from memory map when a connection is terminated. A RuntimeException might be thrown in StreamState$buffers.next() by one of ExternalShuffleBlockResolver$getBlockData... breaking the loop through streams.entrySet(), keeping StreamStates in memory forever leaking memory.
That may happen when an application is terminated abruptly and executors removed before the connection is terminated or if shuffleIndexCache fails to get ShuffleIndexInformation
References:
https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java#L319
https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java#L357
https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L195
https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L208
https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L330
Does this PR introduce any user-facing change?
No
How was this patch tested?
Unit test added