MAPREDUCE-7020. Task timeout in uber mode can crash AM. Contributed by Peter Bacsko
This commit is contained in:
parent
e990904dd5
commit
6eef3d7f1a
@ -369,14 +369,16 @@ public AMFeedback statusUpdate(TaskAttemptID taskAttemptID,
|
||||
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
|
||||
TypeConverter.toYarn(taskAttemptID);
|
||||
|
||||
AMFeedback feedback = new AMFeedback();
|
||||
AtomicReference<TaskAttemptStatus> lastStatusRef =
|
||||
attemptIdToStatus.get(yarnAttemptID);
|
||||
if (lastStatusRef == null) {
|
||||
throw new IllegalStateException("Status update was called"
|
||||
+ " with illegal TaskAttemptId: " + yarnAttemptID);
|
||||
LOG.error("Status update was called with illegal TaskAttemptId: "
|
||||
+ yarnAttemptID);
|
||||
feedback.setTaskFound(false);
|
||||
return feedback;
|
||||
}
|
||||
|
||||
AMFeedback feedback = new AMFeedback();
|
||||
feedback.setTaskFound(true);
|
||||
|
||||
// Propagating preemption to the task if TASK_PREEMPTION is enabled
|
||||
|
@ -487,13 +487,15 @@ public void testCoalescedStatusUpdatesCleared()
|
||||
assertEquals(Phase.REDUCE, status.phase);
|
||||
}
|
||||
|
||||
@Test(expected = IllegalStateException.class)
|
||||
@Test
|
||||
public void testStatusUpdateFromUnregisteredTask()
|
||||
throws IOException, InterruptedException{
|
||||
configureMocks();
|
||||
startListener(false);
|
||||
|
||||
listener.statusUpdate(attemptID, firstReduceStatus);
|
||||
AMFeedback feedback = listener.statusUpdate(attemptID, firstReduceStatus);
|
||||
|
||||
assertFalse(feedback.getTaskFound());
|
||||
}
|
||||
|
||||
private void configureMocks() {
|
||||
|
@ -855,6 +855,9 @@ public void run() {
|
||||
long taskProgressInterval = MRJobConfUtil.
|
||||
getTaskProgressReportInterval(conf);
|
||||
|
||||
boolean uberized = conf.getBoolean("mapreduce.task.uberized",
|
||||
false);
|
||||
|
||||
while (!taskDone.get()) {
|
||||
synchronized (lock) {
|
||||
done = false;
|
||||
@ -893,9 +896,14 @@ public void run() {
|
||||
// if Task Tracker is not aware of our task ID (probably because it died and
|
||||
// came back up), kill ourselves
|
||||
if (!taskFound) {
|
||||
LOG.warn("Parent died. Exiting "+taskId);
|
||||
resetDoneFlag();
|
||||
System.exit(66);
|
||||
if (uberized) {
|
||||
taskDone.set(true);
|
||||
break;
|
||||
} else {
|
||||
LOG.warn("Parent died. Exiting "+taskId);
|
||||
resetDoneFlag();
|
||||
System.exit(66);
|
||||
}
|
||||
}
|
||||
|
||||
// Set a flag that says we should preempt this is read by
|
||||
|
Loading…
Reference in New Issue
Block a user