Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16639#discussion_r103503390
  
    --- 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 --
    
    Thanks, I like that msg better.  I changed it slightly so the original 
exception is at the end, otherwise its hard to tell where the original 
exception ends and you are back to the error msg.  Here's what the new msg 
looks like from the test case now:
    
    ```
    17/02/27 16:33:43.953 Executor task launch worker for task 0 WARN Executor: 
TID 0 encountered a org.apache.spark.shuffle.FetchFailedException and failed, 
but the org.apache.spark.shuffle.FetchFailedException was hidden by another 
exception.  Spark is handling this like a fetch failure and ignoring the other 
exception: java.lang.RuntimeException: User Exception that hides the original 
exception
    ```
    
    You have a good point about the uncaught exception handler, I have added 
that back.  I wondered whether I should add those lines inside the `case t: 
Throwable if hasFetchFailure ` block, or make it a condition for the case 
itself `case t: Throwable if hasFetchFailure && !Utils.isFatalError(t)`.  I 
decided to make it part of the condition, since that is more like the old 
behavior, and a fetch failure that happens during an OOM may not be real.
    
    I also looked into adding a unit test for this handling -- it requires some 
refactoring, potentially more work than its worth, so I put it in a separate 
commit.
    
     I'd rather avoid changing the behavior for accumulators here.  
Accumulators have such weird semantics its not clear what they should do, we 
can fix that separately if we really want to.


---
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

Reply via email to