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

Josh Rosen edited comment on SPARK-20178 at 5/22/17 8:54 PM:
-------------------------------------------------------------

Sure, let me clarify:

When a FetchFailure occurs, the DAGScheduler receives a fetch failure message 
of the form {{FetchFailed(bmAddress, shuffleId, mapId, reduceId, 
failureMessage)}}. As of today's Spark master branch, the DAGScheduler handles 
this failure by marking that individual output as unavailable and by marking 
all outputs on that executor as unavailable (see 
https://github.com/apache/spark/blob/9b09101938399a3490c3c9bde9e5f07031140fdf/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1339).
 

As a shorthand, let's call the current behavior {{remove(shuffleId, mapId)}} 
followed by {{remove(blockManagerId)}}. My proposal was to replace this by 
{{remove(shuffleId, blockManagerId)}} to remove all outputs from the 
fetch-failed shuffle on that block manager.

{quote}
I think this is basically what you are proposing except waiting for a 
configurable amount of failures rather then doing it immediately. Thoughts?
{quote}

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.


was (Author: joshrosen):
Sure, let me clarify:

When a FetchFailure occurs, the DAGScheduler receives a fetch failure message 
of the form {{FetchFailed(bmAddress, shuffleId, mapId, reduceId, 
failureMessage)}}. As of today's Spark master branch, the DAGScheduler handles 
this failure by marking that individual output as unavailable ( and by marking 
all outputs on that executor as unavailable (see 
https://github.com/apache/spark/blob/9b09101938399a3490c3c9bde9e5f07031140fdf/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1339).
 

As a shorthand, let's call the current behavior {{remove(shuffleId, mapId)}} 
followed by {{remove(blockManagerId)}}. My proposal was to replace this by 
{{remove(shuffleId, blockManagerId)}} to remove all outputs from the 
fetch-failed shuffle on that block manager.

{quote}
I think this is basically what you are proposing except waiting for a 
configurable amount of failures rather then doing it immediately. Thoughts?
{quote}

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.

> 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: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to