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-36842][Core] TaskSchedulerImpl - stop TaskResultGetter properly #34098

Closed

Conversation

lxian
Copy link
Contributor

@lxian lxian commented Sep 24, 2021

What changes were proposed in this pull request?

Catch exception during TaskSchedulerImpl.stop() so that all components can be stopped properly

Why are the changes needed?

Otherwise some threads won't be stopped during spark session restart

Does this PR introduce any user-facing change?

NO

How was this patch tested?

It's tested by

  1. create a new spark session in yarn-client mode
  2. kill the spark application on yarn
  3. check that the spark context is stopped and create a new spark session
  4. do the above steps multiple times and verify that no task-result-getter threads number doesn't increase

@HyukjinKwon
Copy link
Member

cc @mridulm, @Ngone51 and @tgravescs FYI

@mridulm
Copy link
Contributor

mridulm commented Sep 26, 2021

The change looks good to me.
Do you want to do the same within SparkEnv.stop as well ?

@lxian
Copy link
Contributor Author

lxian commented Sep 26, 2021

The change looks good to me.
Do you want to do the same within SparkEnv.stop as well ?

sure, let me do it as well

@HyukjinKwon
Copy link
Member

ok to test

@SparkQA
Copy link

SparkQA commented Sep 27, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48149/

@SparkQA
Copy link

SparkQA commented Sep 27, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48149/

@@ -928,13 +928,17 @@ private[spark] class TaskSchedulerImpl(
override def stop(): Unit = {
speculationScheduler.shutdown()
if (backend != null) {
backend.stop()
Utils.tryLogNonFatalError {
backend.stop()
Copy link
Member

Choose a reason for hiding this comment

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

What's the exception you encountered here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

when deployed in yarn, org.apache.spark.scheduler.cluster.YarnSchedulerBackend#stop will call requestTotalExecutors() on stop. If the yarn application is killed already, it will receive an IOException on sending the RPC.

Copy link
Contributor

Choose a reason for hiding this comment

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

Checking more, what is the exception thrown in barrierCoordinator.stop ?
The should be defensive, and should not have resulted in failures.

Copy link
Member

Choose a reason for hiding this comment

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

How about wrapping the others too?

@SparkQA
Copy link

SparkQA commented Sep 27, 2021

Test build #143637 has finished for PR 34098 at commit b7580d1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@lxian
Copy link
Contributor Author

lxian commented Sep 27, 2021

The change looks good to me.
Do you want to do the same within SparkEnv.stop as well ?

I've checked SparkEnv.stop and I didn't find obvious unchecked exceptions that would break the stop(). I think it may not be necessary to do the same try..catch to SparkEnv.stop

@tgravescs
Copy link
Contributor

change look fine to me, it would be nice to have the stack trace on the exception thrown.

@mridulm
Copy link
Contributor

mridulm commented Sep 28, 2021

I've checked SparkEnv.stop and I didn't find obvious unchecked exceptions that would break the stop(). I think it may not be necessary to do the same try..catch to SparkEnv.stop

We have the following which can throw exceptions:

  • mapOutputTracker.stop() can throw SparkException in case of timeout.
  • blockManager.stop() can throw InterruptedException.
  • metricsSystem.stop() could throw exception - depends on the sink.
  • rpcEnv.shutdown() could throw InterruptedException (and others ?).
    • rpcEnv.awaitTermination could throw InterruptedException.

Note that sparkEnv.stop itself is protected - and so would not cause sc stop to be blocked.
Pls check if any of the above would prevent cleanup - and so reinit of a new sc to fail/be problematic.

@lxian
Copy link
Contributor Author

lxian commented Sep 29, 2021

I've checked SparkEnv.stop and I didn't find obvious unchecked exceptions that would break the stop(). I think it may not be necessary to do the same try..catch to SparkEnv.stop

We have the following which can throw exceptions:

  • mapOutputTracker.stop() can throw SparkException in case of timeout.

  • blockManager.stop() can throw InterruptedException.

  • metricsSystem.stop() could throw exception - depends on the sink.

  • rpcEnv.shutdown() could throw InterruptedException (and others ?).

    • rpcEnv.awaitTermination could throw InterruptedException.

Note that sparkEnv.stop itself is protected - and so would not cause sc stop to be blocked. Pls check if any of the above would prevent cleanup - and so reinit of a new sc to fail/be problematic.

  1. mapOutputTracker.stop(). I think the SparkException is already been wrapped with try..catch. I copied the code snippet below
  override def stop(): Unit = {
    mapOutputTrackerMasterMessages.offer(PoisonPill)
    threadpool.shutdown()
    try {
      sendTracker(StopMapOutputTracker)
    } catch {
      case e: SparkException =>
        logError("Could not tell tracker we are stopping.", e)
    }
    trackerEndpoint = null
    shuffleStatuses.clear()
  }
  1. metricsSystem.stop(). I've checked the current implementations of sink, and I didn't find one implementation that would through an Exception.
  2. And as for InterruptedException, I think they don't belong to the "NonFatal" type of Throwables. Maybe the best way to handle it is just let it been thrown out ?

@mridulm
Copy link
Contributor

mridulm commented Oct 31, 2021

Thanks for digging more @lxian !
Apologies for the delay in getting back on this; and to add to the answers to my queries.

  • Re: mapOutputTracker.stop() can throw SparkException in case of timeout
    • As @lxian pointed out, this cant happen now after Holden's changes (I think I might have been looking at a different branch, sorry for the confusion).
  • Re: metricsSystem.stop() could throw exception - depends on the sink.
    • As @lxian detailed, current spark Sink's should not cause this to happen. Having said that:
    • Spark supports plugging in custom Sink's : so looking only at what exists in our codebase is unfortunately insufficient.
      • An exception here prevents everything else in SparkEnv.stop from running
    • To be defensive, handling this would be better - thoughts ?

Both of these below are related to InterruptedException:

  • blockManager.stop() can throw InterruptedException
  • rpcEnv.awaitTermination could throw InterruptedException

I agree with @lxian, that is not caught by Utils.tryLogNonFatalError anyway - so let us preserve existing behavior for that.

Given the above, can we address the potential issue with Sink.close ?

@github-actions
Copy link

github-actions bot commented Feb 9, 2022

We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!

@github-actions github-actions bot added the Stale label Feb 9, 2022
@github-actions github-actions bot closed this Feb 10, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
7 participants