MAPREDUCE-4607. Race condition in ReduceTask completion can result in Task being incorrectly failed. Contributed by Bikas Saha.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1383422 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
bdcbc2d1ef
commit
3b46295c28
@ -71,6 +71,7 @@
|
|||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
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.TaskId;
|
||||||
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
|
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
|
||||||
@ -86,6 +87,7 @@
|
|||||||
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.TaskAttemptKillEvent;
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
|
||||||
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
|
||||||
@ -120,6 +122,7 @@
|
|||||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
|
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
|
||||||
|
import org.apache.hadoop.yarn.state.MultipleArcTransition;
|
||||||
import org.apache.hadoop.yarn.state.SingleArcTransition;
|
import org.apache.hadoop.yarn.state.SingleArcTransition;
|
||||||
import org.apache.hadoop.yarn.state.StateMachine;
|
import org.apache.hadoop.yarn.state.StateMachine;
|
||||||
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
||||||
@ -128,6 +131,8 @@
|
|||||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
import org.apache.hadoop.yarn.util.RackResolver;
|
import org.apache.hadoop.yarn.util.RackResolver;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of TaskAttempt interface.
|
* Implementation of TaskAttempt interface.
|
||||||
*/
|
*/
|
||||||
@ -404,10 +409,10 @@ TaskAttemptEventType.TA_CONTAINER_CLEANED, new TaskCleanupTransition())
|
|||||||
TaskAttemptState.FAILED,
|
TaskAttemptState.FAILED,
|
||||||
TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE,
|
TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE,
|
||||||
new TooManyFetchFailureTransition())
|
new TooManyFetchFailureTransition())
|
||||||
.addTransition(
|
.addTransition(TaskAttemptState.SUCCEEDED,
|
||||||
TaskAttemptState.SUCCEEDED, TaskAttemptState.KILLED,
|
EnumSet.of(TaskAttemptState.SUCCEEDED, TaskAttemptState.KILLED),
|
||||||
TaskAttemptEventType.TA_KILL,
|
TaskAttemptEventType.TA_KILL,
|
||||||
new KilledAfterSuccessTransition())
|
new KilledAfterSuccessTransition())
|
||||||
.addTransition(
|
.addTransition(
|
||||||
TaskAttemptState.SUCCEEDED, TaskAttemptState.SUCCEEDED,
|
TaskAttemptState.SUCCEEDED, TaskAttemptState.SUCCEEDED,
|
||||||
TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
||||||
@ -1483,6 +1488,9 @@ private static class TooManyFetchFailureTransition implements
|
|||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Override
|
@Override
|
||||||
public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
|
public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
|
||||||
|
// too many fetch failure can only happen for map tasks
|
||||||
|
Preconditions
|
||||||
|
.checkArgument(taskAttempt.getID().getTaskId().getTaskType() == TaskType.MAP);
|
||||||
//add to diagnostic
|
//add to diagnostic
|
||||||
taskAttempt.addDiagnosticInfo("Too Many fetch failures.Failing the attempt");
|
taskAttempt.addDiagnosticInfo("Too Many fetch failures.Failing the attempt");
|
||||||
//set the finish time
|
//set the finish time
|
||||||
@ -1506,15 +1514,30 @@ public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static class KilledAfterSuccessTransition implements
|
private static class KilledAfterSuccessTransition implements
|
||||||
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptState> {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Override
|
@Override
|
||||||
public void transition(TaskAttemptImpl taskAttempt,
|
public TaskAttemptState transition(TaskAttemptImpl taskAttempt,
|
||||||
TaskAttemptEvent event) {
|
TaskAttemptEvent event) {
|
||||||
TaskAttemptKillEvent msgEvent = (TaskAttemptKillEvent) event;
|
if(taskAttempt.getID().getTaskId().getTaskType() == TaskType.REDUCE) {
|
||||||
//add to diagnostic
|
// after a reduce task has succeeded, its outputs are in safe in HDFS.
|
||||||
taskAttempt.addDiagnosticInfo(msgEvent.getMessage());
|
// logically such a task should not be killed. we only come here when
|
||||||
|
// there is a race condition in the event queue. E.g. some logic sends
|
||||||
|
// a kill request to this attempt when the successful completion event
|
||||||
|
// for this task is already in the event queue. so the kill event will
|
||||||
|
// get executed immediately after the attempt is marked successful and
|
||||||
|
// result in this transition being exercised.
|
||||||
|
// ignore this for reduce tasks
|
||||||
|
LOG.info("Ignoring killed event for successful reduce task attempt" +
|
||||||
|
taskAttempt.getID().toString());
|
||||||
|
return TaskAttemptState.SUCCEEDED;
|
||||||
|
}
|
||||||
|
if(event instanceof TaskAttemptKillEvent) {
|
||||||
|
TaskAttemptKillEvent msgEvent = (TaskAttemptKillEvent) event;
|
||||||
|
//add to diagnostic
|
||||||
|
taskAttempt.addDiagnosticInfo(msgEvent.getMessage());
|
||||||
|
}
|
||||||
|
|
||||||
// not setting a finish time since it was set on success
|
// not setting a finish time since it was set on success
|
||||||
assert (taskAttempt.getFinishTime() != 0);
|
assert (taskAttempt.getFinishTime() != 0);
|
||||||
@ -1528,6 +1551,7 @@ public void transition(TaskAttemptImpl taskAttempt,
|
|||||||
.getTaskId().getJobId(), tauce));
|
.getTaskId().getJobId(), tauce));
|
||||||
taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
|
taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
|
||||||
taskAttempt.attemptId, TaskEventType.T_ATTEMPT_KILLED));
|
taskAttempt.attemptId, TaskEventType.T_ATTEMPT_KILLED));
|
||||||
|
return TaskAttemptState.KILLED;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -191,12 +191,12 @@ TaskEventType.T_ADD_SPEC_ATTEMPT, new RedundantScheduleTransition())
|
|||||||
TaskEventType.T_ADD_SPEC_ATTEMPT))
|
TaskEventType.T_ADD_SPEC_ATTEMPT))
|
||||||
|
|
||||||
// Transitions from SUCCEEDED state
|
// Transitions from SUCCEEDED state
|
||||||
.addTransition(TaskState.SUCCEEDED, //only possible for map tasks
|
.addTransition(TaskState.SUCCEEDED,
|
||||||
EnumSet.of(TaskState.SCHEDULED, TaskState.SUCCEEDED, TaskState.FAILED),
|
EnumSet.of(TaskState.SCHEDULED, TaskState.SUCCEEDED, TaskState.FAILED),
|
||||||
TaskEventType.T_ATTEMPT_FAILED, new MapRetroactiveFailureTransition())
|
TaskEventType.T_ATTEMPT_FAILED, new RetroactiveFailureTransition())
|
||||||
.addTransition(TaskState.SUCCEEDED, //only possible for map tasks
|
.addTransition(TaskState.SUCCEEDED,
|
||||||
EnumSet.of(TaskState.SCHEDULED, TaskState.SUCCEEDED),
|
EnumSet.of(TaskState.SCHEDULED, TaskState.SUCCEEDED),
|
||||||
TaskEventType.T_ATTEMPT_KILLED, new MapRetroactiveKilledTransition())
|
TaskEventType.T_ATTEMPT_KILLED, new RetroactiveKilledTransition())
|
||||||
// Ignore-able transitions.
|
// Ignore-able transitions.
|
||||||
.addTransition(
|
.addTransition(
|
||||||
TaskState.SUCCEEDED, TaskState.SUCCEEDED,
|
TaskState.SUCCEEDED, TaskState.SUCCEEDED,
|
||||||
@ -897,7 +897,7 @@ protected TaskState getDefaultState(Task task) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class MapRetroactiveFailureTransition
|
private static class RetroactiveFailureTransition
|
||||||
extends AttemptFailedTransition {
|
extends AttemptFailedTransition {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -912,7 +912,7 @@ public TaskState transition(TaskImpl task, TaskEvent event) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
//verify that this occurs only for map task
|
// a successful REDUCE task should not be overridden
|
||||||
//TODO: consider moving it to MapTaskImpl
|
//TODO: consider moving it to MapTaskImpl
|
||||||
if (!TaskType.MAP.equals(task.getType())) {
|
if (!TaskType.MAP.equals(task.getType())) {
|
||||||
LOG.error("Unexpected event for REDUCE task " + event.getType());
|
LOG.error("Unexpected event for REDUCE task " + event.getType());
|
||||||
@ -938,42 +938,46 @@ protected TaskState getDefaultState(Task task) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class MapRetroactiveKilledTransition implements
|
private static class RetroactiveKilledTransition implements
|
||||||
MultipleArcTransition<TaskImpl, TaskEvent, TaskState> {
|
MultipleArcTransition<TaskImpl, TaskEvent, TaskState> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TaskState transition(TaskImpl task, TaskEvent event) {
|
public TaskState transition(TaskImpl task, TaskEvent event) {
|
||||||
// verify that this occurs only for map task
|
TaskAttemptId attemptId = null;
|
||||||
|
if (event instanceof TaskTAttemptEvent) {
|
||||||
|
TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
|
||||||
|
attemptId = castEvent.getTaskAttemptID();
|
||||||
|
if (task.getState() == TaskState.SUCCEEDED &&
|
||||||
|
!attemptId.equals(task.successfulAttempt)) {
|
||||||
|
// don't allow a different task attempt to override a previous
|
||||||
|
// succeeded state
|
||||||
|
return TaskState.SUCCEEDED;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// a successful REDUCE task should not be overridden
|
||||||
// TODO: consider moving it to MapTaskImpl
|
// TODO: consider moving it to MapTaskImpl
|
||||||
if (!TaskType.MAP.equals(task.getType())) {
|
if (!TaskType.MAP.equals(task.getType())) {
|
||||||
LOG.error("Unexpected event for REDUCE task " + event.getType());
|
LOG.error("Unexpected event for REDUCE task " + event.getType());
|
||||||
task.internalError(event.getType());
|
task.internalError(event.getType());
|
||||||
}
|
}
|
||||||
|
|
||||||
TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
|
// successful attempt is now killed. reschedule
|
||||||
TaskAttemptId attemptId = attemptEvent.getTaskAttemptID();
|
// tell the job about the rescheduling
|
||||||
if(task.successfulAttempt == attemptId) {
|
unSucceed(task);
|
||||||
// successful attempt is now killed. reschedule
|
task.handleTaskAttemptCompletion(attemptId,
|
||||||
// tell the job about the rescheduling
|
TaskAttemptCompletionEventStatus.KILLED);
|
||||||
unSucceed(task);
|
task.eventHandler.handle(new JobMapTaskRescheduledEvent(task.taskId));
|
||||||
task.handleTaskAttemptCompletion(
|
// typically we are here because this map task was run on a bad node and
|
||||||
attemptId,
|
// we want to reschedule it on a different node.
|
||||||
TaskAttemptCompletionEventStatus.KILLED);
|
// Depending on whether there are previous failed attempts or not this
|
||||||
task.eventHandler.handle(new JobMapTaskRescheduledEvent(task.taskId));
|
// can SCHEDULE or RESCHEDULE the container allocate request. If this
|
||||||
// typically we are here because this map task was run on a bad node and
|
// SCHEDULE's then the dataLocal hosts of this taskAttempt will be used
|
||||||
// we want to reschedule it on a different node.
|
// from the map splitInfo. So the bad node might be sent as a location
|
||||||
// Depending on whether there are previous failed attempts or not this
|
// to the RM. But the RM would ignore that just like it would ignore
|
||||||
// can SCHEDULE or RESCHEDULE the container allocate request. If this
|
// currently pending container requests affinitized to bad nodes.
|
||||||
// SCHEDULE's then the dataLocal hosts of this taskAttempt will be used
|
task.addAndScheduleAttempt();
|
||||||
// from the map splitInfo. So the bad node might be sent as a location
|
return TaskState.SCHEDULED;
|
||||||
// to the RM. But the RM would ignore that just like it would ignore
|
|
||||||
// currently pending container requests affinitized to bad nodes.
|
|
||||||
task.addAndScheduleAttempt();
|
|
||||||
return TaskState.SCHEDULED;
|
|
||||||
} else {
|
|
||||||
// nothing to do
|
|
||||||
return TaskState.SUCCEEDED;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -180,7 +180,7 @@ public void testCompletedMapsForReduceSlowstart() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
public void testUpdatedNodes() throws Exception {
|
public void testUpdatedNodes() throws Exception {
|
||||||
int runCount = 0;
|
int runCount = 0;
|
||||||
MRApp app = new MRAppWithHistory(2, 1, false, this.getClass().getName(),
|
MRApp app = new MRAppWithHistory(2, 2, false, this.getClass().getName(),
|
||||||
true, ++runCount);
|
true, ++runCount);
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
// after half of the map completion, reduce will start
|
// after half of the map completion, reduce will start
|
||||||
@ -189,7 +189,7 @@ public void testUpdatedNodes() throws Exception {
|
|||||||
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
||||||
Job job = app.submit(conf);
|
Job job = app.submit(conf);
|
||||||
app.waitForState(job, JobState.RUNNING);
|
app.waitForState(job, JobState.RUNNING);
|
||||||
Assert.assertEquals("Num tasks not correct", 3, job.getTasks().size());
|
Assert.assertEquals("Num tasks not correct", 4, job.getTasks().size());
|
||||||
Iterator<Task> it = job.getTasks().values().iterator();
|
Iterator<Task> it = job.getTasks().values().iterator();
|
||||||
Task mapTask1 = it.next();
|
Task mapTask1 = it.next();
|
||||||
Task mapTask2 = it.next();
|
Task mapTask2 = it.next();
|
||||||
@ -272,18 +272,19 @@ public void testUpdatedNodes() throws Exception {
|
|||||||
|
|
||||||
// rerun
|
// rerun
|
||||||
// in rerun the 1st map will be recovered from previous run
|
// in rerun the 1st map will be recovered from previous run
|
||||||
app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false,
|
app = new MRAppWithHistory(2, 2, false, this.getClass().getName(), false,
|
||||||
++runCount);
|
++runCount);
|
||||||
conf = new Configuration();
|
conf = new Configuration();
|
||||||
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
||||||
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
||||||
job = app.submit(conf);
|
job = app.submit(conf);
|
||||||
app.waitForState(job, JobState.RUNNING);
|
app.waitForState(job, JobState.RUNNING);
|
||||||
Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
|
Assert.assertEquals("No of tasks not correct", 4, job.getTasks().size());
|
||||||
it = job.getTasks().values().iterator();
|
it = job.getTasks().values().iterator();
|
||||||
mapTask1 = it.next();
|
mapTask1 = it.next();
|
||||||
mapTask2 = it.next();
|
mapTask2 = it.next();
|
||||||
Task reduceTask = it.next();
|
Task reduceTask1 = it.next();
|
||||||
|
Task reduceTask2 = it.next();
|
||||||
|
|
||||||
// map 1 will be recovered, no need to send done
|
// map 1 will be recovered, no need to send done
|
||||||
app.waitForState(mapTask1, TaskState.SUCCEEDED);
|
app.waitForState(mapTask1, TaskState.SUCCEEDED);
|
||||||
@ -306,19 +307,36 @@ public void testUpdatedNodes() throws Exception {
|
|||||||
Assert.assertEquals("Expecting 1 more completion events for success", 3,
|
Assert.assertEquals("Expecting 1 more completion events for success", 3,
|
||||||
events.length);
|
events.length);
|
||||||
|
|
||||||
app.waitForState(reduceTask, TaskState.RUNNING);
|
app.waitForState(reduceTask1, TaskState.RUNNING);
|
||||||
TaskAttempt task3Attempt = reduceTask.getAttempts().values().iterator()
|
app.waitForState(reduceTask2, TaskState.RUNNING);
|
||||||
|
|
||||||
|
TaskAttempt task3Attempt = reduceTask1.getAttempts().values().iterator()
|
||||||
.next();
|
.next();
|
||||||
app.getContext()
|
app.getContext()
|
||||||
.getEventHandler()
|
.getEventHandler()
|
||||||
.handle(
|
.handle(
|
||||||
new TaskAttemptEvent(task3Attempt.getID(),
|
new TaskAttemptEvent(task3Attempt.getID(),
|
||||||
TaskAttemptEventType.TA_DONE));
|
TaskAttemptEventType.TA_DONE));
|
||||||
app.waitForState(reduceTask, TaskState.SUCCEEDED);
|
app.waitForState(reduceTask1, TaskState.SUCCEEDED);
|
||||||
|
app.getContext()
|
||||||
|
.getEventHandler()
|
||||||
|
.handle(
|
||||||
|
new TaskAttemptEvent(task3Attempt.getID(),
|
||||||
|
TaskAttemptEventType.TA_KILL));
|
||||||
|
app.waitForState(reduceTask1, TaskState.SUCCEEDED);
|
||||||
|
|
||||||
|
TaskAttempt task4Attempt = reduceTask2.getAttempts().values().iterator()
|
||||||
|
.next();
|
||||||
|
app.getContext()
|
||||||
|
.getEventHandler()
|
||||||
|
.handle(
|
||||||
|
new TaskAttemptEvent(task4Attempt.getID(),
|
||||||
|
TaskAttemptEventType.TA_DONE));
|
||||||
|
app.waitForState(reduceTask2, TaskState.SUCCEEDED);
|
||||||
|
|
||||||
events = job.getTaskAttemptCompletionEvents(0, 100);
|
events = job.getTaskAttemptCompletionEvents(0, 100);
|
||||||
Assert.assertEquals("Expecting 1 more completion events for success", 4,
|
Assert.assertEquals("Expecting 2 more completion events for reduce success",
|
||||||
events.length);
|
5, events.length);
|
||||||
|
|
||||||
// job succeeds
|
// job succeeds
|
||||||
app.waitForState(job, JobState.SUCCEEDED);
|
app.waitForState(job, JobState.SUCCEEDED);
|
||||||
|
@ -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 MockTaskImpl(JobId jobId, int partition,
|
|||||||
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 TaskType getType() {
|
|||||||
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 @@ private class MockTaskAttemptImpl extends TaskAttemptImpl {
|
|||||||
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 TaskAttemptId getAttemptId() {
|
|||||||
|
|
||||||
@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 TaskAttemptState getState() {
|
|||||||
|
|
||||||
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 void run(JobConf job, TaskUmbilicalProtocol umbilical)
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isMapTask() {
|
public boolean isMapTask() {
|
||||||
return true;
|
return (taskType == TaskType.MAP);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
@ -228,13 +236,14 @@ public void setup() {
|
|||||||
when(taskSplitMetaInfo.getLocations()).thenReturn(dataLocations);
|
when(taskSplitMetaInfo.getLocations()).thenReturn(dataLocations);
|
||||||
|
|
||||||
taskAttempts = new ArrayList<MockTaskAttemptImpl>();
|
taskAttempts = new ArrayList<MockTaskAttemptImpl>();
|
||||||
|
}
|
||||||
|
|
||||||
mockTask = new MockTaskImpl(jobId, partition, dispatcher.getEventHandler(),
|
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 @@ private void assertTaskSucceededState() {
|
|||||||
@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 void testInit() {
|
|||||||
*/
|
*/
|
||||||
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 void testScheduleTask() {
|
|||||||
*/
|
*/
|
||||||
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 void testKillScheduledTask() {
|
|||||||
*/
|
*/
|
||||||
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 void testKillScheduledTaskAttempt() {
|
|||||||
*/
|
*/
|
||||||
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 void testLaunchTaskAttempt() {
|
|||||||
*/
|
*/
|
||||||
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 void testKillRunningTaskAttempt() {
|
|||||||
@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 void testTaskProgress() {
|
|||||||
|
|
||||||
@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 void testFailureDuringTaskAttemptCommit() {
|
|||||||
assertTaskSucceededState();
|
assertTaskSucceededState();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
private void runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType failEvent) {
|
||||||
public void testSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
|
|
||||||
TaskId taskId = getNewTaskID();
|
TaskId taskId = getNewTaskID();
|
||||||
scheduleTaskAttempt(taskId);
|
scheduleTaskAttempt(taskId);
|
||||||
launchTaskAttempt(getLastAttempt().getAttemptId());
|
launchTaskAttempt(getLastAttempt().getAttemptId());
|
||||||
@ -489,11 +505,34 @@ public void testSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
|
|||||||
|
|
||||||
// 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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user