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

    https://github.com/apache/spark/pull/5636#discussion_r38605474
  
    --- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
    @@ -473,6 +473,283 @@ class DAGSchedulerSuite
         assertDataStructuresEmpty()
       }
     
    +
    +  // Helper function to validate state when creating tests for task 
failures
    +  def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) {
    +    assert(stageAttempt.stageId === stageId)
    +    assert(stageAttempt.stageAttemptId == attempt)
    +  }
    +
    +  def makeCompletions(stageAttempt: TaskSet, reduceParts: Int): 
Seq[(Success.type, MapStatus)] = {
    +    stageAttempt.tasks.zipWithIndex.map { case (task, idx) =>
    +      (Success, makeMapStatus("host" + ('A' + idx).toChar, reduceParts))
    +    }.toSeq
    +  }
    +
    +  // Helper functions to extract commonly used code in Fetch Failure test 
cases
    +  def setupStageAbortTest(sc: SparkContext) {
    +    sc.listenerBus.addListener(new EndListener())
    +    ended = false
    +    jobResult = null
    +  }
    +
    +  // Create a new Listener to confirm that the listenerBus sees the JobEnd 
message
    +  // when we abort the stage. This message will also be consumed by the 
EventLoggingListener
    +  // so this will propagate up to the user.
    +  var ended = false
    +  var jobResult : JobResult = null
    +
    +  class EndListener extends SparkListener {
    +    override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
    +      jobResult = jobEnd.jobResult
    +      ended = true
    +    }
    +  }
    +
    +  /**
    +   * Common code to get the next stage attempt, confirm it's the one we 
expect, and complete it
    +   * succesfullly.
    +   *
    +   * @param stageId - The current stageId
    +   * @param attemptIdx - The current attempt count
    +   * @param numShufflePartitions - The number of partitions in the next 
stage
    +   */
    +  def completeNextShuffleMapSuccesfully(
    +      stageId: Int,
    +      attemptIdx: Int,
    +      numShufflePartitions: Int): Unit = {
    +    val stageAttempt = taskSets.last
    +    checkStageId(stageId, attemptIdx, stageAttempt)
    +    complete(stageAttempt, makeCompletions(stageAttempt, 
numShufflePartitions))
    +  }
    +
    +  /**
    +   * Common code to get the next stage attempt, confirm it's the one we 
expect, and complete it
    +   * with all FetchFailure.
    +   *
    +   * @param stageId - The current stageId
    +   * @param attemptIdx - The current attempt count
    +   * @param shuffleDep - The shuffle dependency of the stage with a fetch 
failure
    +   */
    +  def completeNextStageWithFetchFailure(
    +      stageId: Int,
    +      attemptIdx: Int,
    +      shuffleDep: ShuffleDependency[_, _, _]): Unit = {
    +    val stageAttempt = taskSets.last
    +    checkStageId(stageId, attemptIdx, stageAttempt)
    +
    +    complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map{ case 
(task, idx) =>
    +      (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, 
idx, "ignored"), null)
    +    }.toSeq)
    +  }
    +
    +  /**
    +   * Common code to get the next result stage attempt, confirm it's the 
one we expect, and
    +   * complete it with a success where we return 42.
    +   *
    +   * @param stageId - The current stageId
    +   * @param attemptIdx - The current attempt count
    +   */
    +  def completeNextResultStageWithSuccess (stageId: Int, attemptIdx: Int): 
