MAPREDUCE-6815. Fix flaky TestKill.testKillTask(). Contributed by Haibo Chen

This commit is contained in:
Jason Lowe 2016-12-02 17:22:11 +00:00
parent f304ccae3c
commit 0cfd7ad21f
2 changed files with 2 additions and 1 deletions

View File

@ -259,6 +259,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
// d. TA processes TA_KILL event and sends T_ATTEMPT_KILLED to the task.
.addTransition(TaskStateInternal.KILLED, TaskStateInternal.KILLED,
EnumSet.of(TaskEventType.T_KILL,
TaskEventType.T_SCHEDULE,
TaskEventType.T_ATTEMPT_KILLED,
TaskEventType.T_ADD_SPEC_ATTEMPT))

View File

@ -105,7 +105,7 @@ public class TestKill {
Job job = app.submit(new Configuration());
//wait and vailidate for Job to become RUNNING
app.waitForState(job, JobState.RUNNING);
app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING);
Map<TaskId,Task> tasks = job.getTasks();
Assert.assertEquals("No of tasks is not correct", 2,
tasks.size());