You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Imran Rashid (JIRA)" <ji...@apache.org> on 2017/01/18 18:22:26 UTC

[jira] [Created] (SPARK-19276) FetchFailures can be hidden be user (or sql) exception handling

Imran Rashid created SPARK-19276:
------------------------------------

             Summary: FetchFailures can be hidden be user (or sql) exception handling
                 Key: SPARK-19276
                 URL: https://issues.apache.org/jira/browse/SPARK-19276
             Project: Spark
          Issue Type: Bug
          Components: Scheduler, Spark Core, SQL
    Affects Versions: 2.1.0
            Reporter: Imran Rashid
            Priority: Critical


The scheduler handles node failures by looking for a special {{FetchFailedException}} thrown by the shuffle block fetcher.  This is handled in {{Executor}} and then passed as a special msg back to the driver: https://github.com/apache/spark/blob/278fa1eb305220a85c816c948932d6af8fa619aa/core/src/main/scala/org/apache/spark/executor/Executor.scala#L403

However, user code exists in between the shuffle block fetcher and that catch block -- it could intercept the exception, wrap it with something else, and throw a different exception.  If that happens, spark treats it as an ordinary task failure, and retries the task, rather than regenerating the missing shuffle data.  The task eventually is retried 4 times, its doomed to fail each time, and the job is failed.

You might think that no user code should do that -- but even sparksql does it:
https://github.com/apache/spark/blob/278fa1eb305220a85c816c948932d6af8fa619aa/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala#L214

I think the right fix here is to also set a fetch failure status in the {{TaskContextImpl}}, so the executor can check that instead of just one exception.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org