Unit = {
    +    val stageAttempt = taskSets.last
    +    checkStageId(stageId, attemptIdx, stageAttempt)
    +    assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage])
    +    complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map(_ => 
(Success, 42)).toSeq)
    +  }
    +
    +  /**
    +   * In this test, we simulate a job where many tasks in the same stage 
fail. We want to show
    +   * that many fetch failures inside a single stage attempt do not trigger 
an abort
    +   * on their own, but only when there are enough failing stage attempts.
    +   */
    +  test("Multiple tasks w/ fetch failures in same stage attempt should not 
abort the stage.") {
    +    setupStageAbortTest(sc)
    +
    +    val parts = 8
    +    val shuffleMapRdd = new MyRDD(sc, parts, Nil)
    +    val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
    +    val shuffleId = shuffleDep.shuffleId
    +    val reduceRdd = new MyRDD(sc, parts, List(shuffleDep))
    +    submit(reduceRdd, (0 until parts).toArray)
    +
    +    completeNextShuffleMapSuccesfully(0, 0, numShufflePartitions = parts)
    +
    +    completeNextStageWithFetchFailure(1, 0, shuffleDep)
    +
    +    // Resubmit and confirm that now all is well
    +    scheduler.resubmitFailedStages()
    +
    +    assert(scheduler.runningStages.nonEmpty)
    +    assert(!ended)
    +
    +    // Complete stage 0 and then stage 1 with a "42"
    +    completeNextShuffleMapSuccesfully(0, 1, numShufflePartitions = parts)
    +    completeNextResultStageWithSuccess(1, 1)
    +
    +    // Confirm job finished succesfully
    +    sc.listenerBus.waitUntilEmpty(1000)
    +    assert(ended === true)
    +    assert(results === (0 until parts).map{idx => idx -> 42}.toMap)
    +    assertDataStructuresEmpty()
    +  }
    +
    +  /**
    +   * In this test we simulate a job failure where the first stage 
completes successfully and
    +   * the second stage fails due to a fetch failure. Multiple successive 
fetch failures of a stage
    +   * trigger an overall job abort to avoid endless retries.
    +   */
    +  test("Multiple consecutive stage failures should lead to job being 
aborted.") {
    +    setupStageAbortTest(sc)
    +
    +    val shuffleMapRdd = new MyRDD(sc, 2, Nil)
    +    val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
    +    val shuffleId = shuffleDep.shuffleId
    +    val reduceRdd = new MyRDD(sc, 2, List(shuffleDep))
    +    submit(reduceRdd, Array(0, 1))
    +
    +    for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES) {
    +      // Complete all the tasks for the current attempt of stage 0 
successfully
    +      completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 
2)
    +
    +      // Now we should have a new taskSet, for a new attempt of stage 1.
    +      // Fail all these tasks with FetchFailure
    +      completeNextStageWithFetchFailure(1, attempt, shuffleDep)
    +
    +      // this will (potentially) trigger a resubmission of stage 0, since 
we've lost some of its
    +      // map output, for the next iteration through the loop
    +      scheduler.resubmitFailedStages()
    +
    +      if (attempt < Stage.MAX_CONSECUTIVE_FAILURES-1) {
    +        assert(scheduler.runningStages.nonEmpty)
    +        assert(!ended)
    +      } else {
    +        // Stage should have been aborted and removed from running stages
    +        assertDataStructuresEmpty()
    +        sc.listenerBus.waitUntilEmpty(1000)
    +        assert(ended)
    +        jobResult match {
    +          case JobFailed(reason) =>
    +            assert(reason.getMessage.contains("ResultStage 1 () has failed 
the maximum"))
    +          case other => fail(s"expected JobFailed, not $other")
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * In this test, we create a job with two consecutive shuffles, and 
simulate 2 failures for each
    +   * shuffle fetch. In total In total, the job has had four failures 
overall but not four failures
    +   * for a particular stage, and as such should not be aborted.
    +   */
    +  test("Failures in different stages should not trigger an overall abort") 
{
    +    setupStageAbortTest(sc)
    +
    +    val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache()
    +    val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
    +    val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)).cache()
    +    val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null)
    +    val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo))
    +    submit(finalRdd, Array(0))
    +
    +    // In the first two iterations, Stage 0 succeeds and stage 1 fails. In 
the next two iterations,
    +    // stage 2 fails.
    +    for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES) {
    +      // Complete all the tasks for the current attempt of stage 0 
successfully
    +      completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 
2)
    +
    +      if (attempt < Stage.MAX_CONSECUTIVE_FAILURES / 2) {
    +        // Now we should have a new taskSet, for a new attempt of stage 1.
    +        // Fail all these tasks with FetchFailure
    +        completeNextStageWithFetchFailure(1, attempt, shuffleDepOne)
    +      } else {
    +        completeNextShuffleMapSuccesfully(1, attempt, numShufflePartitions 
= 1)
    +
    +        // Fail stage 2
    +        completeNextStageWithFetchFailure(2, attempt - 
Stage.MAX_CONSECUTIVE_FAILURES / 2,
    +          shuffleDepTwo)
    +      }
    +
    +      // this will (potentially) trigger a resubmission of stage 0, since 
