|
@@ -18,10 +18,21 @@
|
|
|
|
|
|
package org.apache.hadoop.mapreduce.v2.app;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.mockito.Mockito.atLeast;
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
+
|
|
|
import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
|
|
|
import junit.framework.Assert;
|
|
|
|
|
@@ -31,36 +42,66 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.io.NullWritable;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
+import org.apache.hadoop.mapred.JobConf;
|
|
|
+import org.apache.hadoop.mapreduce.Counters;
|
|
|
+import org.apache.hadoop.mapreduce.JobCounter;
|
|
|
+import org.apache.hadoop.mapreduce.JobID;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
|
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
|
+import org.apache.hadoop.mapreduce.TaskAttemptID;
|
|
|
+import org.apache.hadoop.mapreduce.TaskID;
|
|
|
+import org.apache.hadoop.mapreduce.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.Event;
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.EventType;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
|
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
|
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
|
|
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
|
|
|
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
|
|
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
|
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.event.JobCounterUpdateEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
|
|
|
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.TaskEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.impl.MapTaskImpl;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
|
|
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
|
+import org.apache.hadoop.security.Credentials;
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
+import org.apache.hadoop.yarn.Clock;
|
|
|
+import org.apache.hadoop.yarn.ClusterInfo;
|
|
|
+import org.apache.hadoop.yarn.SystemClock;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
+import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.ArgumentCaptor;
|
|
|
|
|
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
|
|
public class TestRecovery {
|
|
@@ -75,6 +116,7 @@ public class TestRecovery {
|
|
|
private Text val1 = new Text("val1");
|
|
|
private Text val2 = new Text("val2");
|
|
|
|
|
|
+
|
|
|
/**
|
|
|
* AM with 2 maps and 1 reduce. For 1st map, one attempt fails, one attempt
|
|
|
* completely disappears because of failed launch, one attempt gets killed and
|
|
@@ -1011,6 +1053,427 @@ public class TestRecovery {
|
|
|
app.verifyCompleted();
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testRecoverySuccessAttempt() {
|
|
|
+ LOG.info("--- START: testRecoverySuccessAttempt ---");
|
|
|
+
|
|
|
+ long clusterTimestamp = System.currentTimeMillis();
|
|
|
+ EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
+ MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
|
|
+ mockEventHandler);
|
|
|
+
|
|
|
+ TaskId taskId = recoverMapTask.getID();
|
|
|
+ JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
|
|
+ TaskID taskID = new TaskID(jobID,
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
|
|
+
|
|
|
+ //Mock up the TaskAttempts
|
|
|
+ Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
|
|
+
|
|
|
+ TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
|
|
+ TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
|
|
+ TaskAttemptState.SUCCEEDED);
|
|
|
+ mockTaskAttempts.put(taId1, mockTAinfo1);
|
|
|
+
|
|
|
+ TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
|
|
+ TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
|
|
+ TaskAttemptState.FAILED);
|
|
|
+ mockTaskAttempts.put(taId2, mockTAinfo2);
|
|
|
+
|
|
|
+ OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
|
|
+ TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
|
|
+ when(mockTaskInfo.getTaskStatus()).thenReturn("SUCCEEDED");
|
|
|
+ when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
|
|
+ when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
|
|
+
|
|
|
+ recoverMapTask.handle(
|
|
|
+ new TaskRecoverEvent(taskId, mockTaskInfo,mockCommitter, true));
|
|
|
+
|
|
|
+ ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
|
|
+ verify(mockEventHandler,atLeast(1)).handle(
|
|
|
+ (org.apache.hadoop.yarn.event.Event) arg.capture());
|
|
|
+
|
|
|
+ Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptState>();
|
|
|
+ finalAttemptStates.put(taId1, TaskAttemptState.SUCCEEDED);
|
|
|
+ finalAttemptStates.put(taId2, TaskAttemptState.FAILED);
|
|
|
+
|
|
|
+ List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
|
|
+ jobHistoryEvents.add(EventType.TASK_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FINISHED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
|
|
+ jobHistoryEvents.add(EventType.TASK_FINISHED);
|
|
|
+ recoveryChecker(recoverMapTask, TaskState.SUCCEEDED, finalAttemptStates,
|
|
|
+ arg, jobHistoryEvents, 2L, 1L);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRecoveryAllFailAttempts() {
|
|
|
+ LOG.info("--- START: testRecoveryAllFailAttempts ---");
|
|
|
+
|
|
|
+ long clusterTimestamp = System.currentTimeMillis();
|
|
|
+ EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
+ MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
|
|
+ mockEventHandler);
|
|
|
+
|
|
|
+ TaskId taskId = recoverMapTask.getID();
|
|
|
+ JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
|
|
+ TaskID taskID = new TaskID(jobID,
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
|
|
+
|
|
|
+ //Mock up the TaskAttempts
|
|
|
+ Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
|
|
+
|
|
|
+ TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
|
|
+ TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
|
|
+ TaskAttemptState.FAILED);
|
|
|
+ mockTaskAttempts.put(taId1, mockTAinfo1);
|
|
|
+
|
|
|
+ TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
|
|
+ TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
|
|
+ TaskAttemptState.FAILED);
|
|
|
+ mockTaskAttempts.put(taId2, mockTAinfo2);
|
|
|
+
|
|
|
+ OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
|
|
+
|
|
|
+ TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
|
|
+ when(mockTaskInfo.getTaskStatus()).thenReturn("FAILED");
|
|
|
+ when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
|
|
+ when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
|
|
+ when(mockTaskInfo.getError()).thenReturn("error string");
|
|
|
+ when(mockTaskInfo.getTaskType()).thenReturn(TaskType.MAP);
|
|
|
+
|
|
|
+ recoverMapTask.handle(
|
|
|
+ new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
|
|
+
|
|
|
+ ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
|
|
+ verify(mockEventHandler,atLeast(1)).handle(
|
|
|
+ (org.apache.hadoop.yarn.event.Event) arg.capture());
|
|
|
+
|
|
|
+ Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptState>();
|
|
|
+ finalAttemptStates.put(taId1, TaskAttemptState.FAILED);
|
|
|
+ finalAttemptStates.put(taId2, TaskAttemptState.FAILED);
|
|
|
+
|
|
|
+ List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
|
|
+ jobHistoryEvents.add(EventType.TASK_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
|
|
+ jobHistoryEvents.add(EventType.TASK_FAILED);
|
|
|
+ recoveryChecker(recoverMapTask, TaskState.FAILED, finalAttemptStates,
|
|
|
+ arg, jobHistoryEvents, 2L, 2L);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRecoveryTaskSuccessAllAttemptsFail() {
|
|
|
+ LOG.info("--- START: testRecoveryTaskSuccessAllAttemptsFail ---");
|
|
|
+
|
|
|
+ long clusterTimestamp = System.currentTimeMillis();
|
|
|
+ EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
+ MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
|
|
+ mockEventHandler);
|
|
|
+
|
|
|
+ TaskId taskId = recoverMapTask.getID();
|
|
|
+ JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
|
|
+ TaskID taskID = new TaskID(jobID,
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
|
|
+
|
|
|
+ //Mock up the TaskAttempts
|
|
|
+ Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
|
|
+
|
|
|
+ TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
|
|
+ TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
|
|
+ TaskAttemptState.FAILED);
|
|
|
+ mockTaskAttempts.put(taId1, mockTAinfo1);
|
|
|
+
|
|
|
+ TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
|
|
+ TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
|
|
+ TaskAttemptState.FAILED);
|
|
|
+ mockTaskAttempts.put(taId2, mockTAinfo2);
|
|
|
+
|
|
|
+ OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
|
|
+ TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
|
|
+ when(mockTaskInfo.getTaskStatus()).thenReturn("SUCCEEDED");
|
|
|
+ when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
|
|
+ when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
|
|
+
|
|
|
+ recoverMapTask.handle(
|
|
|
+ new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
|
|
+
|
|
|
+ ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
|
|
+ verify(mockEventHandler,atLeast(1)).handle(
|
|
|
+ (org.apache.hadoop.yarn.event.Event) arg.capture());
|
|
|
+
|
|
|
+ TaskAttemptID taId3 = new TaskAttemptID(taskID, 2000);
|
|
|
+ Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptState>();
|
|
|
+ finalAttemptStates.put(taId1, TaskAttemptState.FAILED);
|
|
|
+ finalAttemptStates.put(taId2, TaskAttemptState.FAILED);
|
|
|
+ finalAttemptStates.put(taId3, TaskAttemptState.NEW);
|
|
|
+
|
|
|
+ List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
|
|
+ jobHistoryEvents.add(EventType.TASK_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
|
|
+ recoveryChecker(recoverMapTask, TaskState.RUNNING, finalAttemptStates,
|
|
|
+ arg, jobHistoryEvents, 2L, 2L);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRecoveryTaskSuccessAllAttemptsSucceed() {
|
|
|
+ LOG.info("--- START: testRecoveryTaskSuccessAllAttemptsFail ---");
|
|
|
+
|
|
|
+ long clusterTimestamp = System.currentTimeMillis();
|
|
|
+ EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
+ MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
|
|
+ mockEventHandler);
|
|
|
+
|
|
|
+ TaskId taskId = recoverMapTask.getID();
|
|
|
+ JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
|
|
+ TaskID taskID = new TaskID(jobID,
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
|
|
+
|
|
|
+ //Mock up the TaskAttempts
|
|
|
+ Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
|
|
+
|
|
|
+ TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
|
|
+ TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
|
|
+ TaskAttemptState.SUCCEEDED);
|
|
|
+ mockTaskAttempts.put(taId1, mockTAinfo1);
|
|
|
+
|
|
|
+ TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
|
|
+ TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
|
|
+ TaskAttemptState.SUCCEEDED);
|
|
|
+ mockTaskAttempts.put(taId2, mockTAinfo2);
|
|
|
+
|
|
|
+ OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
|
|
+ TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
|
|
+ when(mockTaskInfo.getTaskStatus()).thenReturn("SUCCEEDED");
|
|
|
+ when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
|
|
+ when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
|
|
+
|
|
|
+ recoverMapTask.handle(
|
|
|
+ new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
|
|
+
|
|
|
+ ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
|
|
+ verify(mockEventHandler,atLeast(1)).handle(
|
|
|
+ (org.apache.hadoop.yarn.event.Event) arg.capture());
|
|
|
+
|
|
|
+ Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptState>();
|
|
|
+ finalAttemptStates.put(taId1, TaskAttemptState.SUCCEEDED);
|
|
|
+ finalAttemptStates.put(taId2, TaskAttemptState.SUCCEEDED);
|
|
|
+
|
|
|
+ List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
|
|
+ jobHistoryEvents.add(EventType.TASK_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FINISHED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_FINISHED);
|
|
|
+ jobHistoryEvents.add(EventType.TASK_FINISHED);
|
|
|
+ recoveryChecker(recoverMapTask, TaskState.SUCCEEDED, finalAttemptStates,
|
|
|
+ arg, jobHistoryEvents, 2L, 0L);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRecoveryAllAttemptsKilled() {
|
|
|
+ LOG.info("--- START: testRecoveryAllAttemptsKilled ---");
|
|
|
+
|
|
|
+ long clusterTimestamp = System.currentTimeMillis();
|
|
|
+ EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
+ MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
|
|
+ mockEventHandler);
|
|
|
+
|
|
|
+ TaskId taskId = recoverMapTask.getID();
|
|
|
+ JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
|
|
+ TaskID taskID = new TaskID(jobID,
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
|
|
+
|
|
|
+ //Mock up the TaskAttempts
|
|
|
+ Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
|
|
+ TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
|
|
+ TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
|
|
+ TaskAttemptState.KILLED);
|
|
|
+ mockTaskAttempts.put(taId1, mockTAinfo1);
|
|
|
+
|
|
|
+ TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
|
|
+ TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
|
|
+ TaskAttemptState.KILLED);
|
|
|
+ mockTaskAttempts.put(taId2, mockTAinfo2);
|
|
|
+
|
|
|
+ OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
|
|
+ TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
|
|
+ when(mockTaskInfo.getTaskStatus()).thenReturn("KILLED");
|
|
|
+ when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
|
|
+ when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
|
|
+ when(mockTaskInfo.getError()).thenReturn("");
|
|
|
+ when(mockTaskInfo.getTaskType()).thenReturn(TaskType.MAP);
|
|
|
+
|
|
|
+ recoverMapTask.handle(
|
|
|
+ new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
|
|
+
|
|
|
+ ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
|
|
+ verify(mockEventHandler,atLeast(1)).handle(
|
|
|
+ (org.apache.hadoop.yarn.event.Event) arg.capture());
|
|
|
+
|
|
|
+ Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
|
|
+ new HashMap<TaskAttemptID, TaskAttemptState>();
|
|
|
+ finalAttemptStates.put(taId1, TaskAttemptState.KILLED);
|
|
|
+ finalAttemptStates.put(taId2, TaskAttemptState.KILLED);
|
|
|
+
|
|
|
+ List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
|
|
+ jobHistoryEvents.add(EventType.TASK_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_KILLED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
|
|
+ jobHistoryEvents.add(EventType.MAP_ATTEMPT_KILLED);
|
|
|
+ jobHistoryEvents.add(EventType.TASK_FAILED);
|
|
|
+ recoveryChecker(recoverMapTask, TaskState.KILLED, finalAttemptStates,
|
|
|
+ arg, jobHistoryEvents, 2L, 0L);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void recoveryChecker(MapTaskImpl checkTask, TaskState finalState,
|
|
|
+ Map<TaskAttemptID, TaskAttemptState> finalAttemptStates,
|
|
|
+ ArgumentCaptor<Event> arg, List<EventType> expectedJobHistoryEvents,
|
|
|
+ long expectedMapLaunches, long expectedFailedMaps) {
|
|
|
+
|
|
|
+ assertEquals("Final State of Task", finalState, checkTask.getState());
|
|
|
+
|
|
|
+ Map<TaskAttemptId, TaskAttempt> recoveredAttempts =
|
|
|
+ checkTask.getAttempts();
|
|
|
+ assertEquals("Expected Number of Task Attempts",
|
|
|
+ finalAttemptStates.size(), recoveredAttempts.size());
|
|
|
+ for (TaskAttemptID taID : finalAttemptStates.keySet()) {
|
|
|
+ assertEquals("Expected Task Attempt State",
|
|
|
+ finalAttemptStates.get(taID),
|
|
|
+ recoveredAttempts.get(TypeConverter.toYarn(taID)).getState());
|
|
|
+ }
|
|
|
+
|
|
|
+ Iterator<Event> ie = arg.getAllValues().iterator();
|
|
|
+ int eventNum = 0;
|
|
|
+ long totalLaunchedMaps = 0;
|
|
|
+ long totalFailedMaps = 0;
|
|
|
+ boolean jobTaskEventReceived = false;
|
|
|
+
|
|
|
+ while (ie.hasNext()) {
|
|
|
+ Object current = ie.next();
|
|
|
+ ++eventNum;
|
|
|
+ LOG.info(eventNum + " " + current.getClass().getName());
|
|
|
+ String className = current.getClass().getName();
|
|
|
+ if (className.equals("org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent")) {
|
|
|
+ JobHistoryEvent jhe = (JobHistoryEvent) current;
|
|
|
+ LOG.info(expectedJobHistoryEvents.get(0).toString() + " " +
|
|
|
+ jhe.getHistoryEvent().getEventType().toString() + " " +
|
|
|
+ jhe.getJobID());
|
|
|
+ assertEquals(expectedJobHistoryEvents.get(0),
|
|
|
+ jhe.getHistoryEvent().getEventType());
|
|
|
+ expectedJobHistoryEvents.remove(0);
|
|
|
+ } else if (className.equals("org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent")) {
|
|
|
+ JobCounterUpdateEvent jcue = (JobCounterUpdateEvent) current;
|
|
|
+
|
|
|
+ LOG.info("JobCounterUpdateEvent "
|
|
|
+ + jcue.getCounterUpdates().get(0).getCounterKey()
|
|
|
+ + " " + jcue.getCounterUpdates().get(0).getIncrementValue());
|
|
|
+ if (jcue.getCounterUpdates().get(0).getCounterKey() ==
|
|
|
+ JobCounter.NUM_FAILED_MAPS) {
|
|
|
+ totalFailedMaps += jcue.getCounterUpdates().get(0)
|
|
|
+ .getIncrementValue();
|
|
|
+ } else if (jcue.getCounterUpdates().get(0).getCounterKey() ==
|
|
|
+ JobCounter.TOTAL_LAUNCHED_MAPS) {
|
|
|
+ totalLaunchedMaps += jcue.getCounterUpdates().get(0)
|
|
|
+ .getIncrementValue();
|
|
|
+ }
|
|
|
+ } else if (className.equals("org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent")) {
|
|
|
+ JobTaskEvent jte = (JobTaskEvent) current;
|
|
|
+ assertEquals(jte.getState(), finalState);
|
|
|
+ jobTaskEventReceived = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ assertTrue(jobTaskEventReceived || (finalState == TaskState.RUNNING));
|
|
|
+ assertEquals("Did not process all expected JobHistoryEvents",
|
|
|
+ 0, expectedJobHistoryEvents.size());
|
|
|
+ assertEquals("Expected Map Launches",
|
|
|
+ expectedMapLaunches, totalLaunchedMaps);
|
|
|
+ assertEquals("Expected Failed Maps",
|
|
|
+ expectedFailedMaps, totalFailedMaps);
|
|
|
+ }
|
|
|
+
|
|
|
+ private MapTaskImpl getMockMapTask(long clusterTimestamp, EventHandler eh) {
|
|
|
+
|
|
|
+ ApplicationId appId = BuilderUtils.newApplicationId(clusterTimestamp, 1);
|
|
|
+ JobId jobId = MRBuilderUtils.newJobId(appId, 1);
|
|
|
+
|
|
|
+ int partitions = 2;
|
|
|
+
|
|
|
+ Path remoteJobConfFile = mock(Path.class);
|
|
|
+ JobConf conf = new JobConf();
|
|
|
+ TaskAttemptListener taskAttemptListener = mock(TaskAttemptListener.class);
|
|
|
+ Token<JobTokenIdentifier> jobToken =
|
|
|
+ (Token<JobTokenIdentifier>) mock(Token.class);
|
|
|
+ Credentials credentials = null;
|
|
|
+ Clock clock = new SystemClock();
|
|
|
+ int appAttemptId = 3;
|
|
|
+ MRAppMetrics metrics = mock(MRAppMetrics.class);
|
|
|
+ Resource minContainerRequirements = mock(Resource.class);
|
|
|
+ when(minContainerRequirements.getMemory()).thenReturn(1000);
|
|
|
+
|
|
|
+ ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
|
|
+ when(clusterInfo.getMinContainerCapability()).thenReturn(
|
|
|
+ minContainerRequirements);
|
|
|
+ AppContext appContext = mock(AppContext.class);
|
|
|
+ when(appContext.getClusterInfo()).thenReturn(clusterInfo);
|
|
|
+
|
|
|
+ TaskSplitMetaInfo taskSplitMetaInfo = mock(TaskSplitMetaInfo.class);
|
|
|
+ MapTaskImpl mapTask = new MapTaskImpl(jobId, partitions,
|
|
|
+ eh, remoteJobConfFile, conf,
|
|
|
+ taskSplitMetaInfo, taskAttemptListener, jobToken, credentials, clock,
|
|
|
+ appAttemptId, metrics, appContext);
|
|
|
+ return mapTask;
|
|
|
+ }
|
|
|
+
|
|
|
+ private TaskAttemptInfo getMockTaskAttemptInfo(TaskAttemptID tai,
|
|
|
+ TaskAttemptState tas) {
|
|
|
+
|
|
|
+ ContainerId ci = mock(ContainerId.class);
|
|
|
+ Counters counters = mock(Counters.class);
|
|
|
+ TaskType tt = TaskType.MAP;
|
|
|
+
|
|
|
+ long finishTime = System.currentTimeMillis();
|
|
|
+
|
|
|
+ TaskAttemptInfo mockTAinfo = mock(TaskAttemptInfo.class);
|
|
|
+
|
|
|
+ when(mockTAinfo.getAttemptId()).thenReturn(tai);
|
|
|
+ when(mockTAinfo.getContainerId()).thenReturn(ci);
|
|
|
+ when(mockTAinfo.getCounters()).thenReturn(counters);
|
|
|
+ when(mockTAinfo.getError()).thenReturn("");
|
|
|
+ when(mockTAinfo.getFinishTime()).thenReturn(finishTime);
|
|
|
+ when(mockTAinfo.getHostname()).thenReturn("localhost");
|
|
|
+ when(mockTAinfo.getHttpPort()).thenReturn(23);
|
|
|
+ when(mockTAinfo.getMapFinishTime()).thenReturn(finishTime - 1000L);
|
|
|
+ when(mockTAinfo.getPort()).thenReturn(24);
|
|
|
+ when(mockTAinfo.getRackname()).thenReturn("defaultRack");
|
|
|
+ when(mockTAinfo.getShuffleFinishTime()).thenReturn(finishTime - 2000L);
|
|
|
+ when(mockTAinfo.getShufflePort()).thenReturn(25);
|
|
|
+ when(mockTAinfo.getSortFinishTime()).thenReturn(finishTime - 3000L);
|
|
|
+ when(mockTAinfo.getStartTime()).thenReturn(finishTime -10000);
|
|
|
+ when(mockTAinfo.getState()).thenReturn("task in progress");
|
|
|
+ when(mockTAinfo.getTaskStatus()).thenReturn(tas.toString());
|
|
|
+ when(mockTAinfo.getTaskType()).thenReturn(tt);
|
|
|
+ when(mockTAinfo.getTrackerName()).thenReturn("TrackerName");
|
|
|
+ return mockTAinfo;
|
|
|
+ }
|
|
|
+
|
|
|
private void writeBadOutput(TaskAttempt attempt, Configuration conf)
|
|
|
throws Exception {
|
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf,
|
|
@@ -1145,5 +1608,16 @@ public class TestRecovery {
|
|
|
public static void main(String[] arg) throws Exception {
|
|
|
TestRecovery test = new TestRecovery();
|
|
|
test.testCrashed();
|
|
|
+ test.testMultipleCrashes();
|
|
|
+ test.testOutputRecovery();
|
|
|
+ test.testOutputRecoveryMapsOnly();
|
|
|
+ test.testRecoveryWithOldCommiter();
|
|
|
+ test.testSpeculative();
|
|
|
+ test.testRecoveryWithoutShuffleSecret();
|
|
|
+ test.testRecoverySuccessAttempt();
|
|
|
+ test.testRecoveryAllFailAttempts();
|
|
|
+ test.testRecoveryTaskSuccessAllAttemptsFail();
|
|
|
+ test.testRecoveryTaskSuccessAllAttemptsSucceed();
|
|
|
+ test.testRecoveryAllAttemptsKilled();
|
|
|
}
|
|
|
}
|