소스 검색

MAPREDUCE-6815. Fix flaky TestKill.testKillTask(). Contributed by Haibo Chen
(cherry picked from commit 0cfd7ad21f4457513ed3416e5d77f3123bfe9da0)

Jason Lowe 8 년 전
부모
커밋
69f66424b9

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

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

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java

@@ -100,7 +100,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());