|
@@ -18,22 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.mapreduce.v2.app.job.impl;
|
|
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertFalse;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.mockito.Mockito.mock;
|
|
|
-import static org.mockito.Mockito.spy;
|
|
|
-import static org.mockito.Mockito.times;
|
|
|
-import static org.mockito.Mockito.verify;
|
|
|
-import static org.mockito.Mockito.when;
|
|
|
-
|
|
|
-import java.io.IOException;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.Iterator;
|
|
|
-import java.util.Map;
|
|
|
-
|
|
|
-import org.junit.Assert;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -71,6 +55,10 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunched
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
|
|
|
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.TaskAttemptKillEvent;
|
|
|
+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.util.MRBuilderUtils;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -87,9 +75,25 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.util.Clock;
|
|
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.ArgumentCaptor;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.Iterator;
|
|
|
+import java.util.Map;
|
|
|
+
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.spy;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
+
|
|
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
|
|
public class TestTaskAttempt{
|
|
|
|
|
@@ -944,7 +948,46 @@ public class TestTaskAttempt{
|
|
|
+ " Task attempt finish time is not the same ",
|
|
|
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(), new SystemClock(),
|
|
|
+ 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
|
|
|
public void testContainerKillAfterAssigned() throws Exception {
|
|
|
ApplicationId appId = ApplicationId.newInstance(1, 2);
|
|
@@ -994,7 +1037,7 @@ public class TestTaskAttempt{
|
|
|
taImpl.getInternalState(), TaskAttemptStateInternal.ASSIGNED);
|
|
|
taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
|
TaskAttemptEventType.TA_KILL));
|
|
|
- assertEquals("Task should be in KILLED state",
|
|
|
+ assertEquals("Task should be in KILL_CONTAINER_CLEANUP state",
|
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
taImpl.getInternalState());
|
|
|
}
|
|
@@ -1051,7 +1094,7 @@ public class TestTaskAttempt{
|
|
|
TaskAttemptEventType.TA_KILL));
|
|
|
assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
|
eventHandler.internalError);
|
|
|
- assertEquals("Task should be in KILLED state",
|
|
|
+ assertEquals("Task should be in KILL_CONTAINER_CLEANUP state",
|
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
taImpl.getInternalState());
|
|
|
}
|
|
@@ -1112,16 +1155,96 @@ public class TestTaskAttempt{
|
|
|
TaskAttemptEventType.TA_KILL));
|
|
|
assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
|
eventHandler.internalError);
|
|
|
- assertEquals("Task should be in KILLED state",
|
|
|
+ assertEquals("Task should be in KILL_CONTAINER_CLEANUP state",
|
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
taImpl.getInternalState());
|
|
|
}
|
|
|
|
|
|
+ @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.getInternalState(),
|
|
|
+ TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP);
|
|
|
+
|
|
|
+ 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());
|
|
|
+ }
|
|
|
+
|
|
|
+ private TaskAttemptImpl createTaskAttemptImpl(
|
|
|
+ MockEventHandler eventHandler) {
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(1, 2);
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
+ ApplicationAttemptId.newInstance(appId, 0);
|
|
|
+ JobId jobId = MRBuilderUtils.newJobId(appId, 1);
|
|
|
+ TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
|
|
|
+ TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
|
|
|
+ Path jobFile = mock(Path.class);
|
|
|
+
|
|
|
+ TaskAttemptListener taListener = mock(TaskAttemptListener.class);
|
|
|
+ when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0));
|
|
|
+
|
|
|
+ JobConf jobConf = new JobConf();
|
|
|
+ jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
|
|
|
+ jobConf.setBoolean("fs.file.impl.disable.cache", true);
|
|
|
+ jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, "");
|
|
|
+ jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
|
|
|
+
|
|
|
+ TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
|
|
|
+ when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
|
|
|
+
|
|
|
+ AppContext appCtx = mock(AppContext.class);
|
|
|
+ ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
|
|
+ when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
|
|
+
|
|
|
+ TaskAttemptImpl taImpl =
|
|
|
+ new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
|
|
|
+ splits, jobConf, taListener,
|
|
|
+ mock(Token.class), new Credentials(),
|
|
|
+ new SystemClock(), appCtx);
|
|
|
+
|
|
|
+ NodeId nid = NodeId.newInstance("127.0.0.1", 0);
|
|
|
+ ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
|
|
|
+ Container container = mock(Container.class);
|
|
|
+ when(container.getId()).thenReturn(contId);
|
|
|
+ when(container.getNodeId()).thenReturn(nid);
|
|
|
+ when(container.getNodeHttpAddress()).thenReturn("localhost:0");
|
|
|
+
|
|
|
+ taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
|
+ TaskAttemptEventType.TA_SCHEDULE));
|
|
|
+ taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
|
|
|
+ container, mock(Map.class)));
|
|
|
+ taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
|
|
|
+ return taImpl;
|
|
|
+ }
|
|
|
+
|
|
|
public static class MockEventHandler implements EventHandler {
|
|
|
public boolean internalError;
|
|
|
+ public TaskEvent lastTaskEvent;
|
|
|
|
|
|
@Override
|
|
|
public void handle(Event event) {
|
|
|
+ if (event instanceof TaskEvent) {
|
|
|
+ lastTaskEvent = (TaskEvent)event;
|
|
|
+ }
|
|
|
if (event instanceof JobEvent) {
|
|
|
JobEvent je = ((JobEvent) event);
|
|
|
if (JobEventType.INTERNAL_ERROR == je.getType()) {
|