-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19276][CORE] Fetch Failure handling robust to user error handling #16639
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 9 commits
b93c37f
9635980
0a60aef
bbef893
730fd83
4494673
14f5125
08491c5
7840480
22da707
2a49705
84eae14
bee5621
ad47611
965506a
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 |
|---|---|---|
|
|
@@ -148,6 +148,8 @@ private[spark] class Executor( | |
|
|
||
| startDriverHeartbeater() | ||
|
|
||
| private[executor] def numRunningTasks: Int = runningTasks.size() | ||
|
|
||
| def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { | ||
| val tr = new TaskRunner(context, taskDescription) | ||
| runningTasks.put(taskDescription.taskId, tr) | ||
|
|
@@ -340,6 +342,14 @@ private[spark] class Executor( | |
| } | ||
| } | ||
| } | ||
| task.context.fetchFailed.foreach { fetchFailure => | ||
| // uh-oh. it appears the user code has caught the fetch-failure without throwing any | ||
| // other exceptions. Its *possible* this is what the user meant to do (though highly | ||
| // unlikely). So we will log an error and keep going. | ||
| logError(s"TID ${taskId} completed successfully though internally it encountered " + | ||
| s"unrecoverable fetch failures! Most likely this means user code is incorrectly " + | ||
| s"swallowing Spark's internal ${classOf[FetchFailedException]}", fetchFailure) | ||
| } | ||
| val taskFinish = System.currentTimeMillis() | ||
| val taskFinishCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { | ||
| threadMXBean.getCurrentThreadCpuTime | ||
|
|
@@ -400,8 +410,16 @@ private[spark] class Executor( | |
| execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) | ||
|
|
||
| } catch { | ||
| case ffe: FetchFailedException => | ||
| val reason = ffe.toTaskFailedReason | ||
| case t: Throwable if hasFetchFailure => | ||
| val reason = task.context.fetchFailed.get.toTaskFailedReason | ||
|
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. tiny nit: but does it make sense to store the taskFailedReason (rather than the actual exception) in the task context? |
||
| if (!t.isInstanceOf[FetchFailedException]) { | ||
| // there was a fetch failure in the task, but some user code wrapped that exception | ||
| // and threw something else. Regardless, we treat it as a fetch failure. | ||
| logWarning(s"TID ${taskId} encountered a ${classOf[FetchFailedException]} and " + | ||
| s"failed, but did not directly throw the ${classOf[FetchFailedException]}. " + | ||
| s"Spark is still handling the fetch failure, but these exceptions should not be " + | ||
| s"intercepted by user code.") | ||
|
||
| } | ||
| setTaskFinishedAndClearInterruptStatus() | ||
| execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(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. nit: Probably log a similar message as above ?
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. do you mean the msg I added about "TID ${taskId} completed successfully though internally it encountered unrecoverable fetch failures!"? I wouldn't think we'd want to log anything special here. I'm trying to make this a "normal" code path. The user is allowed to allowed to do this. (sparksql already does.) we could log a warning, but then this change should be accompanied by auditing the code and making sure we never do this ourselves.
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. Yes, something along those lines ... |
||
|
|
||
|
|
@@ -460,6 +478,10 @@ private[spark] class Executor( | |
| runningTasks.remove(taskId) | ||
| } | ||
| } | ||
|
|
||
| private def hasFetchFailure: Boolean = { | ||
| task != null && task.context != null && task.context.fetchFailed.isDefined | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
|
|
||
| package org.apache.spark.shuffle | ||
|
|
||
| import org.apache.spark.{FetchFailed, TaskFailedReason} | ||
| import org.apache.spark.{FetchFailed, TaskContext, TaskFailedReason} | ||
| import org.apache.spark.storage.BlockManagerId | ||
| import org.apache.spark.util.Utils | ||
|
|
||
|
|
@@ -26,6 +26,11 @@ import org.apache.spark.util.Utils | |
| * back to DAGScheduler (through TaskEndReason) so we'd resubmit the previous stage. | ||
| * | ||
| * Note that bmAddress can be null. | ||
| * | ||
| * To prevent user code from hiding this fetch failure, in the constructor we call | ||
| * [[TaskContext.setFetchFailed()]]. This means that you *must* throw this exception immediately | ||
| * after creating it -- you cannot create it, check some condition, and then decide to ignore it | ||
| * (or risk triggering any other exceptions). See SPARK-19276. | ||
| */ | ||
| private[spark] class FetchFailedException( | ||
| bmAddress: BlockManagerId, | ||
|
|
@@ -45,6 +50,12 @@ private[spark] class FetchFailedException( | |
| this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) | ||
| } | ||
|
|
||
| // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code | ||
| // which intercepts this exception (possibly wrapping it), the Executor can still tell there was | ||
| // a fetch failure, and send the correct error msg back to the driver. The TaskContext won't be | ||
| // defined if this is run on the driver (just in test cases) -- we can safely ignore then. | ||
|
||
| Option(TaskContext.get()).map(_.setFetchFailed(this)) | ||
|
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. Since creation of an Exception does not necessarily mean it should get thrown - we must explicitly add this expectation to the documentation/contract of FetchFailedException constructor - indicating that we expect it to be created only for it to be thrown immediately.
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. yes, good point. I added to the docs, does it look OK? I also considered making the call to |
||
|
|
||
| def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, | ||
| Utils.exceptionString(this)) | ||
| } | ||
|
|
||
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.
Can the above case be eliminated with the addition of this one?
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.
yeah, as this is now, you could eliminate this -- I left it separate for now just to highlight that we can differentiate two special cases, which we could handle in a few different ways.
It seems clear in this case, we should fail the task with a FetchFailure. But do we also want to log an error or something indicating bad user code? Kinda minor, but might be a good idea. (Suggested by @mridulm above as well, I think.)
1a) or the FetchFailed isn't part of the thrown exception at all.
As I mentioned in my response to your other question, I'd like to consider this exactly the same as (1).
Should we succeed the task, or fail it? I don't really know how this would happen. It seems really unlikely the user meant to do this. But then again, maybe the user did? I chose to just log an error but still succeed the task. (@markhamstra commented about this on the jira as well.)
its pretty easy to change the code for whatever the desired behavior is, just waiting for a clear decision.
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 agree with Mridul's comment on (1) (that it would be nice to log a warning in this case) and your assessment of 2. To handle (1), you could have just this one case, and then log a warning if !t.isInstanceOf[FetchFailedException]