-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-46895][CORE][3.5] Replace Timer with single thread scheduled executor #47945
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-46895][CORE][3.5] Replace Timer with single thread scheduled executor #47945
Conversation
Kindly ping @beliefer @LuciferYang @dongjoon-hyun. |
This is only a minor improvement for Spark 4.0, not a bug fix, and it should not be backported to 3.x. |
@LuciferYang Thanks for reply! It actually includes a fix for https://issues.apache.org/jira/browse/SPARK-49479 as a side effect, by adding shutdown of a timer in BarrierCoordinator. |
@jshmchenxi, Can we submit separate pull requests along with their primitive forms? |
// TODO SPARK-25030 Create a Timer() in the mainClass submitted to SparkSubmit makes it unable to | ||
// fetch result, we shall fix the issue. | ||
private lazy val timer = new Timer("BarrierCoordinator barrier epoch increment timer") | ||
private lazy val timer = ThreadUtils.newSingleThreadScheduledExecutor( |
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.
The reason was that there is a non-daemon Timer thread named `BarrierCoordinator barrier epoch increment timer`, which prevented the driver JVM from stopping.
@jshmchenxi Based on the description in SPARK-49479, do you think the root cause of the problem is that the thread named BarrierCoordinator barrier epoch increment timer
is not a daemon thread? However, it seems that the currently used newSingleThreadScheduledExecutor
does not solve this problem because both the ThreadFactoryBuilder
and the scheduled task
s are not set to be Daemon
. Maybe we should consider using newDaemonThreadPoolScheduledExecutor
instead? So, does the master branch also have this problem? Can you provide a reproducible ut for the reviewer to verify the problem?
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.
That's right, we need to change to using newDaemonThreadPoolScheduledExecutor
to solve this issue. The master branch also has this problem. I'll submit another PR with a reproducible test. Thanks for your suggestion!
…or for ConsoleProgressBar ### What changes were proposed in this pull request? This PR propose to replace `Timer` with single thread scheduled executor for `ConsoleProgressBar`. ### Why are the changes needed? The javadoc recommends `ScheduledThreadPoolExecutor` instead of `Timer`.  This change based on the following two points. **System time sensitivity** Timer scheduling is based on the absolute time of the operating system and is sensitive to the operating system's time. Once the operating system's time changes, Timer scheduling is no longer precise. The scheduled Thread Pool Executor scheduling is based on relative time and is not affected by changes in operating system time. **Are anomalies captured** Timer does not capture exceptions thrown by Timer Tasks, and in addition, Timer is single threaded. Once a scheduling task encounters an exception, the entire thread will terminate and other tasks that need to be scheduled will no longer be executed. The scheduled Thread Pool Executor implements scheduling functions based on a thread pool. After a task throws an exception, other tasks can still execute normally. ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Manual tests. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes apache#44701 from beliefer/replace-timer-with-scheduled-executor. Authored-by: beliefer <beliefer@163.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> (cherry picked from commit fc66576)
### What changes were proposed in this pull request? This PR propose to replace `Timer` with single thread scheduled executor. ### Why are the changes needed? The javadoc recommends `ScheduledThreadPoolExecutor` instead of `Timer`.  This change based on the following two points. **System time sensitivity** Timer scheduling is based on the absolute time of the operating system and is sensitive to the operating system's time. Once the operating system's time changes, Timer scheduling is no longer precise. The scheduled Thread Pool Executor scheduling is based on relative time and is not affected by changes in operating system time. **Are anomalies captured** Timer does not capture exceptions thrown by Timer Tasks, and in addition, Timer is single threaded. Once a scheduling task encounters an exception, the entire thread will terminate and other tasks that need to be scheduled will no longer be executed. The scheduled Thread Pool Executor implements scheduling functions based on a thread pool. After a task throws an exception, other tasks can still execute normally. ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? GA tests. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes apache#44718 from beliefer/replace-timer-with-threadpool. Authored-by: beliefer <beliefer@163.com> Signed-off-by: yangjie01 <yangjie01@baidu.com> (cherry picked from commit 5d5b3a5)
8d6f7f1
to
ef562e3
Compare
Yes, I created another PR for the first commit: #47957. The second commit is in this PR. And the fix for SPARK-49479 is #47956. It depends on the other 2 commits to fix in branch 3.5. |
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. |
Depends on the first commit for fixing SPARK-49479 in branch 3.5: #47957
-> This is the second commit for fixing SPARK-49479 in branch 3.5.
Third commit for fixing SPARK-49479 in branch 3.5: #47956
What changes were proposed in this pull request?
This PR propose to replace
Timer
with single thread scheduled executor.Why are the changes needed?
The javadoc recommends

ScheduledThreadPoolExecutor
instead ofTimer
.This change based on the following two points.
System time sensitivity
Timer scheduling is based on the absolute time of the operating system and is sensitive to the operating system's time. Once the operating system's time changes, Timer scheduling is no longer precise.
The scheduled Thread Pool Executor scheduling is based on relative time and is not affected by changes in operating system time.
Are anomalies captured
Timer does not capture exceptions thrown by Timer Tasks, and in addition, Timer is single threaded. Once a scheduling task encounters an exception, the entire thread will terminate and other tasks that need to be scheduled will no longer be executed.
The scheduled Thread Pool Executor implements scheduling functions based on a thread pool. After a task throws an exception, other tasks can still execute normally.
Does this PR introduce any user-facing change?
'No'.
How was this patch tested?
GA tests.
Was this patch authored or co-authored using generative AI tooling?
'No'.