Skip to content
Closed
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Refine the testcase
  • Loading branch information
suyanNone committed Jul 21, 2015
commit e1e0b66abc7430e88b8a58ec0c81a8d500bf6caa
Original file line number Diff line number Diff line change
Expand Up @@ -748,48 +748,77 @@ class DAGSchedulerSuite
val reduceRdd = new MyRDD(sc, 1, List(shuffleDep))
submit(reduceRdd, Array(0))

// things start out smoothly, stage 0 completes with no issues
complete(taskSets(0), Seq(
(Success, makeMapStatus("hostB", shuffleMapRdd.partitions.size)),
(Success, makeMapStatus("hostB", shuffleMapRdd.partitions.size)),
(Success, makeMapStatus("hostA", shuffleMapRdd.partitions.size))
))

complete(taskSets(1), Seq(
(Success, makeMapStatus("hostA", reduceRdd.partitions.size)),
(Success, makeMapStatus("hostB", reduceRdd.partitions.size))
))
// then one executor dies, and a task fails in stage 1
runEvent(ExecutorLost("exec-hostA"))
// Resubmit already succcessd in hostA task
runEvent(CompletionEvent(taskSets(1).tasks(0), Resubmitted,
null, null, createFakeTaskInfo(), null))

// Cause mapOutTracker remove hostA outputs for taskset(0).
// Task that resubmitted will fetch matadata failed.
runEvent(CompletionEvent(taskSets(1).tasks(0),
FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"),
null, null, createFakeTaskInfo(), null))

// FetchFailed cause resubmit failed Stages.
scheduler.resubmitFailedStages()

runEvent(CompletionEvent(taskSets(0).tasks(2), Success,
// so we resubmit stage 0, which completes happily
Thread.sleep(1000)
val stage0Resubmit = taskSets(2)
assert(stage0Resubmit.stageId == 0)
assert(stage0Resubmit.attempt === 1)
val task = stage0Resubmit.tasks(0)
assert(task.partitionId === 2)
runEvent(CompletionEvent(task, Success,
makeMapStatus("hostC", shuffleMapRdd.partitions.size), null, createFakeTaskInfo(), null))

try {
mapOutputTracker.getServerStatuses(0, 2)
} catch {
case e: MetadataFetchFailedException => fail("Should not throw metadataFetchFailedException")
}

runEvent(CompletionEvent(taskSets(1).tasks(0), Success,
// now here is where things get tricky : we will now have a task set representing
// the second attempt for stage 1, but we *also* have some tasks for the first attempt for
// stage 1 still going
val stage1Resubmit = taskSets(3)
assert(stage1Resubmit.stageId == 1)
assert(stage1Resubmit.attempt === 1)
assert(stage1Resubmit.tasks.length === 3)

// we'll have some tasks finish from the first attempt, and some finish from the second attempt,
// so that we actually have all stage outputs, though no attempt has completed all its
// tasks
runEvent(CompletionEvent(taskSets(3).tasks(0), Success,
makeMapStatus("hostC", reduceRdd.partitions.size), null, createFakeTaskInfo(), null))
runEvent(CompletionEvent(taskSets(1).tasks(2), Success,
runEvent(CompletionEvent(taskSets(3).tasks(1), Success,
makeMapStatus("hostC", reduceRdd.partitions.size), null, createFakeTaskInfo(), null))
// late task finish from the first attempt
runEvent(CompletionEvent(taskSets(1).tasks(2), Success,
makeMapStatus("hostB", reduceRdd.partitions.size), null, createFakeTaskInfo(), null))

// What should happen now is that we submit stage 2. However, we might not see an error
// b/c of DAGScheduler's error handling (it tends to swallow errors and just log them). But
// we can check some conditions.
// Note that the really important thing here is not so much that we submit stage 2 *immediately*
// but that we don't end up with some error from these interleaved completions. It would also
// be OK (though sub-optimal) if stage 2 simply waited until the resubmission of stage 1 had
// all its tasks complete

// check that we have all the map output for stage 0 (it should have been there even before
// the last round of completions from stage 1, but just to double check it hasn't been messed
// up)
(0 until 3).foreach { reduceIdx =>
val arr = mapOutputTracker.getServerStatuses(0, reduceIdx)
assert(arr != null)
assert(arr.nonEmpty)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getServerStatuses has been removed in master -- I guess both of these should be

val statuses = mapOutputTracker.getMapSizesByExecutorId(0, reduceIdx)
assert(statuses != null)
assert(statuses.nonEmpty)

The new code will now throw an exception if we're missing the map output data, but I feel like its probably still good to leave those asserts in.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

may the below code will be more better?

try {
        mapOutputTracker.getMapSizesByExecutorId(0, reduceIdx)
      } catch {
        case e: Exception => fail("")
      }

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we don't use try / case e: Exception => fail("") to fail tests when there is an exception -- we just let the exception fail the test directly. You get more info in the stack trace that way. So I think its better to just leave it bare.

You could just put in a comment explaining what the point is:

// this would throw an exception if the map status hadn't been registered
mapOutputTracker.getMapSizesByExecutorId(0, reduceIdx)

I still slightly prefer leaving the asserts in there. Yes, they are kinda pointless with the current behavior of getMapSizesByExecutorId -- but I'd just like to be a bit more defensive, in case that behavior changes in the future. (eg., maybe some future refactoring makes them stop throwing exceptions for some reason).

Maybe to be very clear, you could include the asserts and more comments:

// this would throw an exception if the map status hadn't been registered
val statuses = mapOutputTracker.getMapSizesByExecutorId(0, reduceIdx)
// really we should have already thrown an exception rather than fail either of these
// asserts, but just to be extra defensive let's double check the statuses are OK
assert(statuses != null)
assert(statuses.nonEmpty)

This is pretty minor, though, I don't feel strongly about it.

}

// and check we have all the map output for stage 1
(0 until 1).foreach { reduceIdx =>
val arr = mapOutputTracker.getServerStatuses(1,reduceIdx)
assert(arr != null)
assert(arr.nonEmpty)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same here

}

val stage = scheduler.stageIdToStage(taskSets(1).stageId).asInstanceOf[ShuffleMapStage]
assert(stage.attemptId === 2)
assert(stage.isAvailable)
assert(stage.pendingTasks.isEmpty)
// and check that stage 2 has been submitted
assert(taskSets.size == 5)
val stage2TaskSet = taskSets(4)
assert(stage2TaskSet.stageId == 2)
assert(stage2TaskSet.attempt == 0)
}

/**
Expand Down