-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-50768][CORE] Introduce TaskContext.createResourceUninterruptibly to avoid stream leak by task interruption #49413
Conversation
core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
Outdated
Show resolved
Hide resolved
|
||
// Leave some time for the task to be interrupted during the | ||
// creation of `InterruptionSensitiveInputStream`. | ||
Thread.sleep(5000) |
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.
How important is this sleep within the task? Could it potentially make the test flaky?
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.
Good point
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.
It is necessary to ensure the task is interrupted during InterruptionSensitiveInputStream#initialize()
so that we can test the leaked stream. Increase the sleep time should less likely be flaky.
core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
Outdated
Show resolved
Hide resolved
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
Could someone help merge this PR if we're good to proceed? Or if I can also merge myself with enough approvals? |
@Ngone51 Does branch-3.5 also need this bug fix? |
@LuciferYang Thanks. Yes, I think so. Do I need send a separate PR? |
Merged into master for Spark 4.0. Thanks @Ngone51 @xuanyuanking and @HyukjinKwon . @Ngone51 Due to code conflicts, it cannot be directly merged into branch-3.5. If needed, please submit a separate pr. Thanks ~ |
Sure, I'd submit a separate PR for branch-3.5. |
Hi @LuciferYang I found we didn't backport #48483 to branch-3.5 for the same issue. Generally speaking, this is not a critial bug fix but only for the corner case. Not sure if we want to backport it or not. @HyukjinKwon Any idea about the backport policy for this kind of fix? is it a requirement or optional? |
please go ahead. I think it's fien |
Please hold one backporting for now, @Ngone51 . It's because this could introduce a flakiness into In addition, if you can, please give us a confirmation that the failures of current master branch are irrelevant to this PR. |
override private[spark] def interruptible(): Boolean = taskContext.interruptible() | ||
|
||
override private[spark] def pendingInterrupt(threadToInterrupt: Option[Thread], reason: String) | ||
: Unit = { |
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 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.
I'll fix it with a followup. Thanks!
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.
Fixed at #49508
} | ||
|
||
override private[spark] def createResourceUninterruptibly[T <: Closeable](resourceBuilder: => T) | ||
: T = { |
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.
im taking a look now for the failure |
Thank you so much! I looked at, but didn't find any clear clue. |
Seems unrelated, I tested the result of reverting the current one, but |
found the cuase. working on the fix |
no tihs pr is fine |
@HyukjinKwon Are you saying this pr caused the test to fail? |
nono this PR is fine. I will make a PR soon with a PR description with 20 mins |
here: #49500 |
// Should be protected by `TaskContext.synchronized`. | ||
private var pendingInterruptRequest: Option[(Option[Thread], String)] = None | ||
|
||
// Whether this task is able to be interrupted. Should be protected by `TaskContext.synchronized`. |
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.
or maybe mark as @transient
?
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.
Are you referring to @volatile
?
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.
Since we already have TaskContext.synchronized
protected, @volatile
could be redundant. And if we use @volatile
only, it's not thread safety as we could have two threads modify _interruptible
concurrently.
|
||
override def interruptible(): Boolean = TaskContext.synchronized(_interruptible) | ||
|
||
override def pendingInterrupt(threadToInterrupt: Option[Thread], reason: String): Unit = { |
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.
This API looks weird if threadToInterrupt
is None
, as there is nothing to interrupt.
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.
When threadToInterrupt=None
, it would still "interrupt" the task by invoking TaskContext.markInterrupted()
. It just won't invoke Thread.interrupt()
on the task thread.
} | ||
|
||
TaskContext.synchronized { | ||
interruptIfRequired() |
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 it a safeguard that the caller may mistakenly call pendingInterrupt
even if the task is interruptable?
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.
No. This is for the case where resource creation happens after the task has been marked as interrupted
. In that case, pendingInterruptRequest
is None
and reasonIfKilled
is not None
, and killTaskIfInterrupted()
would throw TaskKilledException()
to stop the task thread.
FYI I created a followup PR (#49508) to use |
…ninterruptibly() to risky resource creations ### What changes were proposed in this pull request? This is a follow-up PR for #49413. This PR intends to apply `TaskContext.createResourceUninterruptibly()` to the resource creation where it has the potential risk of resource leak in the case of task cancellation. ### Why are the changes needed? Avoid resource leak. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? n/a ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49508 from Ngone51/SPARK-50768-followup. Authored-by: Yi Wu <yi.wu@databricks.com> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
…ninterruptibly() to risky resource creations ### What changes were proposed in this pull request? This is a follow-up PR for #49413. This PR intends to apply `TaskContext.createResourceUninterruptibly()` to the resource creation where it has the potential risk of resource leak in the case of task cancellation. ### Why are the changes needed? Avoid resource leak. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? n/a ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49508 from Ngone51/SPARK-50768-followup. Authored-by: Yi Wu <yi.wu@databricks.com> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit 9b32334) Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
…atabricks] (#11972) This fixes #11971 . This unblocks the scala_2.13 nightly build jobs. MockTaskContext extends TaskContext in spark-rapids code. Recently additional definitions were added in Spark-4.0's TaskContext. In this PR, we added the missing methods to MockTaskContext.scala file. We are not overriding these methods so that it can compile on all Spark versions. In Spark-4.0, these were added in TaskContext.scala: PR : apache/spark#49413 ``` private[spark] def interruptible(): Boolean private[spark] def pendingInterrupt(threadToInterrupt: Option[Thread], reason: String): Unit private[spark] def createResourceUninterruptibly[T <: Closeable](resourceBuilder: => T): T ``` Signed-off-by: Niranjan Artal <nartal@nvidia.com> Co-authored-by: Gera Shegalov <gshegalov@nvidia.com>
What changes were proposed in this pull request?
This PR fixes the potential stream leak issue by introduing
TaskContext.createResourceUninterruptibly
.When a task is using
TaskContext.createResourceUninterruptibly
to create the resource, the task would be marked as uninterruptible. Thus, any interruption request during the call toTaskContext.createResourceUninterruptibly
would be delayed until the creation finishes.This PR introduces an new lock contention between
Task.kill
andTaskContext.createResourceUninterruptibly
. But I think it is acceptable given that both are not on the hot-path.(I will submmit a followup to apply
TaskContext.createResourceUninterruptibly
in the codebase if this PR is approved by the community.)Why are the changes needed?
We had #48483 tried to fix the potential stream leak issue by task interruption. It mitigates the issue by using
But this utility function has an issue that
resource.close()
would leak open resouces ifinitialize(resource)
also created some resources internally, especially wheninitialize(resource)
is interrupted (See the example ofInterruptionSensitiveInputStream
in the test).Does this PR introduce any user-facing change?
No.
How was this patch tested?
Added a unit test.
Was this patch authored or co-authored using generative AI tooling?
No.