MAPREDUCE-5746. Job diagnostics can implicate wrong task for a failed job. (Jason Lowe via kasha)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1567666 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Karthik Kambatla 2014-02-12 16:04:18 +00:00
parent 41ec3cce76
commit 308f5faf18
3 changed files with 47 additions and 3 deletions

View File

@ -159,6 +159,9 @@ Release 2.4.0 - UNRELEASED
BUG FIXES BUG FIXES
MAPREDUCE-5746. Job diagnostics can implicate wrong task for a failed job.
(Jason Lowe via kasha)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -353,9 +353,11 @@ private void handleTaskFailedEvent(TaskFailedEvent event) {
taskInfo.error = StringInterner.weakIntern(event.getError()); taskInfo.error = StringInterner.weakIntern(event.getError());
taskInfo.failedDueToAttemptId = event.getFailedAttemptID(); taskInfo.failedDueToAttemptId = event.getFailedAttemptID();
taskInfo.counters = event.getCounters(); taskInfo.counters = event.getCounters();
if (info.errorInfo.isEmpty()) {
info.errorInfo = "Task " + taskInfo.taskId + " failed " + info.errorInfo = "Task " + taskInfo.taskId + " failed " +
taskInfo.attemptsMap.size() + " times "; taskInfo.attemptsMap.size() + " times ";
} }
}
private void handleTaskStartedEvent(TaskStartedEvent event) { private void handleTaskStartedEvent(TaskStartedEvent event) {
TaskInfo taskInfo = new TaskInfo(); TaskInfo taskInfo = new TaskInfo();

View File

@ -40,6 +40,8 @@
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskID;
import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.TypeConverter;
@ -51,7 +53,9 @@
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
import org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent; import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskStartedEvent;
import org.apache.hadoop.mapreduce.v2.api.records.JobId; 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.JobState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -69,7 +73,6 @@
import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo; import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo;
import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory; import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo; import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo; import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.net.DNSToSwitchMapping; import org.apache.hadoop.net.DNSToSwitchMapping;
@ -730,4 +733,40 @@ public void testPartialJob() throws Exception {
assertNull(test.getAMInfos()); assertNull(test.getAMInfos());
} }
@Test
public void testMultipleFailedTasks() throws Exception {
JobHistoryParser parser =
new JobHistoryParser(Mockito.mock(FSDataInputStream.class));
EventReader reader = Mockito.mock(EventReader.class);
final AtomicInteger numEventsRead = new AtomicInteger(0); // Hack!
final org.apache.hadoop.mapreduce.TaskType taskType =
org.apache.hadoop.mapreduce.TaskType.MAP;
final TaskID[] tids = new TaskID[2];
JobID jid = new JobID("1", 1);
tids[0] = new TaskID(jid, taskType, 0);
tids[1] = new TaskID(jid, taskType, 1);
Mockito.when(reader.getNextEvent()).thenAnswer(
new Answer<HistoryEvent>() {
public HistoryEvent answer(InvocationOnMock invocation)
throws IOException {
// send two task start and two task fail events for tasks 0 and 1
int eventId = numEventsRead.getAndIncrement();
TaskID tid = tids[eventId & 0x1];
if (eventId < 2) {
return new TaskStartedEvent(tid, 0, taskType, "");
}
if (eventId < 4) {
TaskFailedEvent tfe = new TaskFailedEvent(tid, 0, taskType,
"failed", "FAILED", null, new Counters());
tfe.setDatum(tfe.getDatum());
return tfe;
}
return null;
}
});
JobInfo info = parser.parse(reader);
assertTrue("Task 0 not implicated",
info.getErrorInfo().contains(tids[0].toString()));
}
} }