You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ss...@apache.org on 2013/01/22 20:08:25 UTC
svn commit: r1437105 - in
/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project: ./
hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/
hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org...
Author: sseth
Date: Tue Jan 22 19:08:24 2013
New Revision: 1437105
URL: http://svn.apache.org/viewvc?rev=1437105&view=rev
Log:
MAPREDUCE-4946. Fix a performance problem for large jobs by reducing the number of map completion event type conversions. Contributed by Jason Lowe.
Modified:
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryJobs.java
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt Tue Jan 22 19:08:24 2013
@@ -13,6 +13,9 @@ Release 0.23.7 - UNRELEASED
OPTIMIZATIONS
+ MAPREDUCE-4946. Fix a performance problem for large jobs by reducing the
+ number of map completion event type conversions. (Jason Lowe via sseth)
+
BUG FIXES
MAPREDUCE-4278. cannot run two local jobs in parallel from the same
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java Tue Jan 22 19:08:24 2013
@@ -275,14 +275,13 @@ public class TaskAttemptListenerImpl ext
boolean shouldReset = false;
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
- org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent[] events =
+ TaskCompletionEvent[] events =
context.getJob(attemptID.getTaskId().getJobId()).getMapAttemptCompletionEvents(
startIndex, maxEvents);
taskHeartbeatHandler.progressing(attemptID);
- return new MapTaskCompletionEventsUpdate(
- TypeConverter.fromYarn(events), shouldReset);
+ return new MapTaskCompletionEventsUpdate(events, shouldReset);
}
@Override
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java Tue Jan 22 19:08:24 2013
@@ -24,6 +24,7 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
@@ -88,7 +89,7 @@ public interface Job {
TaskAttemptCompletionEvent[]
getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);
- TaskAttemptCompletionEvent[]
+ TaskCompletionEvent[]
getMapAttemptCompletionEvents(int startIndex, int maxEvents);
/**
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java Tue Jan 22 19:08:24 2013
@@ -43,6 +43,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobACLsManager;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.JobContext;
@@ -125,6 +126,9 @@ public class JobImpl implements org.apac
private static final TaskAttemptCompletionEvent[]
EMPTY_TASK_ATTEMPT_COMPLETION_EVENTS = new TaskAttemptCompletionEvent[0];
+ private static final TaskCompletionEvent[]
+ EMPTY_TASK_COMPLETION_EVENTS = new TaskCompletionEvent[0];
+
private static final Log LOG = LogFactory.getLog(JobImpl.class);
//The maximum fraction of fetch failures allowed for a map
@@ -185,7 +189,8 @@ public class JobImpl implements org.apac
private int allowedMapFailuresPercent = 0;
private int allowedReduceFailuresPercent = 0;
private List<TaskAttemptCompletionEvent> taskAttemptCompletionEvents;
- private List<TaskAttemptCompletionEvent> mapAttemptCompletionEvents;
+ private List<TaskCompletionEvent> mapAttemptCompletionEvents;
+ private List<Integer> taskCompletionIdxToMapCompletionIdx;
private final List<String> diagnostics = new ArrayList<String>();
//task/attempt related datastructures
@@ -653,27 +658,31 @@ public class JobImpl implements org.apac
@Override
public TaskAttemptCompletionEvent[] getTaskAttemptCompletionEvents(
int fromEventId, int maxEvents) {
- return getAttemptCompletionEvents(taskAttemptCompletionEvents,
- fromEventId, maxEvents);
+ TaskAttemptCompletionEvent[] events = EMPTY_TASK_ATTEMPT_COMPLETION_EVENTS;
+ readLock.lock();
+ try {
+ if (taskAttemptCompletionEvents.size() > fromEventId) {
+ int actualMax = Math.min(maxEvents,
+ (taskAttemptCompletionEvents.size() - fromEventId));
+ events = taskAttemptCompletionEvents.subList(fromEventId,
+ actualMax + fromEventId).toArray(events);
+ }
+ return events;
+ } finally {
+ readLock.unlock();
+ }
}
@Override
- public TaskAttemptCompletionEvent[] getMapAttemptCompletionEvents(
- int startIndex, int maxEvents) {
- return getAttemptCompletionEvents(mapAttemptCompletionEvents,
- startIndex, maxEvents);
- }
-
- private TaskAttemptCompletionEvent[] getAttemptCompletionEvents(
- List<TaskAttemptCompletionEvent> eventList,
+ public TaskCompletionEvent[] getMapAttemptCompletionEvents(
int startIndex, int maxEvents) {
- TaskAttemptCompletionEvent[] events = EMPTY_TASK_ATTEMPT_COMPLETION_EVENTS;
+ TaskCompletionEvent[] events = EMPTY_TASK_COMPLETION_EVENTS;
readLock.lock();
try {
- if (eventList.size() > startIndex) {
+ if (mapAttemptCompletionEvents.size() > startIndex) {
int actualMax = Math.min(maxEvents,
- (eventList.size() - startIndex));
- events = eventList.subList(startIndex,
+ (mapAttemptCompletionEvents.size() - startIndex));
+ events = mapAttemptCompletionEvents.subList(startIndex,
actualMax + startIndex).toArray(events);
}
return events;
@@ -1178,7 +1187,9 @@ public class JobImpl implements org.apac
new ArrayList<TaskAttemptCompletionEvent>(
job.numMapTasks + job.numReduceTasks + 10);
job.mapAttemptCompletionEvents =
- new ArrayList<TaskAttemptCompletionEvent>(job.numMapTasks + 10);
+ new ArrayList<TaskCompletionEvent>(job.numMapTasks + 10);
+ job.taskCompletionIdxToMapCompletionIdx = new ArrayList<Integer>(
+ job.numMapTasks + job.numReduceTasks + 10);
job.allowedMapFailuresPercent =
job.conf.getInt(MRJobConfig.MAP_FAILURES_MAX_PERCENT, 0);
@@ -1490,17 +1501,35 @@ public class JobImpl implements org.apac
//eventId is equal to index in the arraylist
tce.setEventId(job.taskAttemptCompletionEvents.size());
job.taskAttemptCompletionEvents.add(tce);
+ int mapEventIdx = -1;
if (TaskType.MAP.equals(tce.getAttemptId().getTaskId().getTaskType())) {
- job.mapAttemptCompletionEvents.add(tce);
+ // we track map completions separately from task completions because
+ // - getMapAttemptCompletionEvents uses index ranges specific to maps
+ // - type converting the same events over and over is expensive
+ mapEventIdx = job.mapAttemptCompletionEvents.size();
+ job.mapAttemptCompletionEvents.add(TypeConverter.fromYarn(tce));
}
-
+ job.taskCompletionIdxToMapCompletionIdx.add(mapEventIdx);
+
//make the previous completion event as obsolete if it exists
- Object successEventNo =
+ Integer successEventNo =
job.successAttemptCompletionEventNoMap.remove(tce.getAttemptId().getTaskId());
if (successEventNo != null) {
TaskAttemptCompletionEvent successEvent =
- job.taskAttemptCompletionEvents.get((Integer) successEventNo);
+ job.taskAttemptCompletionEvents.get(successEventNo);
successEvent.setStatus(TaskAttemptCompletionEventStatus.OBSOLETE);
+ int mapCompletionIdx =
+ job.taskCompletionIdxToMapCompletionIdx.get(successEventNo);
+ if (mapCompletionIdx >= 0) {
+ // update the corresponding TaskCompletionEvent for the map
+ TaskCompletionEvent mapEvent =
+ job.mapAttemptCompletionEvents.get(mapCompletionIdx);
+ job.mapAttemptCompletionEvents.set(mapCompletionIdx,
+ new TaskCompletionEvent(mapEvent.getEventId(),
+ mapEvent.getTaskAttemptId(), mapEvent.idWithinJob(),
+ mapEvent.isMapTask(), TaskCompletionEvent.Status.OBSOLETE,
+ mapEvent.getTaskTrackerHttp()));
+ }
}
if (TaskAttemptCompletionEventStatus.SUCCEEDED.equals(tce.getStatus())) {
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java Tue Jan 22 19:08:24 2013
@@ -34,6 +34,7 @@ import java.util.Arrays;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
@@ -153,9 +154,12 @@ public class TestTaskAttemptListenerImpl
.thenReturn(Arrays.copyOfRange(taskEvents, 0, 2));
when(mockJob.getTaskAttemptCompletionEvents(2, 100))
.thenReturn(Arrays.copyOfRange(taskEvents, 2, 4));
- when(mockJob.getMapAttemptCompletionEvents(0, 100)).thenReturn(mapEvents);
- when(mockJob.getMapAttemptCompletionEvents(0, 2)).thenReturn(mapEvents);
- when(mockJob.getMapAttemptCompletionEvents(2, 100)).thenReturn(empty);
+ when(mockJob.getMapAttemptCompletionEvents(0, 100)).thenReturn(
+ TypeConverter.fromYarn(mapEvents));
+ when(mockJob.getMapAttemptCompletionEvents(0, 2)).thenReturn(
+ TypeConverter.fromYarn(mapEvents));
+ when(mockJob.getMapAttemptCompletionEvents(2, 100)).thenReturn(
+ TypeConverter.fromYarn(empty));
AppContext appCtx = mock(AppContext.class);
when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java Tue Jan 22 19:08:24 2013
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobACLsManager;
import org.apache.hadoop.mapred.ShuffleHandler;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.FileSystemCounter;
import org.apache.hadoop.mapreduce.JobACL;
@@ -550,7 +551,7 @@ public class MockJobs extends MockApps {
}
@Override
- public TaskAttemptCompletionEvent[] getMapAttemptCompletionEvents(
+ public TaskCompletionEvent[] getMapAttemptCompletionEvents(
int startIndex, int maxEvents) {
return null;
}
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java Tue Jan 22 19:08:24 2013
@@ -25,8 +25,10 @@ import java.util.Arrays;
import java.util.Iterator;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -150,14 +152,16 @@ public class TestFetchFailure {
Assert.assertEquals("Event status not correct for reduce attempt1",
TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus());
- TaskAttemptCompletionEvent mapEvents[] =
+ TaskCompletionEvent mapEvents[] =
job.getMapAttemptCompletionEvents(0, 2);
+ TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events);
Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length);
Assert.assertArrayEquals("Unexpected map events",
- Arrays.copyOfRange(events, 0, 2), mapEvents);
+ Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents);
mapEvents = job.getMapAttemptCompletionEvents(2, 200);
Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length);
- Assert.assertEquals("Unexpected map event", events[2], mapEvents[0]);
+ Assert.assertEquals("Unexpected map event", convertedEvents[2],
+ mapEvents[0]);
}
/**
@@ -395,14 +399,16 @@ public class TestFetchFailure {
Assert.assertEquals("Event status not correct for reduce attempt1",
TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus());
- TaskAttemptCompletionEvent mapEvents[] =
+ TaskCompletionEvent mapEvents[] =
job.getMapAttemptCompletionEvents(0, 2);
+ TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events);
Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length);
Assert.assertArrayEquals("Unexpected map events",
- Arrays.copyOfRange(events, 0, 2), mapEvents);
+ Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents);
mapEvents = job.getMapAttemptCompletionEvents(2, 200);
Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length);
- Assert.assertEquals("Unexpected map event", events[2], mapEvents[0]);
+ Assert.assertEquals("Unexpected map event", convertedEvents[2],
+ mapEvents[0]);
}
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java Tue Jan 22 19:08:24 2013
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
@@ -440,7 +441,7 @@ public class TestRuntimeEstimators {
}
@Override
- public TaskAttemptCompletionEvent[]
+ public TaskCompletionEvent[]
getMapAttemptCompletionEvents(int startIndex, int maxEvents) {
throw new UnsupportedOperationException("Not supported yet.");
}
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java Tue Jan 22 19:08:24 2013
@@ -36,6 +36,7 @@ import org.apache.commons.logging.LogFac
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobACLsManager;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.TaskID;
@@ -183,13 +184,13 @@ public class CompletedJob implements org
}
@Override
- public synchronized TaskAttemptCompletionEvent[] getMapAttemptCompletionEvents(
+ public synchronized TaskCompletionEvent[] getMapAttemptCompletionEvents(
int startIndex, int maxEvents) {
if (mapCompletionEvents == null) {
constructTaskAttemptCompletionEvents();
}
- return getAttemptCompletionEvents(mapCompletionEvents,
- startIndex, maxEvents);
+ return TypeConverter.fromYarn(getAttemptCompletionEvents(
+ mapCompletionEvents, startIndex, maxEvents));
}
private static TaskAttemptCompletionEvent[] getAttemptCompletionEvents(
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java Tue Jan 22 19:08:24 2013
@@ -25,6 +25,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
@@ -154,7 +155,7 @@ public class PartialJob implements org.a
}
@Override
- public TaskAttemptCompletionEvent[] getMapAttemptCompletionEvents(
+ public TaskCompletionEvent[] getMapAttemptCompletionEvents(
int startIndex, int maxEvents) {
return null;
}
Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryJobs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryJobs.java?rev=1437105&r1=1437104&r2=1437105&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryJobs.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/MockHistoryJobs.java Tue Jan 22 19:08:24 2013
@@ -23,6 +23,7 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -143,7 +144,7 @@ public class MockHistoryJobs extends Moc
}
@Override
- public TaskAttemptCompletionEvent[] getMapAttemptCompletionEvents(
+ public TaskCompletionEvent[] getMapAttemptCompletionEvents(
int startIndex, int maxEvents) {
return job.getMapAttemptCompletionEvents(startIndex, maxEvents);
}