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

    https://github.com/apache/spark/pull/5636#discussion_r35800587
  
    --- Diff: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
    @@ -473,6 +473,326 @@ 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
    +  }
    +
    +  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
    +    }
    +  }
    +
    +  /**
    +   * 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 stage abort to avoid endless retries.
    +   */
    +  test("Multiple consecutive stage failures should lead to task 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_STAGE_FAILURES) {
    +      // Complete all the tasks for the current attempt of stage 0 
successfully
    +      val stage0Attempt = taskSets.last
    +      checkStageId(0, attempt, stage0Attempt)
    +
    +      // Run stage 0
    +      complete(stage0Attempt, makeCompletions(stage0Attempt, 2))
    +
    +      // Now we should have a new taskSet, for a new attempt of stage 1.
    +      // We will have one fetch failure for this task set
    +      val stage1Attempt = taskSets.last
    +      checkStageId(1, attempt, stage1Attempt)
    +
    +      val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, 
42)}
    +
    +      // Run Stage 1, this time with a task failure
    +      complete(stage1Attempt,
    +        Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, 
"ignored"), null))
    +          ++ stage1Successes
    +      )
    +
    +      // 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_STAGE_FAILURES-1) {
    +        assert(scheduler.runningStages.nonEmpty)
    +        assert(!ended)
    +      } else {
    +        // Stage has been aborted and removed from running stages
    +        assertDataStructuresEmpty()
    +        sc.listenerBus.waitUntilEmpty(1000)
    +        assert(ended)
    +        assert(jobResult.isInstanceOf[JobFailed])
    +      }
    +    }
    +  }
    +
    +  /**
    +   * In this test we simulate a job failure where there are two failures 
in two different stages.
    +   * Specifically, stage0 fails twice, and then stage1 twice. 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_STAGE_FAILURES) {
    +      // Complete all the tasks for the current attempt of stage 0 
successfully
    +      val stage0Attempt = taskSets.last
    +      checkStageId(0, attempt, stage0Attempt)
    +      // Run stage 0
    +      complete(stage0Attempt, makeCompletions(stage0Attempt, 2))
    +
    +      if (attempt < Stage.MAX_STAGE_FAILURES/2) {
    +        // Now we should have a new taskSet, for a new attempt of stage 1.
    +        // We will have one fetch failure for this task set
    +        val stage1Attempt = taskSets.last
    +        checkStageId(1, attempt, stage1Attempt)
    +
    +        val stage1Successes =
    +          stage1Attempt.tasks.tail.map { _ => (Success, 
makeMapStatus("hostB", 1))}
    +
    +        // Run Stage 1, this time with a task failure
    +        complete(stage1Attempt,
    +          Seq((FetchFailed(makeBlockManagerId("hostA"),
    +              shuffleDepOne.shuffleId, 0, 0, "ignored"), null)
    +          ) ++ stage1Successes
    +        )
    +      } else {
    +        // Run stage 1
    +        val stage1Attempt = taskSets.last
    +        checkStageId(1, attempt, stage1Attempt)
    +        complete(stage1Attempt, makeCompletions(stage1Attempt, 1))
    +
    +        // Fail stage 2
    +        val stage2Attempt = taskSets.last
    +        checkStageId(2, attempt-Stage.MAX_STAGE_FAILURES/2, stage2Attempt)
    +        complete(stage2Attempt, Seq(
    +          (FetchFailed(makeBlockManagerId("hostA"),
    +            shuffleDepTwo.shuffleId, 0, 0, "ignored"), null)))
    +      }
    +
    +      // 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()
    +    }
    +
    +    // Complete all three stages succesfully
    +    val stage0Attempt4 = taskSets.last
    +    checkStageId(0, 4, stage0Attempt4)
    +    complete(stage0Attempt4, makeCompletions(stage0Attempt4, 2))
    +
    +    val stage1Attempt4 = taskSets.last
    +    checkStageId(1, 4, stage1Attempt4)
    +    complete(stage1Attempt4, makeCompletions(stage1Attempt4, 1))
    +
    +    println(taskSets.mkString(","))
    +
    +    val stage2Attempt = taskSets.last
    +    checkStageId(2, Stage.MAX_STAGE_FAILURES/2, stage2Attempt)
    +    complete(stage2Attempt, Seq((Success, 42)))
    +
    +    // The first success is from the success we append in stage 1, the 
second is the one we add here
    +    assert(results === Map(0 -> 42))
    +  }
    +
    +  /**
    +   * In this test we simulate a job failure where a stage may have many 
tasks, many of which fail.
    +   * We want to show that many fetch failures inside a single stage do not 
trigger an abort on
    +   * their own, but only when the stage fails enough times.
    --- End diff --
    
    can you reword slightly to make the attempts more clear : "... fetch 
failures inside a single stage attempt do not trigger an abort on their own, 
but only when there are enough failing stage attempts."


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to