Skip to content
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

[SPARK-41187][CORE] LiveExecutor MemoryLeak in AppStatusListener when ExecutorLost happen #38702

Closed
wants to merge 1 commit into from

Conversation

wineternity
Copy link

@wineternity wineternity commented Nov 18, 2022

What changes were proposed in this pull request?

Ignore the SparkListenerTaskEnd with Reason "Resubmitted" in AppStatusListener to avoid memory leak

Why are the changes needed?

For a long running spark thriftserver, LiveExecutor will be accumulated in the deadExecutors HashMap and cause message event queue processing slowly.
For a every task, actually always sent out a SparkListenerTaskStart event and a SparkListenerTaskEnd event, they are always pairs. But in a executor lost situation, it send out event like following steps.

a) There was a pair of task start and task end event which were fired for the task (let us call it Tr)
b) When executor which ran Tr was lost, while stage is still running, a task end event with reason Resubmitted is fired for Tr.
c) Subsequently, a new task start and task end will be fired for the retry of Tr.

The processing of the Resubmitted task end event in AppStatusListener can lead to negative LiveStage.activeTasks since there's no corresponding SparkListenerTaskStart event for each of them. The negative activeTasks will make the stage always remains in the live stage list as it can never meet the condition activeTasks == 0. This in turn causes the dead executor to never be cleaned up if that live stage's submissionTime is less than the dead executor's removeTime( see isExecutorActiveForLiveStages). Since this kind of SparkListenerTaskEnd is useless here, we simply ignore it.

Check SPARK-41187 for evidences.

Does this PR introduce any user-facing change?

No

How was this patch tested?

New UT Added
Test in thriftserver env

The way to reproduce

I try to reproduce it in spark shell, but it is a little bit handy

  1. start spark-shell , set spark.dynamicAllocation.maxExecutors=2 for convient
    bin/spark-shell --driver-java-options "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8006"
  2. run a job with shuffle
    sc.parallelize(1 to 1000, 10).map { x => Thread.sleep(1000) ; (x % 3, x) }.reduceByKey((a, b) => a + b).collect()
  3. After some ShuffleMapTask finished, kill one or two executor to let tasks resubmitted
  4. check by heap dump or debug or log

@github-actions github-actions bot added the CORE label Nov 18, 2022
@itholic
Copy link
Contributor

itholic commented Nov 18, 2022

Can we change the JIRA format in the title such as "[SPARK-41187][CORE] ...".

Check the Spark contribution guide also would helpful!

@wineternity wineternity changed the title SPARK-41187 [Core] LiveExecutor MemoryLeak in AppStatusListener when ExecutorLost happen [SPARK-41187][Core] LiveExecutor MemoryLeak in AppStatusListener when ExecutorLost happen Nov 18, 2022
@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@wineternity
Copy link
Author

@itholic @vanzin Could you help review this patch? thanks very much.

@HyukjinKwon HyukjinKwon changed the title [SPARK-41187][Core] LiveExecutor MemoryLeak in AppStatusListener when ExecutorLost happen [SPARK-41187][CORE] LiveExecutor MemoryLeak in AppStatusListener when ExecutorLost happen Nov 22, 2022
@wineternity
Copy link
Author

cc @cloud-fan

@cloud-fan
Copy link
Contributor

cc @jiangxb1987 @Ngone51

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The change looks good to me.
+CC @Ngone51

