Github user markhamstra commented on the issue:

    https://github.com/apache/spark/pull/15213
  
    The fix is logically correct; however, the prior code is needlessly complex 
and not as easy to understand as it should be, and the proposed fix doesn't 
improve on that.  I'd like to take the opportunity to make the code easier to 
understand and maintain.  Something like this:
    ```scala
              // It is likely that we receive multiple FetchFailed for a single 
stage (because we have
              // multiple tasks running concurrently on different executors). 
In that case, it is
              // possible the fetch failure has already been handled by the 
scheduler.
              if (runningStages.contains(failedStage)) {
                logInfo(s"Marking $failedStage (${failedStage.name}) as failed 
" +
                  s"due to a fetch failure from $mapStage (${mapStage.name})")
                markStageAsFinished(failedStage, Some(failureMessage))
              } else {
                logDebug(s"Received fetch failure from $task, but its from 
$failedStage which is no " +
                  s"longer running")
              }
    
              val shouldAbortStage =
                failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId) ||
                disallowStageRetryForTest
    
              if (shouldAbortStage) {
                val abortMessage = if (disallowStageRetryForTest) {
                  "Fetch failure will not retry stage due to testing config"
                } else {
                  s"$failedStage (${failedStage.name}) " +
                  s"has failed the maximum allowable number of " +
                  s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " +
                  s"Most recent failure reason: $failureMessage"
                }
                abortStage(failedStage, abortMessage, None)
              } else { // update failedStages and make sure a 
ResubmitFailedStages event is enqueued
                // TODO: Cancel running tasks in the failed stage -- cf. 
SPARK-17064
                val noResubmitEnqueued = failedStages.isEmpty
                failedStages += failedStage
                failedStages += mapStage
                if (noResubmitEnqueued) {
                  // If failedStages is not empty, then a previous FetchFailed 
already went through
                  // this block of code and queued up a ResubmitFailedStages 
event that has not yet
                  // run.  We, therefore, only need to queue up a new 
ResubmitFailedStages event when
                  // failedStages.isEmpty.
                  logInfo(
                    s"Resubmitting $mapStage (${mapStage.name}) and " +
                    s"$failedStage (${failedStage.name}) due to fetch failure"
                  )
                  messageScheduler.schedule(
                    new Runnable {
                      override def run(): Unit = 
eventProcessLoop.post(ResubmitFailedStages)
                    },
                    DAGScheduler.RESUBMIT_TIMEOUT,
                    TimeUnit.MILLISECONDS
                  )
                }
              }
    
              // Mark the map whose fetch failed as broken in the map stage
    ```
    
    This should be equivalent to what you have, @scwf, with the exception that 
`fetchFailedAttemptIds.add(stageAttemptId)` is done even when 
`disallowStageRetryForTest` is `true` -- which seems like a better idea to me.
    
    Also available here:
    
https://github.com/markhamstra/spark/commit/368f82d9789ec04565af835e7cb80d1cdb0ccf0c
    
    @squito 


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to