spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Thomas Graves (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-20178) Improve Scheduler fetch failures
Date Tue, 23 May 2017 13:42:05 GMT

    [ https://issues.apache.org/jira/browse/SPARK-20178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16021199#comment-16021199
] 

Thomas Graves commented on SPARK-20178:
---------------------------------------

| My understanding of today's code is that a single FetchFailed task will trigger a stage
failure and parent stage retry and that the task which experienced the fetch failure will
not be retried within the same task set that scheduled it. I'm basing this off the comment
at https://github.com/apache/spark/blob/9b09101938399a3490c3c9bde9e5f07031140fdf/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L77
and the code at https://github.com/apache/spark/blob/9b09101938399a3490c3c9bde9e5f07031140fdf/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L770
where the TSM prevents re-attempts of FetchFailed tasks.

That is correct but that doesn't mean we can't track the fetch failures on a host across stages.
 You may or may not get multiple fetch failures in the first stage before it is aborted (very
timing dependent) so you are correct that you can't rely on that. But if you track those across
stage attempts and if the max is set to 2 or 3 then it will clear the entire host before the
4 default stage failures.   This might give us a little more confidence its a hard failure
vs a transient failure.   But that does take extra tracking and right now I don't have a good
measure of metrics to tell me how many of different kinds of failures.  So to get the robustness
for now I'm fine with just invalidating it immediately and see how that works.


> Improve Scheduler fetch failures
> --------------------------------
>
>                 Key: SPARK-20178
>                 URL: https://issues.apache.org/jira/browse/SPARK-20178
>             Project: Spark
>          Issue Type: Epic
>          Components: Scheduler
>    Affects Versions: 2.1.0
>            Reporter: Thomas Graves
>
> We have been having a lot of discussions around improving the handling of fetch failures.
 There are 4 jira currently related to this.  
> We should try to get a list of things we want to improve and come up with one cohesive
design.
> SPARK-20163,  SPARK-20091,  SPARK-14649 , and SPARK-19753
> I will put my initial thoughts in a follow on comment.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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


Mime
View raw message