Btw, do you also want to remove the if (event.taskInfo == null) { check in beginning of onTaskEnd ?

Make it a precondition check ? Preconditions.checkNotNull(event.taskInfo)

@wineternity
Copy link
Author

The change looks good to me. +CC @Ngone51

Btw, do you also want to remove the if (event.taskInfo == null) { check in beginning of onTaskEnd ?

Make it a precondition check ? Preconditions.checkNotNull(event.taskInfo)

Yes, it can be change to a precondition check. Maybe I can change it in a new pr after test.

@@ -1849,6 +1849,68 @@ abstract class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter
checkInfoPopulated(listener, logUrlMap, processId)
}

test(s"SPARK-41187: Stage should be removed from liveStages to avoid deadExecutors accumulated") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
test(s"SPARK-41187: Stage should be removed from liveStages to avoid deadExecutors accumulated") {
test("SPARK-41187: Stage should be removed from liveStages to avoid deadExecutors accumulated") {

@Ngone51
Copy link
Member

Ngone51 commented Dec 9, 2022

Btw, do you also want to remove the if (event.taskInfo == null) { check in beginning of onTaskEnd ?

@mridulm Since the latest PR fix doesn't involve the metrics, I think we can skip this removal to keep the current changes as much simpler as possible. We can back to it when working on metrics stuff.

@wineternity wineternity force-pushed the SPARK-41187 branch 2 times, most recently from 634eefe to ea1307c Compare December 9, 2022 09:08
…ExecutorLost happen

Co-authored-by: wuyi <yi.wu@databricks.com>
Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Thank you, @wineternity and all.

@wineternity
Copy link
Author

+1, LGTM. Thank you, @wineternity and all.

my pleasure

@mridulm mridulm closed this in 7e7bc94 Dec 12, 2022
mridulm pushed a commit that referenced this pull request Dec 12, 2022
… ExecutorLost happen

### What changes were proposed in this pull request?
Ignore the SparkListenerTaskEnd with Reason "Resubmitted" in AppStatusListener to avoid memory leak

### Why are the changes needed?
For a long running spark thriftserver,  LiveExecutor will be accumulated in the deadExecutors HashMap and cause message event queue processing slowly.
For a every task, actually always sent out a `SparkListenerTaskStart` event and a `SparkListenerTaskEnd` event, they are always pairs. But in a executor lost situation, it send out event like following steps.

a) There was a pair of task start and task end event which were fired for the task (let us call it Tr)
b) When executor which ran Tr was lost, while stage is still running, a task end event with reason `Resubmitted`  is fired for Tr.
c) Subsequently, a new task start and task end will be fired for the retry of Tr.

The processing of the `Resubmitted` task end event in AppStatusListener can lead to negative `LiveStage.activeTasks` since there's no corresponding `SparkListenerTaskStart` event for each of them. The negative activeTasks will make the stage always remains in the live stage list as it can never meet the condition activeTasks == 0. This in turn causes the dead executor to never be cleaned up if that live stage's submissionTime is less than the dead executor's removeTime( see isExecutorActiveForLiveStages). Since this kind of `SparkListenerTaskEnd` is useless here, we simply ignore it.

Check  [SPARK-41187](https://issues.apache.org/jira/browse/SPARK-41187) for evidences.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
New UT Added
Test in thriftserver env

### The way to reproduce
I try to reproduce it in spark shell, but it is a little bit handy
1.  start spark-shell , set spark.dynamicAllocation.maxExecutors=2 for convient
` bin/spark-shell  --driver-java-options "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8006"`
2. run a job with shuffle
 `sc.parallelize(1 to 1000, 10).map { x => Thread.sleep(1000) ; (x % 3, x) }.reduceByKey((a, b) => a + b).collect()`
3. After some ShuffleMapTask finished, kill one or two executor to let tasks resubmitted
4. check by heap dump or debug or log

Closes #38702 from wineternity/SPARK-41187.

Authored-by: yuanyimeng <yuanyimeng@youzan.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 7e7bc94)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
@mridulm
Copy link
Contributor

mridulm commented Dec 12, 2022

Merged to master and brach-3.3
Thanks for working on this @wineternity !
Thanks for the reviews @cloud-fan, @Ngone51, @dongjoon-hyun :-)

beliefer pushed a commit to beliefer/spark that referenced this pull request Dec 18, 2022
… ExecutorLost happen

### What changes were proposed in this pull request?
Ignore the SparkListenerTaskEnd with Reason "Resubmitted" in AppStatusListener to avoid memory leak

### Why are the changes needed?
For a long running spark thriftserver,  LiveExecutor will be accumulated in the deadExecutors HashMap and cause message event queue processing slowly.
For a every task, actually always sent out a `SparkListenerTaskStart` event and a `SparkListenerTaskEnd` event, they are always pairs. But in a executor lost situation, it send out event like following steps.

a) There was a pair of task start and task end event which were fired for the task (let us call it Tr)
b) When executor which ran Tr was lost, while stage is still running, a task end event with reason `Resubmitted`  is fired for Tr.
c) Subsequently, a new task start and task end will be fired for the retry of Tr.

The processing of the `Resubmitted` task end event in AppStatusListener can lead to negative `LiveStage.activeTasks` since there's no corresponding `SparkListenerTaskStart` event for each of them. The negative activeTasks will make the stage always remains in the live stage list as it can never meet the condition activeTasks == 0. This in turn causes the dead executor to never be cleaned up if that live stage's submissionTime is less than the dead executor's removeTime( see isExecutorActiveForLiveStages). Since this kind of `SparkListenerTaskEnd` is useless here, we simply ignore it.

Check  [SPARK-41187](https://issues.apache.org/jira/browse/SPARK-41187) for evidences.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
New UT Added
Test in thriftserver env

### The way to reproduce
I try to reproduce it in spark shell, but it is a little bit handy
1.  start spark-shell , set spark.dynamicAllocation.maxExecutors=2 for convient
` bin/spark-shell  --driver-java-options "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8006"`
2. run a job with shuffle
 `sc.parallelize(1 to 1000, 10).map { x => Thread.sleep(1000) ; (x % 3, x) }.reduceByKey((a, b) => a + b).collect()`
3. After some ShuffleMapTask finished, kill one or two executor to let tasks resubmitted
4. check by heap dump or debug or log

Closes apache#38702 from wineternity/SPARK-41187.

Authored-by: yuanyimeng <yuanyimeng@youzan.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
7 participants