MAPREDUCE-6384. Add the last reporting reducer info for too many fetch failure diagnostics. Contributed by Chang Li

This commit is contained in:
Jason Lowe 2015-06-30 21:22:30 +00:00
parent 147e020c7a
commit b6ba56457c
7 changed files with 162 additions and 84 deletions

View File

@ -364,6 +364,9 @@ Release 2.8.0 - UNRELEASED
MAPREDUCE-6408. Queue name and user name should be printed on the job page.
(Siqi Li via gera)
MAPREDUCE-6384. Add the last reporting reducer info for too many fetch
failure diagnostics (Chang Li via jlowe)
OPTIMIZATIONS
BUG FIXES

View File

@ -28,13 +28,15 @@ public class JobTaskAttemptFetchFailureEvent extends JobEvent {
private final TaskAttemptId reduce;
private final List<TaskAttemptId> maps;
private final String hostname;
public JobTaskAttemptFetchFailureEvent(TaskAttemptId reduce,
List<TaskAttemptId> maps) {
List<TaskAttemptId> maps, String host) {
super(reduce.getTaskId().getJobId(),
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE);
this.reduce = reduce;
this.maps = maps;
this.hostname = host;
}
public List<TaskAttemptId> getMaps() {
@ -45,4 +47,7 @@ public TaskAttemptId getReduce() {
return reduce;
}
public String getHost() {
return hostname;
}
}

View File

@ -0,0 +1,50 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
/**
* TaskAttemptTooManyFetchFailureEvent is used for TA_TOO_MANY_FETCH_FAILURE.
*/
public class TaskAttemptTooManyFetchFailureEvent extends TaskAttemptEvent {
private TaskAttemptId reduceID;
private String reduceHostname;
/**
* Create a new TaskAttemptTooManyFetchFailureEvent.
* @param attemptId the id of the mapper task attempt
* @param reduceId the id of the reporting reduce task attempt.
* @param reduceHost the hostname of the reporting reduce task attempt.
*/
public TaskAttemptTooManyFetchFailureEvent(TaskAttemptId attemptId,
TaskAttemptId reduceId, String reduceHost) {
super(attemptId, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE);
this.reduceID = reduceId;
this.reduceHostname = reduceHost;
}
public TaskAttemptId getReduceId() {
return reduceID;
}
public String getReduceHost() {
return reduceHostname;
}
}

View File

@ -103,9 +103,8 @@
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptFetchFailureEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobUpdatedNodesEvent;
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.TaskAttemptKillEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
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;
@ -1914,8 +1913,8 @@ public void transition(JobImpl job, JobEvent event) {
&& failureRate >= job.getMaxAllowedFetchFailuresFraction()) {
LOG.info("Too many fetch-failures for output of task attempt: " +
mapId + " ... raising fetch failure to map");
job.eventHandler.handle(new TaskAttemptEvent(mapId,
TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
job.eventHandler.handle(new TaskAttemptTooManyFetchFailureEvent(mapId,
fetchfailureEvent.getReduce(), fetchfailureEvent.getHost()));
job.fetchFailuresMapping.remove(mapId);
}
}

View File

@ -95,6 +95,7 @@
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
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.launcher.ContainerLauncher;
@ -1916,11 +1917,16 @@ private static class TooManyFetchFailureTransition implements
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
TaskAttemptTooManyFetchFailureEvent fetchFailureEvent =
(TaskAttemptTooManyFetchFailureEvent) event;
// too many fetch failure can only happen for map tasks
Preconditions
.checkArgument(taskAttempt.getID().getTaskId().getTaskType() == TaskType.MAP);
//add to diagnostic
taskAttempt.addDiagnosticInfo("Too Many fetch failures.Failing the attempt");
taskAttempt.addDiagnosticInfo("Too many fetch failures."
+ " Failing the attempt. Last failure reported by " +
fetchFailureEvent.getReduceId() +
" from host " + fetchFailureEvent.getReduceHost());
if (taskAttempt.getLaunchTime() != 0) {
taskAttempt.eventHandler
@ -2225,8 +2231,11 @@ public void transition(TaskAttemptImpl taskAttempt,
//this only will happen in reduce attempt type
if (taskAttempt.reportedStatus.fetchFailedMaps != null &&
taskAttempt.reportedStatus.fetchFailedMaps.size() > 0) {
String hostname = taskAttempt.container == null ? "UNKNOWN"
: taskAttempt.container.getNodeId().getHost();
taskAttempt.eventHandler.handle(new JobTaskAttemptFetchFailureEvent(
taskAttempt.attemptId, taskAttempt.reportedStatus.fetchFailedMaps));
taskAttempt.attemptId, taskAttempt.reportedStatus.fetchFailedMaps,
hostname));
}
}
}

View File