we've lost some of its
    +      // map output, for the next iteration through the loop
    +      scheduler.resubmitFailedStages()
    +    }
    +
    +    completeNextShuffleMapSuccesfully(0, 4, numShufflePartitions = 2)
    +    completeNextShuffleMapSuccesfully(1, 4, numShufflePartitions = 1)
    +
    +    // Succeed stage2 with a "42"
    +    completeNextResultStageWithSuccess(2, Stage.MAX_CONSECUTIVE_FAILURES/2)
    +
    +    assert(results === Map(0 -> 42))
    +    assertDataStructuresEmpty()
    +  }
    +
    +  /**
    +   * In this test we demonstrate that only consecutive failures trigger a 
stage abort. A stage may
    +   * fail multiple times, succeed, then fail a few more times (because its 
run again by downstream
    +   * dependencies). The total number of failed attempts for one stage will 
go over the limit,
    +   * but that doesn't matter, since they have successes in the middle.
    +   */
    +  test("Non-consecutive stage failures don't trigger abort") {
    +    setupStageAbortTest(sc)
    +
    +    val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache()
    +    val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
    +    val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)).cache()
    +    val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null)
    +    val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo))
    +    submit(finalRdd, Array(0))
    +
    +    for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES-1) {
    +      // Make each task in stage 0 success
    +      completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 
2)
    +
    +      // Now we should have a new taskSet, for a new attempt of stage 1.
    +      // Fail these tasks with FetchFailure
    +      completeNextStageWithFetchFailure(1, attempt, shuffleDepOne)
    +
    +      scheduler.resubmitFailedStages()
    +
    +      // Confirm we have not yet aborted
    +      assert(scheduler.runningStages.nonEmpty)
    +      assert(!ended)
    +    }
    +
    +    // Rerun stage 0 and 1
    +    completeNextShuffleMapSuccesfully(0, 3, numShufflePartitions = 2)
    +    completeNextShuffleMapSuccesfully(1, 3, numShufflePartitions = 1)
    +
    +    // Fail stage 2
    +    completeNextStageWithFetchFailure(2, 0, shuffleDepTwo)
    +
    +    scheduler.resubmitFailedStages()
    +    // Rerun stage 0
    +    completeNextShuffleMapSuccesfully(0, 4, numShufflePartitions = 2)
    --- End diff --
    
    @andrewor14 I don't think there is an easy way to just keep re-running 
stage 1.  After you've completed it successfully, your only way to trigger it 
again is to have a downstream stage with a fetch failure.  We can't just submit 
another job which has the same dependency, b/c it will actually be a different 
"skipped" stage initially (which now that I think about it, diminishes the need 
for this consecutive failure check, but still seems good, especially if we get 
rid of skipped stages).  So thats why we have to fail stage 2.
    
    We don't *have* to retry stage 0, but its really the natural way stage 
retry works when you have multiple consecutive shuffle map stages, you end up 
with a cascade of stage retries.  Stage 2 has a fetch failure when reading from 
node X, so it says, I've lost all my input from stage 1 that used to live on 
node X, so stage 1 had better re-run to regenerate that input.  But then when 
stage 1 is about to run, it says, wait a minute, I've lost *my* input from 
stage 0 that used to live on node X.  So it tells stage 0 to regenerate that 
input.
    
    We could avoid this situation by carefully putting all the map output from 
stage 0 on some set of nodes, and generating the map output for stage 1 on at 
least one additional node, and making sure the fetch failure in stage 2 is on 
the node which has no shuffle map output for stage 1.  But I think that would 
both make the test setup a lot more complicated, and also avoid the more 
natural cascading failures that are typical with fetch failures.  (In fact, 
this test doesn't exactly *choose* to return stage 0, that is just what is 
scheduled so it has to be done.)
    
    I agree this is complicated, but I'm not sure what to add.  Maybe a comment 
on why we have to rerun stage 0 after the failure of stage 2: because we've 
lost all the map output on the lost host, that includes the input for stage 1, 
which has to be regenerated by stage 0 ?


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