|
@@ -78,9 +78,13 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
|
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptKilledEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -982,7 +986,46 @@ public class TestTaskAttempt{
|
|
+ " Task attempt finish time is not the same ",
|
|
+ " Task attempt finish time is not the same ",
|
|
finishTime, Long.valueOf(taImpl.getFinishTime()));
|
|
finishTime, Long.valueOf(taImpl.getFinishTime()));
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ private void containerKillBeforeAssignment(boolean scheduleAttempt)
|
|
|
|
+ throws Exception {
|
|
|
|
+ MockEventHandler eventHandler = new MockEventHandler();
|
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(1, 2);
|
|
|
|
+ JobId jobId = MRBuilderUtils.newJobId(appId, 1);
|
|
|
|
+ TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
|
|
|
|
+
|
|
|
|
+ TaskAttemptImpl taImpl =
|
|
|
|
+ new MapTaskAttemptImpl(taskId, 1, eventHandler, mock(Path.class), 1,
|
|
|
|
+ mock(TaskSplitMetaInfo.class), new JobConf(),
|
|
|
|
+ mock(TaskAttemptListener.class), mock(Token.class),
|
|
|
|
+ new Credentials(), SystemClock.getInstance(),
|
|
|
|
+ mock(AppContext.class));
|
|
|
|
+ if (scheduleAttempt) {
|
|
|
|
+ taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
|
|
|
|
+ TaskAttemptEventType.TA_SCHEDULE));
|
|
|
|
+ }
|
|
|
|
+ taImpl.handle(new TaskAttemptKillEvent(taImpl.getID(),"", true));
|
|
|
|
+ assertEquals("Task attempt is not in KILLED state", taImpl.getState(),
|
|
|
|
+ TaskAttemptState.KILLED);
|
|
|
|
+ assertEquals("Task attempt's internal state is not KILLED",
|
|
|
|
+ taImpl.getInternalState(), TaskAttemptStateInternal.KILLED);
|
|
|
|
+ assertFalse("InternalError occurred", eventHandler.internalError);
|
|
|
|
+ TaskEvent event = eventHandler.lastTaskEvent;
|
|
|
|
+ assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType());
|
|
|
|
+ // In NEW state, new map attempt should not be rescheduled.
|
|
|
|
+ assertFalse(((TaskTAttemptKilledEvent)event).getRescheduleAttempt());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testContainerKillOnNew() throws Exception {
|
|
|
|
+ containerKillBeforeAssignment(false);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testContainerKillOnUnassigned() throws Exception {
|
|
|
|
+ containerKillBeforeAssignment(true);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testContainerKillAfterAssigned() throws Exception {
|
|
public void testContainerKillAfterAssigned() throws Exception {
|
|
ApplicationId appId = ApplicationId.newInstance(1, 2);
|
|
ApplicationId appId = ApplicationId.newInstance(1, 2);
|
|
@@ -1032,7 +1075,7 @@ public class TestTaskAttempt{
|
|
taImpl.getInternalState(), TaskAttemptStateInternal.ASSIGNED);
|
|
taImpl.getInternalState(), TaskAttemptStateInternal.ASSIGNED);
|
|
taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
TaskAttemptEventType.TA_KILL));
|
|
TaskAttemptEventType.TA_KILL));
|
|
- assertEquals("Task should be in KILLED state",
|
|
|
|
|
|
+ assertEquals("Task should be in KILL_CONTAINER_CLEANUP state",
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
taImpl.getInternalState());
|
|
taImpl.getInternalState());
|
|
}
|
|
}
|
|
@@ -1089,7 +1132,7 @@ public class TestTaskAttempt{
|
|
TaskAttemptEventType.TA_KILL));
|
|
TaskAttemptEventType.TA_KILL));
|
|
assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
eventHandler.internalError);
|
|
eventHandler.internalError);
|
|
- assertEquals("Task should be in KILLED state",
|
|
|
|
|
|
+ assertEquals("Task should be in KILL_CONTAINER_CLEANUP state",
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
taImpl.getInternalState());
|
|
taImpl.getInternalState());
|
|
}
|
|
}
|
|
@@ -1150,12 +1193,11 @@ public class TestTaskAttempt{
|
|
TaskAttemptEventType.TA_KILL));
|
|
TaskAttemptEventType.TA_KILL));
|
|
assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
eventHandler.internalError);
|
|
eventHandler.internalError);
|
|
- assertEquals("Task should be in KILLED state",
|
|
|
|
|
|
+ assertEquals("Task should be in KILL_CONTAINER_CLEANUP state",
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
taImpl.getInternalState());
|
|
taImpl.getInternalState());
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
@Test
|
|
@Test
|
|
public void testKillMapTaskWhileSuccessFinishing() throws Exception {
|
|
public void testKillMapTaskWhileSuccessFinishing() throws Exception {
|
|
MockEventHandler eventHandler = new MockEventHandler();
|
|
MockEventHandler eventHandler = new MockEventHandler();
|
|
@@ -1195,6 +1237,37 @@ public class TestTaskAttempt{
|
|
assertFalse("InternalError occurred", eventHandler.internalError);
|
|
assertFalse("InternalError occurred", eventHandler.internalError);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testKillMapTaskAfterSuccess() throws Exception {
|
|
|
|
+ MockEventHandler eventHandler = new MockEventHandler();
|
|
|
|
+ TaskAttemptImpl taImpl = createTaskAttemptImpl(eventHandler);
|
|
|
|
+
|
|
|
|
+ taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
|
|
|
|
+ TaskAttemptEventType.TA_DONE));
|
|
|
|
+
|
|
|
|
+ assertEquals("Task attempt is not in SUCCEEDED state", taImpl.getState(),
|
|
|
|
+ TaskAttemptState.SUCCEEDED);
|
|
|
|
+ assertEquals("Task attempt's internal state is not " +
|
|
|
|
+ "SUCCESS_FINISHING_CONTAINER", taImpl.getInternalState(),
|
|
|
|
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER);
|
|
|
|
+
|
|
|
|
+ taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
|
|
|
|
+ TaskAttemptEventType.TA_CONTAINER_CLEANED));
|
|
|
|
+ // Send a map task attempt kill event indicating next map attempt has to be
|
|
|
|
+ // reschedule
|
|
|
|
+ taImpl.handle(new TaskAttemptKillEvent(taImpl.getID(),"", true));
|
|
|
|
+ assertEquals("Task attempt is not in KILLED state", taImpl.getState(),
|
|
|
|
+ TaskAttemptState.KILLED);
|
|
|
|
+ assertEquals("Task attempt's internal state is not KILLED",
|
|
|
|
+ taImpl.getInternalState(), TaskAttemptStateInternal.KILLED);
|
|
|
|
+ assertFalse("InternalError occurred", eventHandler.internalError);
|
|
|
|
+ TaskEvent event = eventHandler.lastTaskEvent;
|
|
|
|
+ assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType());
|
|
|
|
+ // Send an attempt killed event to TaskImpl forwarding the same reschedule
|
|
|
|
+ // flag we received in task attempt kill event.
|
|
|
|
+ assertTrue(((TaskTAttemptKilledEvent)event).getRescheduleAttempt());
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testKillMapTaskWhileFailFinishing() throws Exception {
|
|
public void testKillMapTaskWhileFailFinishing() throws Exception {
|
|
MockEventHandler eventHandler = new MockEventHandler();
|
|
MockEventHandler eventHandler = new MockEventHandler();
|
|
@@ -1406,9 +1479,13 @@ public class TestTaskAttempt{
|
|
|
|
|
|
public static class MockEventHandler implements EventHandler {
|
|
public static class MockEventHandler implements EventHandler {
|
|
public boolean internalError;
|
|
public boolean internalError;
|
|
|
|
+ public TaskEvent lastTaskEvent;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void handle(Event event) {
|
|
public void handle(Event event) {
|
|
|
|
+ if (event instanceof TaskEvent) {
|
|
|
|
+ lastTaskEvent = (TaskEvent)event;
|
|
|
|
+ }
|
|
if (event instanceof JobEvent) {
|
|
if (event instanceof JobEvent) {
|
|
JobEvent je = ((JobEvent) event);
|
|
JobEvent je = ((JobEvent) event);
|
|
if (JobEventType.INTERNAL_ERROR == je.getType()) {
|
|
if (JobEventType.INTERNAL_ERROR == je.getType()) {
|