Github user kayousterhout commented on a diff in the pull request: https://github.com/apache/spark/pull/16639#discussion_r103289664 --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala --- @@ -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 + 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.") --- End diff -- I agree with @mridulm that it looks like these lines (473-475 below) need to be added here: ``` if (Utils.isFatalError(t)) { SparkUncaughtExceptionHandler.uncaughtException(t) } ``` I'm less sure about the accumulator updates. It looks like the old code doesn't report accumulators for fetch failed exceptions, but it's not clear to me why we'd report them for some kinds of exceptions but not others. The simplest thing to do seems to be the current approach (since it roughly maintains the old behavior of not updating accumulators for fetch failures) but I don't have a good sense for why this is or is not correct.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org