|
@@ -23,14 +23,17 @@ import java.io.IOException;
|
|
import java.lang.reflect.Constructor;
|
|
import java.lang.reflect.Constructor;
|
|
import java.lang.reflect.InvocationTargetException;
|
|
import java.lang.reflect.InvocationTargetException;
|
|
import java.security.PrivilegedExceptionAction;
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
|
+import java.util.ArrayList;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
|
|
+import org.apache.commons.io.IOUtils;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FileContext;
|
|
import org.apache.hadoop.fs.FileContext;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -45,6 +48,9 @@ import org.apache.hadoop.mapreduce.OutputFormat;
|
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
import org.apache.hadoop.mapreduce.jobhistory.AMStartedEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.AMStartedEvent;
|
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.EventReader;
|
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.EventType;
|
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
|
@@ -89,6 +95,7 @@ import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
|
+import org.apache.hadoop.util.StringInterner;
|
|
import org.apache.hadoop.yarn.Clock;
|
|
import org.apache.hadoop.yarn.Clock;
|
|
import org.apache.hadoop.yarn.ClusterInfo;
|
|
import org.apache.hadoop.yarn.ClusterInfo;
|
|
import org.apache.hadoop.yarn.SystemClock;
|
|
import org.apache.hadoop.yarn.SystemClock;
|
|
@@ -826,16 +833,21 @@ public class MRAppMaster extends CompositeService {
|
|
@Override
|
|
@Override
|
|
public void start() {
|
|
public void start() {
|
|
|
|
|
|
|
|
+ amInfos = new LinkedList<AMInfo>();
|
|
|
|
+
|
|
// Pull completedTasks etc from recovery
|
|
// Pull completedTasks etc from recovery
|
|
if (inRecovery) {
|
|
if (inRecovery) {
|
|
completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
|
|
completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
|
|
amInfos = recoveryServ.getAMInfos();
|
|
amInfos = recoveryServ.getAMInfos();
|
|
|
|
+ } else {
|
|
|
|
+ // Get the amInfos anyways irrespective of whether recovery is enabled or
|
|
|
|
+ // not IF this is not the first AM generation
|
|
|
|
+ if (appAttemptID.getAttemptId() != 1) {
|
|
|
|
+ amInfos.addAll(readJustAMInfos());
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
- // / Create the AMInfo for the current AppMaster
|
|
|
|
- if (amInfos == null) {
|
|
|
|
- amInfos = new LinkedList<AMInfo>();
|
|
|
|
- }
|
|
|
|
|
|
+ // Current an AMInfo for the current AM generation.
|
|
AMInfo amInfo =
|
|
AMInfo amInfo =
|
|
MRBuilderUtils.newAMInfo(appAttemptID, startTime, containerID, nmHost,
|
|
MRBuilderUtils.newAMInfo(appAttemptID, startTime, containerID, nmHost,
|
|
nmPort, nmHttpPort);
|
|
nmPort, nmHttpPort);
|
|
@@ -893,6 +905,51 @@ public class MRAppMaster extends CompositeService {
|
|
startJobs();
|
|
startJobs();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private List<AMInfo> readJustAMInfos() {
|
|
|
|
+ List<AMInfo> amInfos = new ArrayList<AMInfo>();
|
|
|
|
+ FSDataInputStream inputStream = null;
|
|
|
|
+ try {
|
|
|
|
+ inputStream =
|
|
|
|
+ RecoveryService.getPreviousJobHistoryFileStream(getConfig(),
|
|
|
|
+ appAttemptID);
|
|
|
|
+ EventReader jobHistoryEventReader = new EventReader(inputStream);
|
|
|
|
+
|
|
|
|
+ // All AMInfos are contiguous. Track when the first AMStartedEvent
|
|
|
|
+ // appears.
|
|
|
|
+ boolean amStartedEventsBegan = false;
|
|
|
|
+
|
|
|
|
+ HistoryEvent event;
|
|
|
|
+ while ((event = jobHistoryEventReader.getNextEvent()) != null) {
|
|
|
|
+ if (event.getEventType() == EventType.AM_STARTED) {
|
|
|
|
+ if (!amStartedEventsBegan) {
|
|
|
|
+ // First AMStartedEvent.
|
|
|
|
+ amStartedEventsBegan = true;
|
|
|
|
+ }
|
|
|
|
+ AMStartedEvent amStartedEvent = (AMStartedEvent) event;
|
|
|
|
+ amInfos.add(MRBuilderUtils.newAMInfo(
|
|
|
|
+ amStartedEvent.getAppAttemptId(), amStartedEvent.getStartTime(),
|
|
|
|
+ amStartedEvent.getContainerId(),
|
|
|
|
+ StringInterner.weakIntern(amStartedEvent.getNodeManagerHost()),
|
|
|
|
+ amStartedEvent.getNodeManagerPort(),
|
|
|
|
+ amStartedEvent.getNodeManagerHttpPort()));
|
|
|
|
+ } else if (amStartedEventsBegan) {
|
|
|
|
+ // This means AMStartedEvents began and this event is a
|
|
|
|
+ // non-AMStarted event.
|
|
|
|
+ // No need to continue reading all the other events.
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.warn("Could not parse the old history file. "
|
|
|
|
+ + "Will not have old AMinfos ", e);
|
|
|
|
+ } finally {
|
|
|
|
+ if (inputStream != null) {
|
|
|
|
+ IOUtils.closeQuietly(inputStream);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return amInfos;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This can be overridden to instantiate multiple jobs and create a
|
|
* This can be overridden to instantiate multiple jobs and create a
|
|
* workflow.
|
|
* workflow.
|