-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-48547][DEPLOY] Add opt-in flag to have SparkSubmit automatically call System.exit after user code main method exits #46889
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
|
I think this patch also covers the SPARK-34674 and SPARK-42698 cases. And maybe the code introduced in SPARK-34674 can be removed. SPARK-34674 is used to tackle the Spark on K8s exit issues, it also introduces side-effects:
In SPARK-42698, a similar shutdown logic was requested for the YARN client mode to propagate the exit code to AM. |
Thanks for the pointers. I was aware of SPARK-34674 for Spark on Kubernetes (I spotted its logic when editing This raises an interesting question of whether my new flag should also be calling spark/core/src/main/scala/org/apache/spark/SparkContext.scala Lines 691 to 704 in 7cba1ab
would stop the context, but that gives us less explicit control around the relative ordering of the SparkContext stop and any other cleanup activities performed by user- or third-party library shutdown hooks (some of which might be implicitly (and possibly incorrectly) assuming that the SparkContext will have already been stopped before their hooks run). Explicitly stopping the SparkContext would have the advantage of letting us propagate the exit code (as was proposed in #40314). If I do that, though, then my new proposed configuration's name might need to change to reflect the more general behavior (right now There's also a question of defaults and user-facing behavior changes: although "stop the context then call System.exit()" is probably a reasonable behavior for most cases, there might be a long-tail of less common use-cases where users explicitly don't want any automatic cleanup or shutdown. Today, for better or worse, we have a set of on-by-default cleanup behaviors for Spark on Kubernetes and I'm slightly hesitant to want to change those defaults for fear of breaking someone's use-case. For example, maybe a user wants the SparkContext to be automatically stopped on job completion but doesn't want System.exit() to be called (e.g. because they're using the org.apache.spark.launcher package's Maybe we should optimize for some sensible "batteries included" top-level cleanup option which can be completely disabled by users who want to handle the cleanup themselves, but, when enabled, does a reasonably linear standard cleanup flow? |
I believe
... and versus, I do see that some users write shutdown hooks rely on the priority to assume that the execution happens before/after SparkContext shutdown.
Yeah, I understand. Well, on the other hand, Spark 4.0 may be a good opportunity(at least better than minor versions like 4.1, 4.2 ...) to make such a breaking change with a more reasonable and consistent(as mentioned above, the default K8s's shutdown behavior is different from other platforms), as long as it has flags to allow users change back to previous behaviors.
And IIRC, in Spark on YARN cluster mode(the most popular cases in my company and customers), the AM will call |
As We do see perf benefits of |
|
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. |
|
Hi @JoshRosen, do you have a plan to move this PR forward? I consistently get reports from users who migrate Spark workloads from YARN to K8s complaining that daemon threads created by user codes block the driver JVM exit ... |
…ly call System.exit after user code main method exits This PR is based on #46889 authored by JoshRosen ### What changes were proposed in this pull request? This PR adds a new SparkConf flag option, `spark.submit.callSystemExitOnMainExit` (default false), which when true will cause SparkSubmit to call `System.exit()` in the JVM once the user code's main method has exited (for Java / Scala jobs) or once the user's Python or R script has exited. ### Why are the changes needed? This is intended to address a longstanding issue where `spark-submit` runs might hang after user code has completed: [According to Java’s java.lang.Runtime docs](https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/lang/Runtime.html#shutdown): > The Java Virtual Machine initiates the shutdown sequence in response to one of several events: > > 1. when the number of [live](https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/lang/Thread.html#isAlive()) non-daemon threads drops to zero for the first time (see note below on the JNI Invocation API); > 2. when the Runtime.exit or System.exit method is called for the first time; or > 3. when some external event occurs, such as an interrupt or a signal is received from the operating system. For Python and R programs, SparkSubmit’s PythonRunner and RRunner will call System.exit() if the user program exits with a non-zero exit code (see [python](https://github.com/apache/spark/blob/d5c33c6bfb5757b243fc8e1734daeaa4fe3b9b32/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala#L101-L104) and [R](https://github.com/apache/spark/blob/d5c33c6bfb5757b243fc8e1734daeaa4fe3b9b32/core/src/main/scala/org/apache/spark/deploy/RRunner.scala#L109-L111) runner code). But for Java and Scala programs, plus any successful R or Python programs, Spark will not automatically call System.exit. In those situation, the JVM will only shutdown when, via event (1), all non-[daemon](https://stackoverflow.com/questions/2213340/what-is-a-daemon-thread-in-java) threads have exited (unless the job is cancelled and sent an external interrupt / kill signal, corresponding to event (3)). Thus, non-daemon threads might cause logically-completed spark-submit jobs to hang rather than completing. The non-daemon threads are not always under Spark's own control and may not necessarily be cleaned up by `SparkContext.stop()`. Thus, it is useful to have an opt-in functionality to have SparkSubmit automatically call `System.exit()` upon main method exit (which usually, but not always, corresponds to job completion): this option will allow users and data platform operators to enforce System.exit() calls without having to modify individual jobs' code. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new user-facing configuration option for opting in to a behavior change. ### How was this patch tested? New tests in `SparkSubmitSuite`, including one which hangs (failing with a timeout) unless the new option is set to `true`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #52091 from pan3793/SPARK-48547. Lead-authored-by: Josh Rosen <[email protected]> Co-authored-by: Cheng Pan <[email protected]> Signed-off-by: Kent Yao <[email protected]>
What changes were proposed in this pull request?
This PR adds a new SparkConf flag option,
spark.submit.callSystemExitOnMainExit(default false), which when true will cause SparkSubmit to callSystem.exit()in the JVM once the user code's main method has exited (for Java / Scala jobs) or once the user's Python or R script has exited.Why are the changes needed?
This is intended to address a longstanding issue where
spark-submitruns might hang after user code has completed:According to Java’s java.lang.Runtime docs:
For Python and R programs, SparkSubmit’s PythonRunner and RRunner will call System.exit() if the user program exits with a non-zero exit code (see python and R runner code).
But for Java and Scala programs, plus any successful R or Python programs, Spark will not automatically call System.exit.
In those situation, the JVM will only shutdown when, via event (1), all non-daemon threads have exited (unless the job is cancelled and sent an external interrupt / kill signal, corresponding to event (3)).
Thus, non-daemon threads might cause logically-completed spark-submit jobs to hang rather than completing.
The non-daemon threads are not always under Spark's own control and may not necessarily be cleaned up by
SparkContext.stop().Thus, it is useful to have an opt-in functionality to have SparkSubmit automatically call
System.exit()upon main method exit (which usually, but not always, corresponds to job completion): this option will allow users and data platform operators to enforce System.exit() calls without having to modify individual jobs' code.Does this PR introduce any user-facing change?
Yes, it adds a new user-facing configuration option for opting in to a behavior change.
How was this patch tested?
New tests in
SparkSubmitSuite, including one which hangs (failing with a timeout) unless the new option is set totrue.Was this patch authored or co-authored using generative AI tooling?
No.