|
@@ -65,6 +65,7 @@ import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttemptStateInternal;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
|
@@ -795,6 +796,178 @@ public class TestTaskAttempt{
|
|
|
finishTime, Long.valueOf(taImpl.getFinishTime()));
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testContainerKillAfterAssigned() throws Exception {
|
|
|
+ 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);
|
|
|
+
|
|
|
+ MockEventHandler eventHandler = new MockEventHandler();
|
|
|
+ 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);
|
|
|
+ Resource resource = mock(Resource.class);
|
|
|
+ when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
|
|
+ when(resource.getMemory()).thenReturn(1024);
|
|
|
+
|
|
|
+ TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
|
|
+ jobFile, 1, splits, jobConf, taListener, new Token(),
|
|
|
+ new Credentials(), new SystemClock(), appCtx);
|
|
|
+
|
|
|
+ NodeId nid = NodeId.newInstance("127.0.0.2", 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)));
|
|
|
+ assertEquals("Task attempt is not in assinged state",
|
|
|
+ taImpl.getInternalState(), TaskAttemptStateInternal.ASSIGNED);
|
|
|
+ taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
|
+ TaskAttemptEventType.TA_KILL));
|
|
|
+ assertEquals("Task should be in KILLED state",
|
|
|
+ TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
+ taImpl.getInternalState());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testContainerKillWhileRunning() throws Exception {
|
|
|
+ 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);
|
|
|
+
|
|
|
+ MockEventHandler eventHandler = new MockEventHandler();
|
|
|
+ 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);
|
|
|
+ Resource resource = mock(Resource.class);
|
|
|
+ when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
|
|
+ when(resource.getMemory()).thenReturn(1024);
|
|
|
+
|
|
|
+ TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
|
|
+ jobFile, 1, splits, jobConf, taListener, new Token(),
|
|
|
+ new Credentials(), new SystemClock(), appCtx);
|
|
|
+
|
|
|
+ NodeId nid = NodeId.newInstance("127.0.0.2", 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));
|
|
|
+ assertEquals("Task attempt is not in running state", taImpl.getState(),
|
|
|
+ TaskAttemptState.RUNNING);
|
|
|
+ taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
|
+ TaskAttemptEventType.TA_KILL));
|
|
|
+ assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
|
+ eventHandler.internalError);
|
|
|
+ assertEquals("Task should be in KILLED state",
|
|
|
+ TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
+ taImpl.getInternalState());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testContainerKillWhileCommitPending() throws Exception {
|
|
|
+ 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);
|
|
|
+
|
|
|
+ MockEventHandler eventHandler = new MockEventHandler();
|
|
|
+ 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);
|
|
|
+ Resource resource = mock(Resource.class);
|
|
|
+ when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
|
|
+ when(resource.getMemory()).thenReturn(1024);
|
|
|
+
|
|
|
+ TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
|
|
+ jobFile, 1, splits, jobConf, taListener, new Token(),
|
|
|
+ new Credentials(), new SystemClock(), appCtx);
|
|
|
+
|
|
|
+ NodeId nid = NodeId.newInstance("127.0.0.2", 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));
|
|
|
+ assertEquals("Task attempt is not in running state", taImpl.getState(),
|
|
|
+ TaskAttemptState.RUNNING);
|
|
|
+ taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
|
+ TaskAttemptEventType.TA_COMMIT_PENDING));
|
|
|
+ assertEquals("Task should be in COMMIT_PENDING state",
|
|
|
+ TaskAttemptStateInternal.COMMIT_PENDING, taImpl.getInternalState());
|
|
|
+ taImpl.handle(new TaskAttemptEvent(attemptId,
|
|
|
+ TaskAttemptEventType.TA_KILL));
|
|
|
+ assertFalse("InternalError occurred trying to handle TA_KILL",
|
|
|
+ eventHandler.internalError);
|
|
|
+ assertEquals("Task should be in KILLED state",
|
|
|
+ TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
+ taImpl.getInternalState());
|
|
|
+ }
|
|
|
+
|
|
|
public static class MockEventHandler implements EventHandler {
|
|
|
public boolean internalError;
|
|
|
|