@ -94,9 +94,9 @@ public void testFetchFailure() throws Exception {
app.waitForState(reduceAttempt, TaskAttemptState.RUNNING);
//send 3 fetch failures from reduce to trigger map re execution
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
//wait for map Task state move back to RUNNING
app.waitForState(mapTask, TaskState.RUNNING);
@ -215,9 +215,9 @@ public void testFetchFailureWithRecovery() throws Exception {
app.waitForState(reduceAttempt, TaskAttemptState.RUNNING);
//send 3 fetch failures from reduce to trigger map re execution
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
//wait for map Task state move back to RUNNING
app.waitForState(mapTask, TaskState.RUNNING);
@ -324,8 +324,8 @@ public void testFetchFailureMultipleReduces() throws Exception {
updateStatus(app, reduceAttempt3, Phase.SHUFFLE);
//send 2 fetch failures from reduce to prepare for map re execution
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt, mapAttempt1, "host1");
sendFetchFailure(app, reduceAttempt2, mapAttempt1, "host2");
//We should not re-launch the map task yet
assertEquals(TaskState.SUCCEEDED, mapTask.getState());
@ -333,7 +333,7 @@ public void testFetchFailureMultipleReduces() throws Exception {
updateStatus(app, reduceAttempt3, Phase.REDUCE);
//send 3rd fetch failures from reduce to trigger map re execution
sendFetchFailure(app, reduceAttempt, mapAttempt1);
sendFetchFailure(app, reduceAttempt3, mapAttempt1, "host3");
//wait for map Task state move back to RUNNING
app.waitForState(mapTask, TaskState.RUNNING);
@ -342,6 +342,11 @@ public void testFetchFailureMultipleReduces() throws Exception {
Assert.assertEquals("Map TaskAttempt state not correct",
TaskAttemptState.FAILED, mapAttempt1.getState());
Assert.assertEquals(mapAttempt1.getDiagnostics().get(0),
"Too many fetch failures. Failing the attempt. "
+ "Last failure reported by "
+ reduceAttempt3.getID().toString() + " from host host3");
Assert.assertEquals("Num attempts in Map Task not correct",
2, mapTask.getAttempts().size());
@ -411,7 +416,6 @@ public void testFetchFailureMultipleReduces() throws Exception {
mapEvents[0]);
}
private void updateStatus(MRApp app, TaskAttempt attempt, Phase phase) {
TaskAttemptStatusUpdateEvent.TaskAttemptStatus status = new TaskAttemptStatusUpdateEvent.TaskAttemptStatus();
status.counters = new Counters();
@ -430,11 +434,12 @@ private void updateStatus(MRApp app, TaskAttempt attempt, Phase phase) {
}
private void sendFetchFailure(MRApp app, TaskAttempt reduceAttempt,
TaskAttempt mapAttempt) {
TaskAttempt mapAttempt, String hostname) {
app.getContext().getEventHandler().handle(
new JobTaskAttemptFetchFailureEvent(
reduceAttempt.getID(),
Arrays.asList(new TaskAttemptId[] {mapAttempt.getID()})));
Arrays.asList(new TaskAttemptId[] {mapAttempt.getID()}),
hostname));
}
static class MRAppWithHistory extends MRApp {

View File

@ -70,6 +70,7 @@
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
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.rm.ContainerRequestEvent;
@ -507,6 +508,9 @@ public void testDoubleTooManyFetchFailure() throws Exception {
JobId jobId = MRBuilderUtils.newJobId(appId, 1);
TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
TaskId reduceTaskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.REDUCE);
TaskAttemptId reduceTAId =
MRBuilderUtils.newTaskAttemptId(reduceTaskId, 0);
Path jobFile = mock(Path.class);
MockEventHandler eventHandler = new MockEventHandler();
@ -554,8 +558,8 @@ public void testDoubleTooManyFetchFailure() throws Exception {
assertEquals("Task attempt is not in succeeded state", taImpl.getState(),
TaskAttemptState.SUCCEEDED);
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(attemptId,
reduceTAId, "Host"));
assertEquals("Task attempt is not in FAILED state", taImpl.getState(),
TaskAttemptState.FAILED);
taImpl.handle(new TaskAttemptEvent(attemptId,
@ -741,6 +745,9 @@ public void testFetchFailureAttemptFinishTime() throws Exception{
JobId jobId = MRBuilderUtils.newJobId(appId, 1);
TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
TaskId reducetaskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.REDUCE);
TaskAttemptId reduceTAId =
MRBuilderUtils.newTaskAttemptId(reducetaskId, 0);
Path jobFile = mock(Path.class);
MockEventHandler eventHandler = new MockEventHandler();
@ -792,8 +799,8 @@ public void testFetchFailureAttemptFinishTime() throws Exception{
Long finishTime = taImpl.getFinishTime();
Thread.sleep(5);
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(attemptId,
reduceTAId, "Host"));
assertEquals("Task attempt is not in Too Many Fetch Failure state",
taImpl.getState(), TaskAttemptState.FAILED);