|
@@ -84,7 +84,6 @@ public class TestTaskImpl {
|
|
private ApplicationId appId;
|
|
private ApplicationId appId;
|
|
private TaskSplitMetaInfo taskSplitMetaInfo;
|
|
private TaskSplitMetaInfo taskSplitMetaInfo;
|
|
private String[] dataLocations = new String[0];
|
|
private String[] dataLocations = new String[0];
|
|
- private final TaskType taskType = TaskType.MAP;
|
|
|
|
private AppContext appContext;
|
|
private AppContext appContext;
|
|
|
|
|
|
private int startCount = 0;
|
|
private int startCount = 0;
|
|
@@ -97,6 +96,7 @@ public class TestTaskImpl {
|
|
private class MockTaskImpl extends TaskImpl {
|
|
private class MockTaskImpl extends TaskImpl {
|
|
|
|
|
|
private int taskAttemptCounter = 0;
|
|
private int taskAttemptCounter = 0;
|
|
|
|
+ TaskType taskType;
|
|
|
|
|
|
public MockTaskImpl(JobId jobId, int partition,
|
|
public MockTaskImpl(JobId jobId, int partition,
|
|
EventHandler eventHandler, Path remoteJobConfFile, JobConf conf,
|
|
EventHandler eventHandler, Path remoteJobConfFile, JobConf conf,
|
|
@@ -104,11 +104,12 @@ public class TestTaskImpl {
|
|
Token<JobTokenIdentifier> jobToken,
|
|
Token<JobTokenIdentifier> jobToken,
|
|
Credentials credentials, Clock clock,
|
|
Credentials credentials, Clock clock,
|
|
Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
|
|
Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
|
|
- MRAppMetrics metrics, AppContext appContext) {
|
|
|
|
|
|
+ MRAppMetrics metrics, AppContext appContext, TaskType taskType) {
|
|
super(jobId, taskType , partition, eventHandler,
|
|
super(jobId, taskType , partition, eventHandler,
|
|
remoteJobConfFile, conf, taskAttemptListener, committer,
|
|
remoteJobConfFile, conf, taskAttemptListener, committer,
|
|
jobToken, credentials, clock,
|
|
jobToken, credentials, clock,
|
|
completedTasksFromPreviousRun, startCount, metrics, appContext);
|
|
completedTasksFromPreviousRun, startCount, metrics, appContext);
|
|
|
|
+ this.taskType = taskType;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -120,7 +121,7 @@ public class TestTaskImpl {
|
|
protected TaskAttemptImpl createAttempt() {
|
|
protected TaskAttemptImpl createAttempt() {
|
|
MockTaskAttemptImpl attempt = new MockTaskAttemptImpl(getID(), ++taskAttemptCounter,
|
|
MockTaskAttemptImpl attempt = new MockTaskAttemptImpl(getID(), ++taskAttemptCounter,
|
|
eventHandler, taskAttemptListener, remoteJobConfFile, partition,
|
|
eventHandler, taskAttemptListener, remoteJobConfFile, partition,
|
|
- conf, committer, jobToken, credentials, clock, appContext);
|
|
|
|
|
|
+ conf, committer, jobToken, credentials, clock, appContext, taskType);
|
|
taskAttempts.add(attempt);
|
|
taskAttempts.add(attempt);
|
|
return attempt;
|
|
return attempt;
|
|
}
|
|
}
|
|
@@ -142,18 +143,20 @@ public class TestTaskImpl {
|
|
private float progress = 0;
|
|
private float progress = 0;
|
|
private TaskAttemptState state = TaskAttemptState.NEW;
|
|
private TaskAttemptState state = TaskAttemptState.NEW;
|
|
private TaskAttemptId attemptId;
|
|
private TaskAttemptId attemptId;
|
|
|
|
+ private TaskType taskType;
|
|
|
|
|
|
public MockTaskAttemptImpl(TaskId taskId, int id, EventHandler eventHandler,
|
|
public MockTaskAttemptImpl(TaskId taskId, int id, EventHandler eventHandler,
|
|
TaskAttemptListener taskAttemptListener, Path jobFile, int partition,
|
|
TaskAttemptListener taskAttemptListener, Path jobFile, int partition,
|
|
JobConf conf, OutputCommitter committer,
|
|
JobConf conf, OutputCommitter committer,
|
|
Token<JobTokenIdentifier> jobToken,
|
|
Token<JobTokenIdentifier> jobToken,
|
|
Credentials credentials, Clock clock,
|
|
Credentials credentials, Clock clock,
|
|
- AppContext appContext) {
|
|
|
|
|
|
+ AppContext appContext, TaskType taskType) {
|
|
super(taskId, id, eventHandler, taskAttemptListener, jobFile, partition, conf,
|
|
super(taskId, id, eventHandler, taskAttemptListener, jobFile, partition, conf,
|
|
dataLocations, committer, jobToken, credentials, clock, appContext);
|
|
dataLocations, committer, jobToken, credentials, clock, appContext);
|
|
attemptId = Records.newRecord(TaskAttemptId.class);
|
|
attemptId = Records.newRecord(TaskAttemptId.class);
|
|
attemptId.setId(id);
|
|
attemptId.setId(id);
|
|
attemptId.setTaskId(taskId);
|
|
attemptId.setTaskId(taskId);
|
|
|
|
+ this.taskType = taskType;
|
|
}
|
|
}
|
|
|
|
|
|
public TaskAttemptId getAttemptId() {
|
|
public TaskAttemptId getAttemptId() {
|
|
@@ -162,7 +165,7 @@ public class TestTaskImpl {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
protected Task createRemoteTask() {
|
|
protected Task createRemoteTask() {
|
|
- return new MockTask();
|
|
|
|
|
|
+ return new MockTask(taskType);
|
|
}
|
|
}
|
|
|
|
|
|
public float getProgress() {
|
|
public float getProgress() {
|
|
@@ -185,6 +188,11 @@ public class TestTaskImpl {
|
|
|
|
|
|
private class MockTask extends Task {
|
|
private class MockTask extends Task {
|
|
|
|
|
|
|
|
+ private TaskType taskType;
|
|
|
|
+ MockTask(TaskType taskType) {
|
|
|
|
+ this.taskType = taskType;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void run(JobConf job, TaskUmbilicalProtocol umbilical)
|
|
public void run(JobConf job, TaskUmbilicalProtocol umbilical)
|
|
throws IOException, ClassNotFoundException, InterruptedException {
|
|
throws IOException, ClassNotFoundException, InterruptedException {
|
|
@@ -193,7 +201,7 @@ public class TestTaskImpl {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public boolean isMapTask() {
|
|
public boolean isMapTask() {
|
|
- return true;
|
|
|
|
|
|
+ return (taskType == TaskType.MAP);
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|
|
@@ -227,14 +235,15 @@ public class TestTaskImpl {
|
|
taskSplitMetaInfo = mock(TaskSplitMetaInfo.class);
|
|
taskSplitMetaInfo = mock(TaskSplitMetaInfo.class);
|
|
when(taskSplitMetaInfo.getLocations()).thenReturn(dataLocations);
|
|
when(taskSplitMetaInfo.getLocations()).thenReturn(dataLocations);
|
|
|
|
|
|
- taskAttempts = new ArrayList<MockTaskAttemptImpl>();
|
|
|
|
-
|
|
|
|
- mockTask = new MockTaskImpl(jobId, partition, dispatcher.getEventHandler(),
|
|
|
|
|
|
+ taskAttempts = new ArrayList<MockTaskAttemptImpl>();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private MockTaskImpl createMockTask(TaskType taskType) {
|
|
|
|
+ return new MockTaskImpl(jobId, partition, dispatcher.getEventHandler(),
|
|
remoteJobConfFile, conf, taskAttemptListener, committer, jobToken,
|
|
remoteJobConfFile, conf, taskAttemptListener, committer, jobToken,
|
|
credentials, clock,
|
|
credentials, clock,
|
|
completedTasksFromPreviousRun, startCount,
|
|
completedTasksFromPreviousRun, startCount,
|
|
- metrics, appContext);
|
|
|
|
-
|
|
|
|
|
|
+ metrics, appContext, taskType);
|
|
}
|
|
}
|
|
|
|
|
|
@After
|
|
@After
|
|
@@ -342,6 +351,7 @@ public class TestTaskImpl {
|
|
@Test
|
|
@Test
|
|
public void testInit() {
|
|
public void testInit() {
|
|
LOG.info("--- START: testInit ---");
|
|
LOG.info("--- START: testInit ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
assertTaskNewState();
|
|
assertTaskNewState();
|
|
assert(taskAttempts.size() == 0);
|
|
assert(taskAttempts.size() == 0);
|
|
}
|
|
}
|
|
@@ -352,6 +362,7 @@ public class TestTaskImpl {
|
|
*/
|
|
*/
|
|
public void testScheduleTask() {
|
|
public void testScheduleTask() {
|
|
LOG.info("--- START: testScheduleTask ---");
|
|
LOG.info("--- START: testScheduleTask ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
}
|
|
}
|
|
@@ -362,6 +373,7 @@ public class TestTaskImpl {
|
|
*/
|
|
*/
|
|
public void testKillScheduledTask() {
|
|
public void testKillScheduledTask() {
|
|
LOG.info("--- START: testKillScheduledTask ---");
|
|
LOG.info("--- START: testKillScheduledTask ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
killTask(taskId);
|
|
killTask(taskId);
|
|
@@ -374,6 +386,7 @@ public class TestTaskImpl {
|
|
*/
|
|
*/
|
|
public void testKillScheduledTaskAttempt() {
|
|
public void testKillScheduledTaskAttempt() {
|
|
LOG.info("--- START: testKillScheduledTaskAttempt ---");
|
|
LOG.info("--- START: testKillScheduledTaskAttempt ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
killScheduledTaskAttempt(getLastAttempt().getAttemptId());
|
|
killScheduledTaskAttempt(getLastAttempt().getAttemptId());
|
|
@@ -386,6 +399,7 @@ public class TestTaskImpl {
|
|
*/
|
|
*/
|
|
public void testLaunchTaskAttempt() {
|
|
public void testLaunchTaskAttempt() {
|
|
LOG.info("--- START: testLaunchTaskAttempt ---");
|
|
LOG.info("--- START: testLaunchTaskAttempt ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
@@ -398,6 +412,7 @@ public class TestTaskImpl {
|
|
*/
|
|
*/
|
|
public void testKillRunningTaskAttempt() {
|
|
public void testKillRunningTaskAttempt() {
|
|
LOG.info("--- START: testKillRunningTaskAttempt ---");
|
|
LOG.info("--- START: testKillRunningTaskAttempt ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
@@ -407,6 +422,7 @@ public class TestTaskImpl {
|
|
@Test
|
|
@Test
|
|
public void testTaskProgress() {
|
|
public void testTaskProgress() {
|
|
LOG.info("--- START: testTaskProgress ---");
|
|
LOG.info("--- START: testTaskProgress ---");
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
|
|
|
|
// launch task
|
|
// launch task
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
@@ -444,6 +460,7 @@ public class TestTaskImpl {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testFailureDuringTaskAttemptCommit() {
|
|
public void testFailureDuringTaskAttemptCommit() {
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
@@ -469,8 +486,7 @@ public class TestTaskImpl {
|
|
assertTaskSucceededState();
|
|
assertTaskSucceededState();
|
|
}
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
|
- public void testSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
|
|
|
|
|
|
+ private void runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType failEvent) {
|
|
TaskId taskId = getNewTaskID();
|
|
TaskId taskId = getNewTaskID();
|
|
scheduleTaskAttempt(taskId);
|
|
scheduleTaskAttempt(taskId);
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
|
@@ -489,11 +505,34 @@ public class TestTaskImpl {
|
|
|
|
|
|
// Now fail the first task attempt, after the second has succeeded
|
|
// Now fail the first task attempt, after the second has succeeded
|
|
mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(0).getAttemptId(),
|
|
mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(0).getAttemptId(),
|
|
- TaskEventType.T_ATTEMPT_FAILED));
|
|
|
|
|
|
+ failEvent));
|
|
|
|
|
|
// The task should still be in the succeeded state
|
|
// The task should still be in the succeeded state
|
|
assertTaskSucceededState();
|
|
assertTaskSucceededState();
|
|
-
|
|
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
|
|
+ runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_FAILED);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
|
|
|
|
+ mockTask = createMockTask(TaskType.REDUCE);
|
|
|
|
+ runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_FAILED);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
|
|
|
|
+ mockTask = createMockTask(TaskType.MAP);
|
|
|
|
+ runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_KILLED);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
|
|
|
|
+ mockTask = createMockTask(TaskType.REDUCE);
|
|
|
|
+ runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_KILLED);
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|