-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19820] [core] Allow reason to be specified for task kill #17166
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
Changes from all commits
91b8aef
a58d391
02d81b5
170fa34
614ad3c
72b28cb
fda712d
8f7ffb3
348e97a
f5069f7
884a3ad
203a900
6e8593b
5707715
a37c09b
71b41b3
3ec3633
145c78a
8c4381f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -215,7 +215,7 @@ private[spark] class PythonRunner( | |
|
|
||
| case e: Exception if context.isInterrupted => | ||
| logDebug("Exception thrown after task interruption", e) | ||
| throw new TaskKilledException | ||
| throw new TaskKilledException(context.getKillReason().getOrElse("unknown reason")) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do you need the getOrElse here? (since isInterrupted is true, shouldn't this always be defined?)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @mridulm pointed out that should the kill reason get reset to None by a concurrent thread, this would crash. However, it is true that this can't happen in the current implementation. If you think it's clearer, we could throw an AssertionError in this case.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hm ok if Mridul wants this then fine to leave as-is
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @kayousterhout I actually had not considered this, but the use of maybeKillReason in Executor/other places; this was a nice catch by @ericl |
||
|
|
||
| case e: Exception if env.isStopped => | ||
| logDebug("Exception thrown after context is stopped", e) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -158,7 +158,7 @@ private[spark] class Executor( | |
| threadPool.execute(tr) | ||
| } | ||
|
|
||
| def killTask(taskId: Long, interruptThread: Boolean): Unit = { | ||
| def killTask(taskId: Long, interruptThread: Boolean, reason: String): Unit = { | ||
| val taskRunner = runningTasks.get(taskId) | ||
| if (taskRunner != null) { | ||
| if (taskReaperEnabled) { | ||
|
|
@@ -168,7 +168,8 @@ private[spark] class Executor( | |
| case Some(existingReaper) => interruptThread && !existingReaper.interruptThread | ||
| } | ||
| if (shouldCreateReaper) { | ||
| val taskReaper = new TaskReaper(taskRunner, interruptThread = interruptThread) | ||
| val taskReaper = new TaskReaper( | ||
| taskRunner, interruptThread = interruptThread, reason = reason) | ||
|
||
| taskReaperForTask(taskId) = taskReaper | ||
| Some(taskReaper) | ||
| } else { | ||
|
|
@@ -178,7 +179,7 @@ private[spark] class Executor( | |
| // Execute the TaskReaper from outside of the synchronized block. | ||
| maybeNewTaskReaper.foreach(taskReaperPool.execute) | ||
| } else { | ||
| taskRunner.kill(interruptThread = interruptThread) | ||
| taskRunner.kill(interruptThread = interruptThread, reason = reason) | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -189,8 +190,9 @@ private[spark] class Executor( | |
| * tasks instead of taking the JVM down. | ||
| * @param interruptThread whether to interrupt the task thread | ||
| */ | ||
| def killAllTasks(interruptThread: Boolean) : Unit = { | ||
| runningTasks.keys().asScala.foreach(t => killTask(t, interruptThread = interruptThread)) | ||
| def killAllTasks(interruptThread: Boolean, reason: String) : Unit = { | ||
| runningTasks.keys().asScala.foreach(t => | ||
| killTask(t, interruptThread = interruptThread, reason = reason)) | ||
| } | ||
|
|
||
| def stop(): Unit = { | ||
|
|
@@ -217,8 +219,8 @@ private[spark] class Executor( | |
| val threadName = s"Executor task launch worker for task $taskId" | ||
| private val taskName = taskDescription.name | ||
|
|
||
| /** Whether this task has been killed. */ | ||
| @volatile private var killed = false | ||
| /** If specified, this task has been killed and this option contains the reason. */ | ||
| @volatile private var reasonIfKilled: Option[String] = None | ||
|
|
||
| @volatile private var threadId: Long = -1 | ||
|
|
||
|
|
@@ -239,13 +241,13 @@ private[spark] class Executor( | |
| */ | ||
| @volatile var task: Task[Any] = _ | ||
|
|
||
| def kill(interruptThread: Boolean): Unit = { | ||
| logInfo(s"Executor is trying to kill $taskName (TID $taskId)") | ||
| killed = true | ||
| def kill(interruptThread: Boolean, reason: String): Unit = { | ||
| logInfo(s"Executor is trying to kill $taskName (TID $taskId), reason: $reason") | ||
| reasonIfKilled = Some(reason) | ||
| if (task != null) { | ||
| synchronized { | ||
| if (!finished) { | ||
| task.kill(interruptThread) | ||
| task.kill(interruptThread, reason) | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -296,12 +298,13 @@ private[spark] class Executor( | |
|
|
||
| // If this task has been killed before we deserialized it, let's quit now. Otherwise, | ||
| // continue executing the task. | ||
| if (killed) { | ||
| val killReason = reasonIfKilled | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why re-name the variable here (instead of just using reasonIfKilled below)?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we assign to a temporary, then there is no risk of seeing concurrent mutations of the value as we access it below (though, this cannot currently happen).
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ugh in retrospect I think TaskContext should have just clearly documented that an invariant of reasonIfKilled is that, once set, it won't be un-set, and then we'd avoid all of these corner cases. But not worth changing now. |
||
| if (killReason.isDefined) { | ||
| // Throw an exception rather than returning, because returning within a try{} block | ||
| // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl | ||
| // exception will be caught by the catch block, leading to an incorrect ExceptionFailure | ||
| // for the task. | ||
| throw new TaskKilledException | ||
| throw new TaskKilledException(killReason.get) | ||
| } | ||
|
|
||
| logDebug("Task " + taskId + "'s epoch is " + task.epoch) | ||
|
|
@@ -358,9 +361,7 @@ private[spark] class Executor( | |
| } else 0L | ||
|
|
||
| // If the task has been killed, let's fail it. | ||
| if (task.killed) { | ||
| throw new TaskKilledException | ||
| } | ||
| task.context.killTaskIfInterrupted() | ||
|
|
||
| val resultSer = env.serializer.newInstance() | ||
| val beforeSerialization = System.currentTimeMillis() | ||
|
|
@@ -426,15 +427,17 @@ private[spark] class Executor( | |
| setTaskFinishedAndClearInterruptStatus() | ||
| execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) | ||
|
|
||
| case _: TaskKilledException => | ||
| logInfo(s"Executor killed $taskName (TID $taskId)") | ||
| case t: TaskKilledException => | ||
| logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}") | ||
| setTaskFinishedAndClearInterruptStatus() | ||
| execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) | ||
| execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason))) | ||
|
|
||
| case _: InterruptedException if task.killed => | ||
| logInfo(s"Executor interrupted and killed $taskName (TID $taskId)") | ||
| case _: InterruptedException if task.reasonIfKilled.isDefined => | ||
| val killReason = task.reasonIfKilled.getOrElse("unknown reason") | ||
| logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason") | ||
| setTaskFinishedAndClearInterruptStatus() | ||
| execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) | ||
| execBackend.statusUpdate( | ||
| taskId, TaskState.KILLED, ser.serialize(TaskKilled(killReason))) | ||
|
|
||
| case CausedBy(cDE: CommitDeniedException) => | ||
| val reason = cDE.toTaskFailedReason | ||
|
|
@@ -512,7 +515,8 @@ private[spark] class Executor( | |
| */ | ||
| private class TaskReaper( | ||
| taskRunner: TaskRunner, | ||
| val interruptThread: Boolean) | ||
| val interruptThread: Boolean, | ||
| val reason: String) | ||
| extends Runnable { | ||
|
|
||
| private[this] val taskId: Long = taskRunner.taskId | ||
|
|
@@ -533,7 +537,7 @@ private[spark] class Executor( | |
| // Only attempt to kill the task once. If interruptThread = false then a second kill | ||
| // attempt would be a no-op and if interruptThread = true then it may not be safe or | ||
| // effective to interrupt multiple times: | ||
| taskRunner.kill(interruptThread = interruptThread) | ||
| taskRunner.kill(interruptThread = interruptThread, reason = reason) | ||
| // Monitor the killed task until it exits. The synchronization logic here is complicated | ||
| // because we don't want to synchronize on the taskRunner while possibly taking a thread | ||
| // dump, but we also need to be careful to avoid races between checking whether the task | ||
|
|
||
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 this was part of DeveloperApi, what is the impact of making this change ?
In JsonProtocol, in mocked user code ?
If it does introduce backward incompatible changes, is there a way to mitigate this ?
Perhaps make TaskKilled a class with apply/unapply/serde/toString (essentially all that case class provides) and case object with apply with default reason = null (and logged when used as deprecated) ?
Uh oh!
There was an error while loading. Please reload this page.
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 is unfortunately not backwards compatible. I've looked into this, and the issue seems to be that we are converting a case object into a case class. If
TaskKilledwas a case class to start with, compatibility might have been possible (e.g. implement equals() ignoring reason), but as is you would break scala match statements and possibly other things depending on the user code.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 is unfortunate, but looks like it cant be helped if we need this feature.
Probably something to keep in mind with future use of case objects !
Thx for clarifying.