MAPREDUCE-3144. Augmented JobHistory with the information needed for serving aggregated logs. Contributed by Siddharth Seth.
svn merge -c r1185976 --ignore-ancestry ../../trunk/ git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1185977 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
dfe8f79cde
commit
e3b9d11da8
|
@ -359,6 +359,9 @@ Release 0.23.0 - Unreleased
|
||||||
from the NodeManager and set MALLOC_ARENA_MAX for all daemons and
|
from the NodeManager and set MALLOC_ARENA_MAX for all daemons and
|
||||||
containers. (Chris Riccomini via acmurthy)
|
containers. (Chris Riccomini via acmurthy)
|
||||||
|
|
||||||
|
MAPREDUCE-3144. Augmented JobHistory with the information needed for
|
||||||
|
serving aggregated logs. (Siddharth Seth via vinodkv)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
MAPREDUCE-2026. Make JobTracker.getJobCounters() and
|
MAPREDUCE-2026. Make JobTracker.getJobCounters() and
|
||||||
|
|
|
@ -275,7 +275,7 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
* @param jobId the jobId.
|
* @param jobId the jobId.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
protected void setupEventWriter(JobId jobId, JobSubmittedEvent jse)
|
protected void setupEventWriter(JobId jobId)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (stagingDirPath == null) {
|
if (stagingDirPath == null) {
|
||||||
LOG.error("Log Directory is null, returning");
|
LOG.error("Log Directory is null, returning");
|
||||||
|
@ -285,9 +285,6 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
MetaInfo oldFi = fileMap.get(jobId);
|
MetaInfo oldFi = fileMap.get(jobId);
|
||||||
Configuration conf = getConfig();
|
Configuration conf = getConfig();
|
||||||
|
|
||||||
long submitTime = oldFi == null ? jse.getSubmitTime() : oldFi
|
|
||||||
.getJobIndexInfo().getSubmitTime();
|
|
||||||
|
|
||||||
// TODO Ideally this should be written out to the job dir
|
// TODO Ideally this should be written out to the job dir
|
||||||
// (.staging/jobid/files - RecoveryService will need to be patched)
|
// (.staging/jobid/files - RecoveryService will need to be patched)
|
||||||
Path historyFile = JobHistoryUtils.getStagingJobHistoryFile(
|
Path historyFile = JobHistoryUtils.getStagingJobHistoryFile(
|
||||||
|
@ -301,6 +298,8 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
String jobName = context.getJob(jobId).getName();
|
String jobName = context.getJob(jobId).getName();
|
||||||
EventWriter writer = (oldFi == null) ? null : oldFi.writer;
|
EventWriter writer = (oldFi == null) ? null : oldFi.writer;
|
||||||
|
|
||||||
|
Path logDirConfPath =
|
||||||
|
JobHistoryUtils.getStagingConfFile(stagingDirPath, jobId, startCount);
|
||||||
if (writer == null) {
|
if (writer == null) {
|
||||||
try {
|
try {
|
||||||
FSDataOutputStream out = stagingDirFS.create(historyFile, true);
|
FSDataOutputStream out = stagingDirFS.create(historyFile, true);
|
||||||
|
@ -312,31 +311,28 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
+ "[" + jobName + "]");
|
+ "[" + jobName + "]");
|
||||||
throw ioe;
|
throw ioe;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
//Write out conf only if the writer isn't already setup.
|
||||||
Path logDirConfPath = null;
|
if (conf != null) {
|
||||||
if (conf != null) {
|
// TODO Ideally this should be written out to the job dir
|
||||||
// TODO Ideally this should be written out to the job dir
|
// (.staging/jobid/files - RecoveryService will need to be patched)
|
||||||
// (.staging/jobid/files - RecoveryService will need to be patched)
|
FSDataOutputStream jobFileOut = null;
|
||||||
logDirConfPath = JobHistoryUtils.getStagingConfFile(stagingDirPath, jobId,
|
try {
|
||||||
startCount);
|
if (logDirConfPath != null) {
|
||||||
FSDataOutputStream jobFileOut = null;
|
jobFileOut = stagingDirFS.create(logDirConfPath, true);
|
||||||
try {
|
conf.writeXml(jobFileOut);
|
||||||
if (logDirConfPath != null) {
|
jobFileOut.close();
|
||||||
jobFileOut = stagingDirFS.create(logDirConfPath, true);
|
}
|
||||||
conf.writeXml(jobFileOut);
|
} catch (IOException e) {
|
||||||
jobFileOut.close();
|
LOG.info("Failed to write the job configuration file", e);
|
||||||
|
throw e;
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.info("Failed to write the job configuration file", e);
|
|
||||||
throw e;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
MetaInfo fi = new MetaInfo(historyFile, logDirConfPath, writer, submitTime,
|
MetaInfo fi = new MetaInfo(historyFile, logDirConfPath, writer,
|
||||||
user, jobName, jobId);
|
user, jobName, jobId);
|
||||||
fi.getJobSummary().setJobId(jobId);
|
fi.getJobSummary().setJobId(jobId);
|
||||||
fi.getJobSummary().setJobSubmitTime(submitTime);
|
|
||||||
fileMap.put(jobId, fi);
|
fileMap.put(jobId, fi);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -368,11 +364,9 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
synchronized (lock) {
|
synchronized (lock) {
|
||||||
|
|
||||||
// If this is JobSubmitted Event, setup the writer
|
// If this is JobSubmitted Event, setup the writer
|
||||||
if (event.getHistoryEvent().getEventType() == EventType.JOB_SUBMITTED) {
|
if (event.getHistoryEvent().getEventType() == EventType.AM_STARTED) {
|
||||||
try {
|
try {
|
||||||
JobSubmittedEvent jobSubmittedEvent =
|
setupEventWriter(event.getJobID());
|
||||||
(JobSubmittedEvent) event.getHistoryEvent();
|
|
||||||
setupEventWriter(event.getJobID(), jobSubmittedEvent);
|
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
|
LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
|
||||||
ioe);
|
ioe);
|
||||||
|
@ -396,6 +390,12 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
throw new YarnException(e);
|
throw new YarnException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (event.getHistoryEvent().getEventType() == EventType.JOB_SUBMITTED) {
|
||||||
|
JobSubmittedEvent jobSubmittedEvent =
|
||||||
|
(JobSubmittedEvent) event.getHistoryEvent();
|
||||||
|
mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime());
|
||||||
|
}
|
||||||
|
|
||||||
// If this is JobFinishedEvent, close the writer and setup the job-index
|
// If this is JobFinishedEvent, close the writer and setup the job-index
|
||||||
if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
|
if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
|
||||||
try {
|
try {
|
||||||
|
@ -436,6 +436,7 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
JobSubmittedEvent jse = (JobSubmittedEvent) event;
|
JobSubmittedEvent jse = (JobSubmittedEvent) event;
|
||||||
summary.setUser(jse.getUserName());
|
summary.setUser(jse.getUserName());
|
||||||
summary.setQueue(jse.getJobQueueName());
|
summary.setQueue(jse.getJobQueueName());
|
||||||
|
summary.setJobSubmitTime(jse.getSubmitTime());
|
||||||
break;
|
break;
|
||||||
case JOB_INITED:
|
case JOB_INITED:
|
||||||
JobInitedEvent jie = (JobInitedEvent) event;
|
JobInitedEvent jie = (JobInitedEvent) event;
|
||||||
|
@ -588,12 +589,12 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
JobIndexInfo jobIndexInfo;
|
JobIndexInfo jobIndexInfo;
|
||||||
JobSummary jobSummary;
|
JobSummary jobSummary;
|
||||||
|
|
||||||
MetaInfo(Path historyFile, Path conf, EventWriter writer, long submitTime,
|
MetaInfo(Path historyFile, Path conf, EventWriter writer,
|
||||||
String user, String jobName, JobId jobId) {
|
String user, String jobName, JobId jobId) {
|
||||||
this.historyFile = historyFile;
|
this.historyFile = historyFile;
|
||||||
this.confFile = conf;
|
this.confFile = conf;
|
||||||
this.writer = writer;
|
this.writer = writer;
|
||||||
this.jobIndexInfo = new JobIndexInfo(submitTime, -1, user, jobName, jobId, -1, -1, null);
|
this.jobIndexInfo = new JobIndexInfo(-1, -1, user, jobName, jobId, -1, -1, null);
|
||||||
this.jobSummary = new JobSummary();
|
this.jobSummary = new JobSummary();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,10 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
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.net.InetSocketAddress;
|
||||||
import java.security.PrivilegedExceptionAction;
|
import java.security.PrivilegedExceptionAction;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
@ -40,6 +43,8 @@ import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
|
||||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||||
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.AMInfo;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.AMStartedEvent;
|
||||||
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
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.JobId;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||||
|
@ -72,6 +77,7 @@ import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner;
|
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanerImpl;
|
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanerImpl;
|
||||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||||
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
@ -82,6 +88,7 @@ import org.apache.hadoop.yarn.YarnException;
|
||||||
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
|
@ -115,14 +122,20 @@ public class MRAppMaster extends CompositeService {
|
||||||
private static final Log LOG = LogFactory.getLog(MRAppMaster.class);
|
private static final Log LOG = LogFactory.getLog(MRAppMaster.class);
|
||||||
|
|
||||||
private Clock clock;
|
private Clock clock;
|
||||||
private final long startTime = System.currentTimeMillis();
|
private final long startTime;
|
||||||
|
private final long appSubmitTime;
|
||||||
private String appName;
|
private String appName;
|
||||||
private final ApplicationAttemptId appAttemptID;
|
private final ApplicationAttemptId appAttemptID;
|
||||||
|
private final ContainerId containerID;
|
||||||
|
private final String nmHost;
|
||||||
|
private final int nmHttpPort;
|
||||||
protected final MRAppMetrics metrics;
|
protected final MRAppMetrics metrics;
|
||||||
private Set<TaskId> completedTasksFromPreviousRun;
|
private Set<TaskId> completedTasksFromPreviousRun;
|
||||||
|
private List<AMInfo> amInfos;
|
||||||
private AppContext context;
|
private AppContext context;
|
||||||
private Dispatcher dispatcher;
|
private Dispatcher dispatcher;
|
||||||
private ClientService clientService;
|
private ClientService clientService;
|
||||||
|
private Recovery recoveryServ;
|
||||||
private ContainerAllocator containerAllocator;
|
private ContainerAllocator containerAllocator;
|
||||||
private ContainerLauncher containerLauncher;
|
private ContainerLauncher containerLauncher;
|
||||||
private TaskCleaner taskCleaner;
|
private TaskCleaner taskCleaner;
|
||||||
|
@ -131,19 +144,29 @@ public class MRAppMaster extends CompositeService {
|
||||||
private JobTokenSecretManager jobTokenSecretManager =
|
private JobTokenSecretManager jobTokenSecretManager =
|
||||||
new JobTokenSecretManager();
|
new JobTokenSecretManager();
|
||||||
private JobEventDispatcher jobEventDispatcher;
|
private JobEventDispatcher jobEventDispatcher;
|
||||||
|
private boolean inRecovery = false;
|
||||||
|
|
||||||
private Job job;
|
private Job job;
|
||||||
private Credentials fsTokens = new Credentials(); // Filled during init
|
private Credentials fsTokens = new Credentials(); // Filled during init
|
||||||
private UserGroupInformation currentUser; // Will be setup during init
|
private UserGroupInformation currentUser; // Will be setup during init
|
||||||
|
|
||||||
public MRAppMaster(ApplicationAttemptId applicationAttemptId) {
|
public MRAppMaster(ApplicationAttemptId applicationAttemptId,
|
||||||
this(applicationAttemptId, new SystemClock());
|
ContainerId containerId, String nmHost, int nmHttpPort, long appSubmitTime) {
|
||||||
|
this(applicationAttemptId, containerId, nmHost, nmHttpPort,
|
||||||
|
new SystemClock(), appSubmitTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
public MRAppMaster(ApplicationAttemptId applicationAttemptId, Clock clock) {
|
public MRAppMaster(ApplicationAttemptId applicationAttemptId,
|
||||||
|
ContainerId containerId, String nmHost, int nmHttpPort, Clock clock,
|
||||||
|
long appSubmitTime) {
|
||||||
super(MRAppMaster.class.getName());
|
super(MRAppMaster.class.getName());
|
||||||
this.clock = clock;
|
this.clock = clock;
|
||||||
|
this.startTime = clock.getTime();
|
||||||
|
this.appSubmitTime = appSubmitTime;
|
||||||
this.appAttemptID = applicationAttemptId;
|
this.appAttemptID = applicationAttemptId;
|
||||||
|
this.containerID = containerId;
|
||||||
|
this.nmHost = nmHost;
|
||||||
|
this.nmHttpPort = nmHttpPort;
|
||||||
this.metrics = MRAppMetrics.create();
|
this.metrics = MRAppMetrics.create();
|
||||||
LOG.info("Created MRAppMaster for application " + applicationAttemptId);
|
LOG.info("Created MRAppMaster for application " + applicationAttemptId);
|
||||||
}
|
}
|
||||||
|
@ -162,11 +185,11 @@ public class MRAppMaster extends CompositeService {
|
||||||
if (conf.getBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, false)
|
if (conf.getBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, false)
|
||||||
&& appAttemptID.getAttemptId() > 1) {
|
&& appAttemptID.getAttemptId() > 1) {
|
||||||
LOG.info("Recovery is enabled. Will try to recover from previous life.");
|
LOG.info("Recovery is enabled. Will try to recover from previous life.");
|
||||||
Recovery recoveryServ = new RecoveryService(appAttemptID, clock);
|
recoveryServ = new RecoveryService(appAttemptID, clock);
|
||||||
addIfService(recoveryServ);
|
addIfService(recoveryServ);
|
||||||
dispatcher = recoveryServ.getDispatcher();
|
dispatcher = recoveryServ.getDispatcher();
|
||||||
clock = recoveryServ.getClock();
|
clock = recoveryServ.getClock();
|
||||||
completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
|
inRecovery = true;
|
||||||
} else {
|
} else {
|
||||||
dispatcher = new AsyncDispatcher();
|
dispatcher = new AsyncDispatcher();
|
||||||
addIfService(dispatcher);
|
addIfService(dispatcher);
|
||||||
|
@ -327,7 +350,8 @@ public class MRAppMaster extends CompositeService {
|
||||||
// create single job
|
// create single job
|
||||||
Job newJob = new JobImpl(appAttemptID, conf, dispatcher.getEventHandler(),
|
Job newJob = new JobImpl(appAttemptID, conf, dispatcher.getEventHandler(),
|
||||||
taskAttemptListener, jobTokenSecretManager, fsTokens, clock,
|
taskAttemptListener, jobTokenSecretManager, fsTokens, clock,
|
||||||
completedTasksFromPreviousRun, metrics, currentUser.getUserName());
|
completedTasksFromPreviousRun, metrics, currentUser.getUserName(),
|
||||||
|
appSubmitTime, amInfos);
|
||||||
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
|
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
|
||||||
|
|
||||||
dispatcher.register(JobFinishEvent.Type.class,
|
dispatcher.register(JobFinishEvent.Type.class,
|
||||||
|
@ -463,6 +487,10 @@ public class MRAppMaster extends CompositeService {
|
||||||
return completedTasksFromPreviousRun;
|
return completedTasksFromPreviousRun;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<AMInfo> getAllAMInfos() {
|
||||||
|
return amInfos;
|
||||||
|
}
|
||||||
|
|
||||||
public ContainerAllocator getContainerAllocator() {
|
public ContainerAllocator getContainerAllocator() {
|
||||||
return containerAllocator;
|
return containerAllocator;
|
||||||
}
|
}
|
||||||
|
@ -617,11 +645,33 @@ public class MRAppMaster extends CompositeService {
|
||||||
@Override
|
@Override
|
||||||
public void start() {
|
public void start() {
|
||||||
|
|
||||||
///////////////////// Create the job itself.
|
// Pull completedTasks etc from recovery
|
||||||
|
if (inRecovery) {
|
||||||
|
completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
|
||||||
|
amInfos = recoveryServ.getAMInfos();
|
||||||
|
}
|
||||||
|
|
||||||
|
// / Create the AMInfo for the current AppMaster
|
||||||
|
if (amInfos == null) {
|
||||||
|
amInfos = new LinkedList<AMInfo>();
|
||||||
|
}
|
||||||
|
AMInfo amInfo =
|
||||||
|
new AMInfo(appAttemptID, startTime, containerID, nmHost, nmHttpPort);
|
||||||
|
amInfos.add(amInfo);
|
||||||
|
|
||||||
|
// /////////////////// Create the job itself.
|
||||||
job = createJob(getConfig());
|
job = createJob(getConfig());
|
||||||
|
|
||||||
// End of creating the job.
|
// End of creating the job.
|
||||||
|
|
||||||
|
// Send out an MR AM inited event for this AM and all previous AMs.
|
||||||
|
for (AMInfo info : amInfos) {
|
||||||
|
dispatcher.getEventHandler().handle(
|
||||||
|
new JobHistoryEvent(job.getID(), new AMStartedEvent(info
|
||||||
|
.getAppAttemptId(), info.getStartTime(), info.getContainerId(),
|
||||||
|
info.getNodeManagerHost(), info.getNodeManagerHttpPort())));
|
||||||
|
}
|
||||||
|
|
||||||
// metrics system init is really init & start.
|
// metrics system init is really init & start.
|
||||||
// It's more test friendly to put it here.
|
// It's more test friendly to put it here.
|
||||||
DefaultMetricsSystem.initialize("MRAppMaster");
|
DefaultMetricsSystem.initialize("MRAppMaster");
|
||||||
|
@ -723,17 +773,39 @@ public class MRAppMaster extends CompositeService {
|
||||||
|
|
||||||
public static void main(String[] args) {
|
public static void main(String[] args) {
|
||||||
try {
|
try {
|
||||||
String applicationAttemptIdStr = System
|
String containerIdStr =
|
||||||
.getenv(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV);
|
System.getenv(ApplicationConstants.AM_CONTAINER_ID_ENV);
|
||||||
if (applicationAttemptIdStr == null) {
|
String nodeHttpAddressStr =
|
||||||
String msg = ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV
|
System.getenv(ApplicationConstants.NM_HTTP_ADDRESS_ENV);
|
||||||
+ " is null";
|
String appSubmitTimeStr =
|
||||||
|
System.getenv(ApplicationConstants.APP_SUBMIT_TIME_ENV);
|
||||||
|
if (containerIdStr == null) {
|
||||||
|
String msg = ApplicationConstants.AM_CONTAINER_ID_ENV + " is null";
|
||||||
LOG.error(msg);
|
LOG.error(msg);
|
||||||
throw new IOException(msg);
|
throw new IOException(msg);
|
||||||
}
|
}
|
||||||
ApplicationAttemptId applicationAttemptId = ConverterUtils
|
if (nodeHttpAddressStr == null) {
|
||||||
.toApplicationAttemptId(applicationAttemptIdStr);
|
String msg = ApplicationConstants.NM_HTTP_ADDRESS_ENV + " is null";
|
||||||
MRAppMaster appMaster = new MRAppMaster(applicationAttemptId);
|
LOG.error(msg);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
if (appSubmitTimeStr == null) {
|
||||||
|
String msg = ApplicationConstants.APP_SUBMIT_TIME_ENV + " is null";
|
||||||
|
LOG.error(msg);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
|
||||||
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
||||||
|
ApplicationAttemptId applicationAttemptId =
|
||||||
|
containerId.getApplicationAttemptId();
|
||||||
|
InetSocketAddress nodeHttpInetAddr =
|
||||||
|
NetUtils.createSocketAddr(nodeHttpAddressStr);
|
||||||
|
long appSubmitTime = Long.parseLong(appSubmitTimeStr);
|
||||||
|
|
||||||
|
MRAppMaster appMaster =
|
||||||
|
new MRAppMaster(applicationAttemptId, containerId,
|
||||||
|
nodeHttpInetAddr.getHostName(), nodeHttpInetAddr.getPort(),
|
||||||
|
appSubmitTime);
|
||||||
Runtime.getRuntime().addShutdownHook(
|
Runtime.getRuntime().addShutdownHook(
|
||||||
new CompositeServiceShutdownHook(appMaster));
|
new CompositeServiceShutdownHook(appMaster));
|
||||||
YarnConfiguration conf = new YarnConfiguration(new JobConf());
|
YarnConfiguration conf = new YarnConfiguration(new JobConf());
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.mapreduce.JobACL;
|
import org.apache.hadoop.mapreduce.JobACL;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
||||||
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.JobReport;
|
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||||
|
@ -68,5 +69,10 @@ public interface Job {
|
||||||
TaskAttemptCompletionEvent[]
|
TaskAttemptCompletionEvent[]
|
||||||
getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);
|
getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return information for MR AppMasters (previously failed and current)
|
||||||
|
*/
|
||||||
|
List<AMInfo> getAMInfos();
|
||||||
|
|
||||||
boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation);
|
boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation);
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobFinishedEvent;
|
import org.apache.hadoop.mapreduce.jobhistory.JobFinishedEvent;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobInfoChangeEvent;
|
import org.apache.hadoop.mapreduce.jobhistory.JobInfoChangeEvent;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobInitedEvent;
|
import org.apache.hadoop.mapreduce.jobhistory.JobInitedEvent;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
|
import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
|
||||||
|
@ -136,6 +137,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
private final String username;
|
private final String username;
|
||||||
private final Map<JobACL, AccessControlList> jobACLs;
|
private final Map<JobACL, AccessControlList> jobACLs;
|
||||||
private final Set<TaskId> completedTasksFromPreviousRun;
|
private final Set<TaskId> completedTasksFromPreviousRun;
|
||||||
|
private final List<AMInfo> amInfos;
|
||||||
private final Lock readLock;
|
private final Lock readLock;
|
||||||
private final Lock writeLock;
|
private final Lock writeLock;
|
||||||
private final JobId jobId;
|
private final JobId jobId;
|
||||||
|
@ -148,6 +150,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
private final EventHandler eventHandler;
|
private final EventHandler eventHandler;
|
||||||
private final MRAppMetrics metrics;
|
private final MRAppMetrics metrics;
|
||||||
private final String userName;
|
private final String userName;
|
||||||
|
private final long appSubmitTime;
|
||||||
|
|
||||||
private boolean lazyTasksCopyNeeded = false;
|
private boolean lazyTasksCopyNeeded = false;
|
||||||
private volatile Map<TaskId, Task> tasks = new LinkedHashMap<TaskId, Task>();
|
private volatile Map<TaskId, Task> tasks = new LinkedHashMap<TaskId, Task>();
|
||||||
|
@ -354,7 +357,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
private int failedReduceTaskCount = 0;
|
private int failedReduceTaskCount = 0;
|
||||||
private int killedMapTaskCount = 0;
|
private int killedMapTaskCount = 0;
|
||||||
private int killedReduceTaskCount = 0;
|
private int killedReduceTaskCount = 0;
|
||||||
private long submitTime;
|
|
||||||
private long startTime;
|
private long startTime;
|
||||||
private long finishTime;
|
private long finishTime;
|
||||||
private float setupProgress;
|
private float setupProgress;
|
||||||
|
@ -370,7 +372,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
JobTokenSecretManager jobTokenSecretManager,
|
JobTokenSecretManager jobTokenSecretManager,
|
||||||
Credentials fsTokenCredentials, Clock clock,
|
Credentials fsTokenCredentials, Clock clock,
|
||||||
Set<TaskId> completedTasksFromPreviousRun, MRAppMetrics metrics,
|
Set<TaskId> completedTasksFromPreviousRun, MRAppMetrics metrics,
|
||||||
String userName) {
|
String userName, long appSubmitTime, List<AMInfo> amInfos) {
|
||||||
this.applicationAttemptId = applicationAttemptId;
|
this.applicationAttemptId = applicationAttemptId;
|
||||||
this.jobId = recordFactory.newRecordInstance(JobId.class);
|
this.jobId = recordFactory.newRecordInstance(JobId.class);
|
||||||
this.jobName = conf.get(JobContext.JOB_NAME, "<missing job name>");
|
this.jobName = conf.get(JobContext.JOB_NAME, "<missing job name>");
|
||||||
|
@ -378,7 +380,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
this.metrics = metrics;
|
this.metrics = metrics;
|
||||||
this.clock = clock;
|
this.clock = clock;
|
||||||
this.completedTasksFromPreviousRun = completedTasksFromPreviousRun;
|
this.completedTasksFromPreviousRun = completedTasksFromPreviousRun;
|
||||||
|
this.amInfos = amInfos;
|
||||||
this.userName = userName;
|
this.userName = userName;
|
||||||
|
this.appSubmitTime = appSubmitTime;
|
||||||
ApplicationId applicationId = applicationAttemptId.getApplicationId();
|
ApplicationId applicationId = applicationAttemptId.getApplicationId();
|
||||||
jobId.setAppId(applicationId);
|
jobId.setAppId(applicationId);
|
||||||
jobId.setId(applicationId.getId());
|
jobId.setId(applicationId.getId());
|
||||||
|
@ -806,6 +810,11 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
public Map<JobACL, AccessControlList> getJobACLs() {
|
public Map<JobACL, AccessControlList> getJobACLs() {
|
||||||
return Collections.unmodifiableMap(jobACLs);
|
return Collections.unmodifiableMap(jobACLs);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AMInfo> getAMInfos() {
|
||||||
|
return amInfos;
|
||||||
|
}
|
||||||
|
|
||||||
public static class InitTransition
|
public static class InitTransition
|
||||||
implements MultipleArcTransition<JobImpl, JobEvent, JobState> {
|
implements MultipleArcTransition<JobImpl, JobEvent, JobState> {
|
||||||
|
@ -819,7 +828,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public JobState transition(JobImpl job, JobEvent event) {
|
public JobState transition(JobImpl job, JobEvent event) {
|
||||||
job.submitTime = job.clock.getTime();
|
|
||||||
job.metrics.submittedJob(job);
|
job.metrics.submittedJob(job);
|
||||||
job.metrics.preparingJob(job);
|
job.metrics.preparingJob(job);
|
||||||
try {
|
try {
|
||||||
|
@ -830,7 +838,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
JobSubmittedEvent jse = new JobSubmittedEvent(job.oldJobId,
|
JobSubmittedEvent jse = new JobSubmittedEvent(job.oldJobId,
|
||||||
job.conf.get(MRJobConfig.JOB_NAME, "test"),
|
job.conf.get(MRJobConfig.JOB_NAME, "test"),
|
||||||
job.conf.get(MRJobConfig.USER_NAME, "mapred"),
|
job.conf.get(MRJobConfig.USER_NAME, "mapred"),
|
||||||
job.submitTime,
|
job.appSubmitTime,
|
||||||
job.remoteJobConfFile.toString(),
|
job.remoteJobConfFile.toString(),
|
||||||
job.jobACLs, job.conf.get(MRJobConfig.QUEUE_NAME, "default"));
|
job.jobACLs, job.conf.get(MRJobConfig.QUEUE_NAME, "default"));
|
||||||
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
|
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
|
||||||
|
@ -1152,7 +1160,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||||
job.isUber()); //Will transition to state running. Currently in INITED
|
job.isUber()); //Will transition to state running. Currently in INITED
|
||||||
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jie));
|
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jie));
|
||||||
JobInfoChangeEvent jice = new JobInfoChangeEvent(job.oldJobId,
|
JobInfoChangeEvent jice = new JobInfoChangeEvent(job.oldJobId,
|
||||||
job.submitTime, job.startTime);
|
job.appSubmitTime, job.startTime);
|
||||||
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jice));
|
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jice));
|
||||||
job.metrics.runningJob(job);
|
job.metrics.runningJob(job);
|
||||||
|
|
||||||
|
|
|
@ -894,15 +894,20 @@ public abstract class TaskAttemptImpl implements
|
||||||
return jce;
|
return jce;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static TaskAttemptUnsuccessfulCompletionEvent createTaskAttemptUnsuccessfulCompletionEvent(
|
private static
|
||||||
TaskAttemptImpl taskAttempt, TaskAttemptState attemptState) {
|
TaskAttemptUnsuccessfulCompletionEvent
|
||||||
TaskAttemptUnsuccessfulCompletionEvent tauce = new TaskAttemptUnsuccessfulCompletionEvent(
|
createTaskAttemptUnsuccessfulCompletionEvent(TaskAttemptImpl taskAttempt,
|
||||||
TypeConverter.fromYarn(taskAttempt.attemptId),
|
TaskAttemptState attemptState) {
|
||||||
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()),
|
TaskAttemptUnsuccessfulCompletionEvent tauce =
|
||||||
attemptState.toString(), taskAttempt.finishTime,
|
new TaskAttemptUnsuccessfulCompletionEvent(
|
||||||
taskAttempt.nodeHostName == null ? "UNKNOWN" : taskAttempt.nodeHostName,
|
TypeConverter.fromYarn(taskAttempt.attemptId),
|
||||||
StringUtils.join(LINE_SEPARATOR, taskAttempt.getDiagnostics()),
|
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId()
|
||||||
taskAttempt.getProgressSplitBlock().burst());
|
.getTaskType()), attemptState.toString(),
|
||||||
|
taskAttempt.finishTime,
|
||||||
|
taskAttempt.containerMgrAddress == null ? "UNKNOWN"
|
||||||
|
: taskAttempt.containerMgrAddress, StringUtils.join(
|
||||||
|
LINE_SEPARATOR, taskAttempt.getDiagnostics()), taskAttempt
|
||||||
|
.getProgressSplitBlock().burst());
|
||||||
return tauce;
|
return tauce;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1120,11 +1125,15 @@ public abstract class TaskAttemptImpl implements
|
||||||
, 1);
|
, 1);
|
||||||
taskAttempt.eventHandler.handle(jce);
|
taskAttempt.eventHandler.handle(jce);
|
||||||
|
|
||||||
|
LOG.info("TaskAttempt: [" + taskAttempt.attemptId
|
||||||
|
+ "] using containerId: [" + taskAttempt.containerID + " on NM: ["
|
||||||
|
+ taskAttempt.containerMgrAddress + "]");
|
||||||
TaskAttemptStartedEvent tase =
|
TaskAttemptStartedEvent tase =
|
||||||
new TaskAttemptStartedEvent(TypeConverter.fromYarn(taskAttempt.attemptId),
|
new TaskAttemptStartedEvent(TypeConverter.fromYarn(taskAttempt.attemptId),
|
||||||
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()),
|
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()),
|
||||||
taskAttempt.launchTime,
|
taskAttempt.launchTime,
|
||||||
nodeHttpInetAddr.getHostName(), nodeHttpInetAddr.getPort(), taskAttempt.shufflePort);
|
nodeHttpInetAddr.getHostName(), nodeHttpInetAddr.getPort(),
|
||||||
|
taskAttempt.shufflePort, taskAttempt.containerID);
|
||||||
taskAttempt.eventHandler.handle
|
taskAttempt.eventHandler.handle
|
||||||
(new JobHistoryEvent(taskAttempt.attemptId.getTaskId().getJobId(), tase));
|
(new JobHistoryEvent(taskAttempt.attemptId.getTaskId().getJobId(), tase));
|
||||||
taskAttempt.eventHandler.handle
|
taskAttempt.eventHandler.handle
|
||||||
|
@ -1236,7 +1245,8 @@ public abstract class TaskAttemptImpl implements
|
||||||
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
|
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
|
||||||
state.toString(),
|
state.toString(),
|
||||||
this.reportedStatus.mapFinishTime,
|
this.reportedStatus.mapFinishTime,
|
||||||
finishTime, this.nodeHostName == null ? "UNKNOWN" : this.nodeHostName,
|
finishTime, this.containerMgrAddress == null ? "UNKNOWN"
|
||||||
|
: this.containerMgrAddress,
|
||||||
this.reportedStatus.stateString,
|
this.reportedStatus.stateString,
|
||||||
TypeConverter.fromYarn(getCounters()),
|
TypeConverter.fromYarn(getCounters()),
|
||||||
getProgressSplitBlock().burst());
|
getProgressSplitBlock().burst());
|
||||||
|
@ -1249,7 +1259,8 @@ public abstract class TaskAttemptImpl implements
|
||||||
state.toString(),
|
state.toString(),
|
||||||
this.reportedStatus.shuffleFinishTime,
|
this.reportedStatus.shuffleFinishTime,
|
||||||
this.reportedStatus.sortFinishTime,
|
this.reportedStatus.sortFinishTime,
|
||||||
finishTime, this.nodeHostName == null ? "UNKNOWN" : this.nodeHostName,
|
finishTime, this.containerMgrAddress == null ? "UNKNOWN"
|
||||||
|
: this.containerMgrAddress,
|
||||||
this.reportedStatus.stateString,
|
this.reportedStatus.stateString,
|
||||||
TypeConverter.fromYarn(getCounters()),
|
TypeConverter.fromYarn(getCounters()),
|
||||||
getProgressSplitBlock().burst());
|
getProgressSplitBlock().burst());
|
||||||
|
|
|
@ -18,8 +18,10 @@
|
||||||
|
|
||||||
package org.apache.hadoop.mapreduce.v2.app.recover;
|
package org.apache.hadoop.mapreduce.v2.app.recover;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||||
import org.apache.hadoop.yarn.Clock;
|
import org.apache.hadoop.yarn.Clock;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
|
@ -31,4 +33,6 @@ public interface Recovery {
|
||||||
Clock getClock();
|
Clock getClock();
|
||||||
|
|
||||||
Set<TaskId> getCompletedTasks();
|
Set<TaskId> getCompletedTasks();
|
||||||
|
|
||||||
|
List<AMInfo> getAMInfos();
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.recover;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -34,6 +35,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
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.v2.api.records.Phase;
|
import org.apache.hadoop.mapreduce.v2.api.records.Phase;
|
||||||
|
@ -148,6 +150,14 @@ public class RecoveryService extends CompositeService implements Recovery {
|
||||||
public Set<TaskId> getCompletedTasks() {
|
public Set<TaskId> getCompletedTasks() {
|
||||||
return completedTasks.keySet();
|
return completedTasks.keySet();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AMInfo> getAMInfos() {
|
||||||
|
if (jobInfo == null || jobInfo.getAMInfos() == null) {
|
||||||
|
return new LinkedList<AMInfo>();
|
||||||
|
}
|
||||||
|
return new LinkedList<AMInfo>(jobInfo.getAMInfos());
|
||||||
|
}
|
||||||
|
|
||||||
private void parse() throws IOException {
|
private void parse() throws IOException {
|
||||||
// TODO: parse history file based on startCount
|
// TODO: parse history file based on startCount
|
||||||
|
@ -351,15 +361,16 @@ public class RecoveryService extends CompositeService implements Recovery {
|
||||||
private void sendAssignedEvent(TaskAttemptId yarnAttemptID,
|
private void sendAssignedEvent(TaskAttemptId yarnAttemptID,
|
||||||
TaskAttemptInfo attemptInfo) {
|
TaskAttemptInfo attemptInfo) {
|
||||||
LOG.info("Sending assigned event to " + yarnAttemptID);
|
LOG.info("Sending assigned event to " + yarnAttemptID);
|
||||||
ContainerId cId = recordFactory
|
ContainerId cId = attemptInfo.getContainerId();
|
||||||
.newRecordInstance(ContainerId.class);
|
|
||||||
Container container = recordFactory
|
Container container = recordFactory
|
||||||
.newRecordInstance(Container.class);
|
.newRecordInstance(Container.class);
|
||||||
container.setId(cId);
|
container.setId(cId);
|
||||||
container.setNodeId(recordFactory
|
container.setNodeId(recordFactory
|
||||||
.newRecordInstance(NodeId.class));
|
.newRecordInstance(NodeId.class));
|
||||||
|
// NodeId can be obtained from TaskAttemptInfo.hostname - but this will
|
||||||
|
// eventually contain rack info.
|
||||||
container.setContainerToken(null);
|
container.setContainerToken(null);
|
||||||
container.setNodeHttpAddress(attemptInfo.getHostname() + ":" +
|
container.setNodeHttpAddress(attemptInfo.getTrackerName() + ":" +
|
||||||
attemptInfo.getHttpPort());
|
attemptInfo.getHttpPort());
|
||||||
actualHandler.handle(new TaskAttemptContainerAssignedEvent(yarnAttemptID,
|
actualHandler.handle(new TaskAttemptContainerAssignedEvent(yarnAttemptID,
|
||||||
container));
|
container));
|
||||||
|
|
|
@ -77,6 +77,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
import org.apache.hadoop.yarn.service.Service;
|
import org.apache.hadoop.yarn.service.Service;
|
||||||
import org.apache.hadoop.yarn.state.StateMachine;
|
import org.apache.hadoop.yarn.state.StateMachine;
|
||||||
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
||||||
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -118,10 +119,20 @@ public class MRApp extends MRAppMaster {
|
||||||
applicationAttemptId.setAttemptId(startCount);
|
applicationAttemptId.setAttemptId(startCount);
|
||||||
return applicationAttemptId;
|
return applicationAttemptId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static ContainerId getContainerId(ApplicationId applicationId,
|
||||||
|
int startCount) {
|
||||||
|
ApplicationAttemptId appAttemptId =
|
||||||
|
getApplicationAttemptId(applicationId, startCount);
|
||||||
|
ContainerId containerId =
|
||||||
|
BuilderUtils.newContainerId(appAttemptId, startCount);
|
||||||
|
return containerId;
|
||||||
|
}
|
||||||
|
|
||||||
public MRApp(int maps, int reduces, boolean autoComplete, String testName,
|
public MRApp(int maps, int reduces, boolean autoComplete, String testName,
|
||||||
boolean cleanOnStart, int startCount) {
|
boolean cleanOnStart, int startCount) {
|
||||||
super(getApplicationAttemptId(applicationId, startCount));
|
super(getApplicationAttemptId(applicationId, startCount), getContainerId(
|
||||||
|
applicationId, startCount), "testhost", 3333, System.currentTimeMillis());
|
||||||
this.testWorkDir = new File("target", testName);
|
this.testWorkDir = new File("target", testName);
|
||||||
testAbsPath = new Path(testWorkDir.getAbsolutePath());
|
testAbsPath = new Path(testWorkDir.getAbsolutePath());
|
||||||
LOG.info("PathUsed: " + testAbsPath);
|
LOG.info("PathUsed: " + testAbsPath);
|
||||||
|
@ -405,10 +416,10 @@ public class MRApp extends MRAppMaster {
|
||||||
public TestJob(Configuration conf, ApplicationId applicationId,
|
public TestJob(Configuration conf, ApplicationId applicationId,
|
||||||
EventHandler eventHandler, TaskAttemptListener taskAttemptListener,
|
EventHandler eventHandler, TaskAttemptListener taskAttemptListener,
|
||||||
Clock clock, String user) {
|
Clock clock, String user) {
|
||||||
super(getApplicationAttemptId(applicationId, getStartCount()),
|
super(getApplicationAttemptId(applicationId, getStartCount()), conf,
|
||||||
conf, eventHandler, taskAttemptListener,
|
eventHandler, taskAttemptListener, new JobTokenSecretManager(),
|
||||||
new JobTokenSecretManager(), new Credentials(), clock,
|
new Credentials(), clock, getCompletedTaskFromPreviousRun(), metrics,
|
||||||
getCompletedTaskFromPreviousRun(), metrics, user);
|
user, System.currentTimeMillis(), getAllAMInfos());
|
||||||
|
|
||||||
// This "this leak" is okay because the retained pointer is in an
|
// This "this leak" is okay because the retained pointer is in an
|
||||||
// instance variable.
|
// instance variable.
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.FileSystemCounter;
|
||||||
import org.apache.hadoop.mapreduce.JobACL;
|
import org.apache.hadoop.mapreduce.JobACL;
|
||||||
import org.apache.hadoop.mapreduce.JobCounter;
|
import org.apache.hadoop.mapreduce.JobCounter;
|
||||||
import org.apache.hadoop.mapreduce.TaskCounter;
|
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
||||||
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.JobReport;
|
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||||
|
@ -488,6 +489,11 @@ public class MockJobs extends MockApps {
|
||||||
public Map<JobACL, AccessControlList> getJobACLs() {
|
public Map<JobACL, AccessControlList> getJobACLs() {
|
||||||
return Collections.<JobACL, AccessControlList>emptyMap();
|
return Collections.<JobACL, AccessControlList>emptyMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AMInfo> getAMInfos() {
|
||||||
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -36,11 +37,15 @@ public class TestMRAppMaster {
|
||||||
public void testMRAppMasterForDifferentUser() throws IOException,
|
public void testMRAppMasterForDifferentUser() throws IOException,
|
||||||
InterruptedException {
|
InterruptedException {
|
||||||
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000001";
|
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000001";
|
||||||
|
String containerIdStr = "container_1317529182569_0004_000001_1";
|
||||||
String stagingDir = "/tmp/staging";
|
String stagingDir = "/tmp/staging";
|
||||||
String userName = "TestAppMasterUser";
|
String userName = "TestAppMasterUser";
|
||||||
ApplicationAttemptId applicationAttemptId = ConverterUtils
|
ApplicationAttemptId applicationAttemptId = ConverterUtils
|
||||||
.toApplicationAttemptId(applicationAttemptIdStr);
|
.toApplicationAttemptId(applicationAttemptIdStr);
|
||||||
MRAppMasterTest appMaster = new MRAppMasterTest(applicationAttemptId);
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
||||||
|
MRAppMasterTest appMaster =
|
||||||
|
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1,
|
||||||
|
System.currentTimeMillis());
|
||||||
YarnConfiguration conf = new YarnConfiguration();
|
YarnConfiguration conf = new YarnConfiguration();
|
||||||
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
|
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
|
||||||
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
||||||
|
@ -54,8 +59,9 @@ class MRAppMasterTest extends MRAppMaster {
|
||||||
Path stagingDirPath;
|
Path stagingDirPath;
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
|
|
||||||
public MRAppMasterTest(ApplicationAttemptId applicationAttemptId) {
|
public MRAppMasterTest(ApplicationAttemptId applicationAttemptId,
|
||||||
super(applicationAttemptId);
|
ContainerId containerId, String host, int port, long submitTime) {
|
||||||
|
super(applicationAttemptId, containerId, host, port, submitTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -340,8 +340,8 @@ public class TestRMContainerAllocator {
|
||||||
|
|
||||||
public FakeJob(ApplicationAttemptId appAttemptID, Configuration conf,
|
public FakeJob(ApplicationAttemptId appAttemptID, Configuration conf,
|
||||||
int numMaps, int numReduces) {
|
int numMaps, int numReduces) {
|
||||||
super(appAttemptID, conf, null, null, null, null, null, null, null,
|
super(appAttemptID, conf, null, null, null, null, null, null, null, null,
|
||||||
null);
|
System.currentTimeMillis(), null);
|
||||||
this.jobId = MRBuilderUtils
|
this.jobId = MRBuilderUtils
|
||||||
.newJobId(appAttemptID.getApplicationId(), 0);
|
.newJobId(appAttemptID.getApplicationId(), 0);
|
||||||
this.numMaps = numMaps;
|
this.numMaps = numMaps;
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||||
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.AMInfo;
|
||||||
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.TaskAttemptState;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||||
|
@ -46,6 +47,7 @@ public class TestRecovery {
|
||||||
@Test
|
@Test
|
||||||
public void testCrashed() throws Exception {
|
public void testCrashed() throws Exception {
|
||||||
int runCount = 0;
|
int runCount = 0;
|
||||||
|
long am1StartTimeEst = System.currentTimeMillis();
|
||||||
MRApp app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), true, ++runCount);
|
MRApp app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), true, ++runCount);
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
||||||
|
@ -126,9 +128,10 @@ public class TestRecovery {
|
||||||
|
|
||||||
//stop the app
|
//stop the app
|
||||||
app.stop();
|
app.stop();
|
||||||
|
|
||||||
//rerun
|
//rerun
|
||||||
//in rerun the 1st map will be recovered from previous run
|
//in rerun the 1st map will be recovered from previous run
|
||||||
|
long am2StartTimeEst = System.currentTimeMillis();
|
||||||
app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false, ++runCount);
|
app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false, ++runCount);
|
||||||
conf = new Configuration();
|
conf = new Configuration();
|
||||||
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
||||||
|
@ -178,8 +181,27 @@ public class TestRecovery {
|
||||||
task1StartTime, mapTask1.getReport().getStartTime());
|
task1StartTime, mapTask1.getReport().getStartTime());
|
||||||
Assert.assertEquals("Task Finish time not correct",
|
Assert.assertEquals("Task Finish time not correct",
|
||||||
task1FinishTime, mapTask1.getReport().getFinishTime());
|
task1FinishTime, mapTask1.getReport().getFinishTime());
|
||||||
|
Assert.assertEquals(2, job.getAMInfos().size());
|
||||||
|
int attemptNum = 1;
|
||||||
|
// Verify AMInfo
|
||||||
|
for (AMInfo amInfo : job.getAMInfos()) {
|
||||||
|
Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId()
|
||||||
|
.getAttemptId());
|
||||||
|
Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
|
||||||
|
.getApplicationAttemptId());
|
||||||
|
Assert.assertEquals("testhost", amInfo.getNodeManagerHost());
|
||||||
|
Assert.assertEquals(3333, amInfo.getNodeManagerHttpPort());
|
||||||
|
}
|
||||||
|
long am1StartTimeReal = job.getAMInfos().get(0).getStartTime();
|
||||||
|
long am2StartTimeReal = job.getAMInfos().get(1).getStartTime();
|
||||||
|
Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst
|
||||||
|
&& am1StartTimeReal <= am2StartTimeEst);
|
||||||
|
Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst
|
||||||
|
&& am2StartTimeReal <= System.currentTimeMillis());
|
||||||
|
// TODO Add verification of additional data from jobHistory - whatever was
|
||||||
|
// available in the failed attempt should be available here
|
||||||
}
|
}
|
||||||
|
|
||||||
class MRAppWithHistory extends MRApp {
|
class MRAppWithHistory extends MRApp {
|
||||||
public MRAppWithHistory(int maps, int reduces, boolean autoComplete,
|
public MRAppWithHistory(int maps, int reduces, boolean autoComplete,
|
||||||
String testName, boolean cleanOnStart, int startCount) {
|
String testName, boolean cleanOnStart, int startCount) {
|
||||||
|
|
|
@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.mapreduce.JobACL;
|
import org.apache.hadoop.mapreduce.JobACL;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
||||||
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.JobReport;
|
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||||
|
@ -473,6 +474,11 @@ public class TestRuntimeEstimators {
|
||||||
public Map<JobACL, AccessControlList> getJobACLs() {
|
public Map<JobACL, AccessControlList> getJobACLs() {
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AMInfo> getAMInfos() {
|
||||||
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
|
||||||
|
@ -81,7 +82,9 @@ import org.junit.Test;
|
||||||
private class TestMRApp extends MRAppMaster {
|
private class TestMRApp extends MRAppMaster {
|
||||||
|
|
||||||
public TestMRApp(ApplicationAttemptId applicationAttemptId) {
|
public TestMRApp(ApplicationAttemptId applicationAttemptId) {
|
||||||
super(applicationAttemptId);
|
super(applicationAttemptId, BuilderUtils.newContainerId(
|
||||||
|
applicationAttemptId, 1), "testhost", 3333, System
|
||||||
|
.currentTimeMillis());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -69,6 +69,16 @@
|
||||||
]
|
]
|
||||||
},
|
},
|
||||||
|
|
||||||
|
{"type": "record", "name": "AMStarted",
|
||||||
|
"fields": [
|
||||||
|
{"name": "applicationAttemptId", "type": "string"},
|
||||||
|
{"name": "startTime", "type": "long"},
|
||||||
|
{"name": "containerId", "type": "string"},
|
||||||
|
{"name": "nodeManagerHost", "type": "string"},
|
||||||
|
{"name": "nodeManagerHttpPort", "type": "int"}
|
||||||
|
]
|
||||||
|
},
|
||||||
|
|
||||||
{"type": "record", "name": "JobSubmitted",
|
{"type": "record", "name": "JobSubmitted",
|
||||||
"fields": [
|
"fields": [
|
||||||
{"name": "jobid", "type": "string"},
|
{"name": "jobid", "type": "string"},
|
||||||
|
@ -174,7 +184,8 @@
|
||||||
{"name": "startTime", "type": "long"},
|
{"name": "startTime", "type": "long"},
|
||||||
{"name": "trackerName", "type": "string"},
|
{"name": "trackerName", "type": "string"},
|
||||||
{"name": "httpPort", "type": "int"},
|
{"name": "httpPort", "type": "int"},
|
||||||
{"name": "shufflePort", "type": "int"}
|
{"name": "shufflePort", "type": "int"},
|
||||||
|
{"name": "containerId", "type": "string"}
|
||||||
]
|
]
|
||||||
},
|
},
|
||||||
|
|
||||||
|
@ -260,7 +271,8 @@
|
||||||
"CLEANUP_ATTEMPT_STARTED",
|
"CLEANUP_ATTEMPT_STARTED",
|
||||||
"CLEANUP_ATTEMPT_FINISHED",
|
"CLEANUP_ATTEMPT_FINISHED",
|
||||||
"CLEANUP_ATTEMPT_FAILED",
|
"CLEANUP_ATTEMPT_FAILED",
|
||||||
"CLEANUP_ATTEMPT_KILLED"
|
"CLEANUP_ATTEMPT_KILLED",
|
||||||
|
"AM_STARTED"
|
||||||
]
|
]
|
||||||
},
|
},
|
||||||
|
|
||||||
|
@ -272,6 +284,7 @@
|
||||||
"JobFinished",
|
"JobFinished",
|
||||||
"JobInfoChange",
|
"JobInfoChange",
|
||||||
"JobInited",
|
"JobInited",
|
||||||
|
"AMStarted",
|
||||||
"JobPriorityChange",
|
"JobPriorityChange",
|
||||||
"JobStatusChanged",
|
"JobStatusChanged",
|
||||||
"JobSubmitted",
|
"JobSubmitted",
|
||||||
|
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
* 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.jobhistory;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
|
|
||||||
|
import org.apache.avro.util.Utf8;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Event to record start of a task attempt
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
public class AMStartedEvent implements HistoryEvent {
|
||||||
|
private AMStarted datum = new AMStarted();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an event to record the start of an MR AppMaster
|
||||||
|
*
|
||||||
|
* @param appAttemptId
|
||||||
|
* the application attempt id.
|
||||||
|
* @param startTime
|
||||||
|
* the start time of the AM.
|
||||||
|
* @param containerId
|
||||||
|
* the containerId of the AM.
|
||||||
|
* @param nodeManagerHost
|
||||||
|
* the node on which the AM is running.
|
||||||
|
* @param nodeManagerHttpPort
|
||||||
|
* the httpPort for the node running the AM.
|
||||||
|
*/
|
||||||
|
public AMStartedEvent(ApplicationAttemptId appAttemptId, long startTime,
|
||||||
|
ContainerId containerId, String nodeManagerHost, int nodeManagerHttpPort) {
|
||||||
|
datum.applicationAttemptId = new Utf8(appAttemptId.toString());
|
||||||
|
datum.startTime = startTime;
|
||||||
|
datum.containerId = new Utf8(containerId.toString());
|
||||||
|
datum.nodeManagerHost = new Utf8(nodeManagerHost);
|
||||||
|
datum.nodeManagerHttpPort = nodeManagerHttpPort;
|
||||||
|
}
|
||||||
|
|
||||||
|
AMStartedEvent() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public Object getDatum() {
|
||||||
|
return datum;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDatum(Object datum) {
|
||||||
|
this.datum = (AMStarted) datum;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the ApplicationAttemptId
|
||||||
|
*/
|
||||||
|
public ApplicationAttemptId getAppAttemptId() {
|
||||||
|
return ConverterUtils.toApplicationAttemptId(datum.applicationAttemptId
|
||||||
|
.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the start time for the MRAppMaster
|
||||||
|
*/
|
||||||
|
public long getStartTime() {
|
||||||
|
return datum.startTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the ContainerId for the MRAppMaster.
|
||||||
|
*/
|
||||||
|
public ContainerId getContainerId() {
|
||||||
|
return ConverterUtils.toContainerId(datum.containerId.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the node manager host.
|
||||||
|
*/
|
||||||
|
public String getNodeManagerHost() {
|
||||||
|
return datum.nodeManagerHost.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the http port for the tracker.
|
||||||
|
*/
|
||||||
|
public int getNodeManagerHttpPort() {
|
||||||
|
return datum.nodeManagerHttpPort;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Get the attempt id */
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public EventType getEventType() {
|
||||||
|
return EventType.AM_STARTED;
|
||||||
|
}
|
||||||
|
}
|
|
@ -33,7 +33,6 @@ import org.apache.hadoop.mapreduce.Counters;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.io.Decoder;
|
import org.apache.avro.io.Decoder;
|
||||||
import org.apache.avro.io.DecoderFactory;
|
import org.apache.avro.io.DecoderFactory;
|
||||||
import org.apache.avro.io.JsonDecoder;
|
|
||||||
import org.apache.avro.io.DatumReader;
|
import org.apache.avro.io.DatumReader;
|
||||||
import org.apache.avro.specific.SpecificDatumReader;
|
import org.apache.avro.specific.SpecificDatumReader;
|
||||||
|
|
||||||
|
@ -146,8 +145,10 @@ public class EventReader implements Closeable {
|
||||||
result = new TaskAttemptUnsuccessfulCompletionEvent(); break;
|
result = new TaskAttemptUnsuccessfulCompletionEvent(); break;
|
||||||
case CLEANUP_ATTEMPT_KILLED:
|
case CLEANUP_ATTEMPT_KILLED:
|
||||||
result = new TaskAttemptUnsuccessfulCompletionEvent(); break;
|
result = new TaskAttemptUnsuccessfulCompletionEvent(); break;
|
||||||
|
case AM_STARTED:
|
||||||
|
result = new AMStartedEvent(); break;
|
||||||
default:
|
default:
|
||||||
throw new RuntimeException("unexpected event type!");
|
throw new RuntimeException("unexpected event type: " + wrapper.type);
|
||||||
}
|
}
|
||||||
result.setDatum(wrapper.event);
|
result.setDatum(wrapper.event);
|
||||||
return result;
|
return result;
|
||||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.mapreduce.jobhistory;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
@ -37,6 +39,8 @@ import org.apache.hadoop.mapreduce.TaskID;
|
||||||
import org.apache.hadoop.mapred.TaskStatus;
|
import org.apache.hadoop.mapred.TaskStatus;
|
||||||
import org.apache.hadoop.mapreduce.TaskType;
|
import org.apache.hadoop.mapreduce.TaskType;
|
||||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default Parser for the JobHistory files. Typical usage is
|
* Default Parser for the JobHistory files. Typical usage is
|
||||||
|
@ -174,6 +178,9 @@ public class JobHistoryParser {
|
||||||
case CLEANUP_ATTEMPT_FINISHED:
|
case CLEANUP_ATTEMPT_FINISHED:
|
||||||
handleTaskAttemptFinishedEvent((TaskAttemptFinishedEvent) event);
|
handleTaskAttemptFinishedEvent((TaskAttemptFinishedEvent) event);
|
||||||
break;
|
break;
|
||||||
|
case AM_STARTED:
|
||||||
|
handleAMStartedEvent((AMStartedEvent) event);
|
||||||
|
break;
|
||||||
default:
|
default:
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -241,6 +248,7 @@ public class JobHistoryParser {
|
||||||
attemptInfo.trackerName = event.getTrackerName();
|
attemptInfo.trackerName = event.getTrackerName();
|
||||||
attemptInfo.taskType = event.getTaskType();
|
attemptInfo.taskType = event.getTaskType();
|
||||||
attemptInfo.shufflePort = event.getShufflePort();
|
attemptInfo.shufflePort = event.getShufflePort();
|
||||||
|
attemptInfo.containerId = event.getContainerId();
|
||||||
|
|
||||||
taskInfo.attemptsMap.put(attemptId, attemptInfo);
|
taskInfo.attemptsMap.put(attemptId, attemptInfo);
|
||||||
}
|
}
|
||||||
|
@ -305,6 +313,20 @@ public class JobHistoryParser {
|
||||||
info.totalReduces = event.getTotalReduces();
|
info.totalReduces = event.getTotalReduces();
|
||||||
info.uberized = event.getUberized();
|
info.uberized = event.getUberized();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void handleAMStartedEvent(AMStartedEvent event) {
|
||||||
|
AMInfo amInfo = new AMInfo();
|
||||||
|
amInfo.appAttemptId = event.getAppAttemptId();
|
||||||
|
amInfo.startTime = event.getStartTime();
|
||||||
|
amInfo.containerId = event.getContainerId();
|
||||||
|
amInfo.nodeManagerHost = event.getNodeManagerHost();
|
||||||
|
amInfo.nodeManagerHttpPort = event.getNodeManagerHttpPort();
|
||||||
|
if (info.amInfos == null) {
|
||||||
|
info.amInfos = new LinkedList<AMInfo>();
|
||||||
|
}
|
||||||
|
info.amInfos.add(amInfo);
|
||||||
|
info.latestAmInfo = amInfo;
|
||||||
|
}
|
||||||
|
|
||||||
private void handleJobInfoChangeEvent(JobInfoChangeEvent event) {
|
private void handleJobInfoChangeEvent(JobInfoChangeEvent event) {
|
||||||
info.submitTime = event.getSubmitTime();
|
info.submitTime = event.getSubmitTime();
|
||||||
|
@ -348,6 +370,8 @@ public class JobHistoryParser {
|
||||||
Map<JobACL, AccessControlList> jobACLs;
|
Map<JobACL, AccessControlList> jobACLs;
|
||||||
|
|
||||||
Map<TaskID, TaskInfo> tasksMap;
|
Map<TaskID, TaskInfo> tasksMap;
|
||||||
|
List<AMInfo> amInfos;
|
||||||
|
AMInfo latestAmInfo;
|
||||||
boolean uberized;
|
boolean uberized;
|
||||||
|
|
||||||
/** Create a job info object where job information will be stored
|
/** Create a job info object where job information will be stored
|
||||||
|
@ -377,7 +401,9 @@ public class JobHistoryParser {
|
||||||
System.out.println("REDUCE_COUNTERS:" + reduceCounters.toString());
|
System.out.println("REDUCE_COUNTERS:" + reduceCounters.toString());
|
||||||
System.out.println("TOTAL_COUNTERS: " + totalCounters.toString());
|
System.out.println("TOTAL_COUNTERS: " + totalCounters.toString());
|
||||||
System.out.println("UBERIZED: " + uberized);
|
System.out.println("UBERIZED: " + uberized);
|
||||||
|
for (AMInfo amInfo : amInfos) {
|
||||||
|
amInfo.printAll();
|
||||||
|
}
|
||||||
for (TaskInfo ti: tasksMap.values()) {
|
for (TaskInfo ti: tasksMap.values()) {
|
||||||
ti.printAll();
|
ti.printAll();
|
||||||
}
|
}
|
||||||
|
@ -427,6 +453,10 @@ public class JobHistoryParser {
|
||||||
public Map<JobACL, AccessControlList> getJobACLs() { return jobACLs; }
|
public Map<JobACL, AccessControlList> getJobACLs() { return jobACLs; }
|
||||||
/** @return the uberized status of this job */
|
/** @return the uberized status of this job */
|
||||||
public boolean getUberized() { return uberized; }
|
public boolean getUberized() { return uberized; }
|
||||||
|
/** @return the AMInfo for the job's AppMaster */
|
||||||
|
public List<AMInfo> getAMInfos() { return amInfos; }
|
||||||
|
/** @return the AMInfo for the newest AppMaster */
|
||||||
|
public AMInfo getLatestAMInfo() { return latestAmInfo; };
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -509,6 +539,7 @@ public class JobHistoryParser {
|
||||||
int httpPort;
|
int httpPort;
|
||||||
int shufflePort;
|
int shufflePort;
|
||||||
String hostname;
|
String hostname;
|
||||||
|
ContainerId containerId;
|
||||||
|
|
||||||
/** Create a Task Attempt Info which will store attempt level information
|
/** Create a Task Attempt Info which will store attempt level information
|
||||||
* on a history parse.
|
* on a history parse.
|
||||||
|
@ -534,6 +565,7 @@ public class JobHistoryParser {
|
||||||
System.out.println("TRACKER_NAME:" + trackerName);
|
System.out.println("TRACKER_NAME:" + trackerName);
|
||||||
System.out.println("HTTP_PORT:" + httpPort);
|
System.out.println("HTTP_PORT:" + httpPort);
|
||||||
System.out.println("SHUFFLE_PORT:" + shufflePort);
|
System.out.println("SHUFFLE_PORT:" + shufflePort);
|
||||||
|
System.out.println("CONTIANER_ID:" + containerId);
|
||||||
if (counters != null) {
|
if (counters != null) {
|
||||||
System.out.println("COUNTERS:" + counters.toString());
|
System.out.println("COUNTERS:" + counters.toString());
|
||||||
}
|
}
|
||||||
|
@ -569,5 +601,74 @@ public class JobHistoryParser {
|
||||||
public int getHttpPort() { return httpPort; }
|
public int getHttpPort() { return httpPort; }
|
||||||
/** @return the Shuffle port for the tracker */
|
/** @return the Shuffle port for the tracker */
|
||||||
public int getShufflePort() { return shufflePort; }
|
public int getShufflePort() { return shufflePort; }
|
||||||
|
/** @return the ContainerId for the tracker */
|
||||||
|
public ContainerId getContainerId() { return containerId; }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stores AM information
|
||||||
|
*/
|
||||||
|
public static class AMInfo {
|
||||||
|
ApplicationAttemptId appAttemptId;
|
||||||
|
long startTime;
|
||||||
|
ContainerId containerId;
|
||||||
|
String nodeManagerHost;
|
||||||
|
int nodeManagerHttpPort;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a AM Info which will store AM level information on a history
|
||||||
|
* parse.
|
||||||
|
*/
|
||||||
|
public AMInfo() {
|
||||||
|
startTime = -1;
|
||||||
|
nodeManagerHost = "";
|
||||||
|
nodeManagerHttpPort = -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AMInfo(ApplicationAttemptId appAttemptId, long startTime,
|
||||||
|
ContainerId containerId, String nodeManagerHost, int nodeManagerHttpPort) {
|
||||||
|
this.appAttemptId = appAttemptId;
|
||||||
|
this.startTime = startTime;
|
||||||
|
this.containerId = containerId;
|
||||||
|
this.nodeManagerHost = nodeManagerHost;
|
||||||
|
this.nodeManagerHttpPort = nodeManagerHttpPort;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Print all the information about this AM.
|
||||||
|
*/
|
||||||
|
public void printAll() {
|
||||||
|
System.out.println("APPLICATION_ATTEMPT_ID:" + appAttemptId.toString());
|
||||||
|
System.out.println("START_TIME: " + startTime);
|
||||||
|
System.out.println("CONTAINER_ID: " + containerId.toString());
|
||||||
|
System.out.println("NODE_MANAGER_HOST: " + nodeManagerHost);
|
||||||
|
System.out.println("NODE_MANAGER_HTTP_PORT: " + nodeManagerHttpPort);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return the ApplicationAttemptId */
|
||||||
|
public ApplicationAttemptId getAppAttemptId() {
|
||||||
|
return appAttemptId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return the start time of the AM */
|
||||||
|
public long getStartTime() {
|
||||||
|
return startTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return the container id for the AM */
|
||||||
|
public ContainerId getContainerId() {
|
||||||
|
return containerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return the host name for the node manager on which the AM is running */
|
||||||
|
public String getNodeManagerHost() {
|
||||||
|
return nodeManagerHost;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return the http port for the node manager running the AM */
|
||||||
|
public int getNodeManagerHttpPort() {
|
||||||
|
return nodeManagerHttpPort;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,8 +18,6 @@
|
||||||
|
|
||||||
package org.apache.hadoop.mapreduce.jobhistory;
|
package org.apache.hadoop.mapreduce.jobhistory;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.mapreduce.JobID;
|
import org.apache.hadoop.mapreduce.JobID;
|
||||||
|
|
|
@ -18,13 +18,13 @@
|
||||||
|
|
||||||
package org.apache.hadoop.mapreduce.jobhistory;
|
package org.apache.hadoop.mapreduce.jobhistory;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||||
import org.apache.hadoop.mapreduce.TaskID;
|
import org.apache.hadoop.mapreduce.TaskID;
|
||||||
import org.apache.hadoop.mapreduce.TaskType;
|
import org.apache.hadoop.mapreduce.TaskType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
|
|
||||||
import org.apache.avro.util.Utf8;
|
import org.apache.avro.util.Utf8;
|
||||||
|
|
||||||
|
@ -45,10 +45,11 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
|
||||||
* @param trackerName Name of the Task Tracker where attempt is running
|
* @param trackerName Name of the Task Tracker where attempt is running
|
||||||
* @param httpPort The port number of the tracker
|
* @param httpPort The port number of the tracker
|
||||||
* @param shufflePort The shuffle port number of the container
|
* @param shufflePort The shuffle port number of the container
|
||||||
|
* @param containerId The containerId for the task attempt.
|
||||||
*/
|
*/
|
||||||
public TaskAttemptStartedEvent( TaskAttemptID attemptId,
|
public TaskAttemptStartedEvent( TaskAttemptID attemptId,
|
||||||
TaskType taskType, long startTime, String trackerName,
|
TaskType taskType, long startTime, String trackerName,
|
||||||
int httpPort, int shufflePort) {
|
int httpPort, int shufflePort, ContainerId containerId) {
|
||||||
datum.attemptId = new Utf8(attemptId.toString());
|
datum.attemptId = new Utf8(attemptId.toString());
|
||||||
datum.taskid = new Utf8(attemptId.getTaskID().toString());
|
datum.taskid = new Utf8(attemptId.getTaskID().toString());
|
||||||
datum.startTime = startTime;
|
datum.startTime = startTime;
|
||||||
|
@ -56,6 +57,15 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
|
||||||
datum.trackerName = new Utf8(trackerName);
|
datum.trackerName = new Utf8(trackerName);
|
||||||
datum.httpPort = httpPort;
|
datum.httpPort = httpPort;
|
||||||
datum.shufflePort = shufflePort;
|
datum.shufflePort = shufflePort;
|
||||||
|
datum.containerId = new Utf8(containerId.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO Remove after MrV1 is removed.
|
||||||
|
// Using a dummy containerId to prevent jobHistory parse failures.
|
||||||
|
public TaskAttemptStartedEvent(TaskAttemptID attemptId, TaskType taskType,
|
||||||
|
long startTime, String trackerName, int httpPort, int shufflePort) {
|
||||||
|
this(attemptId, taskType, startTime, trackerName, httpPort, shufflePort,
|
||||||
|
ConverterUtils.toContainerId("container_-1_-1_-1_-1"));
|
||||||
}
|
}
|
||||||
|
|
||||||
TaskAttemptStartedEvent() {}
|
TaskAttemptStartedEvent() {}
|
||||||
|
@ -91,5 +101,8 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
|
||||||
? EventType.MAP_ATTEMPT_STARTED
|
? EventType.MAP_ATTEMPT_STARTED
|
||||||
: EventType.REDUCE_ATTEMPT_STARTED;
|
: EventType.REDUCE_ATTEMPT_STARTED;
|
||||||
}
|
}
|
||||||
|
/** Get the ContainerId */
|
||||||
|
public ContainerId getContainerId() {
|
||||||
|
return ConverterUtils.toContainerId(datum.containerId.toString());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.JobACL;
|
||||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||||
|
@ -337,4 +338,9 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
|
||||||
public Path getConfFile() {
|
public Path getConfFile() {
|
||||||
return confFile;
|
return confFile;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AMInfo> getAMInfos() {
|
||||||
|
return jobInfo.getAMInfos();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,8 +29,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
|
||||||
|
@ -85,35 +83,17 @@ public class CompletedTaskAttempt implements TaskAttempt {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ContainerId getAssignedContainerID() {
|
public ContainerId getAssignedContainerID() {
|
||||||
//TODO ContainerId needs to be part of some historyEvent to be able to
|
return attemptInfo.getContainerId();
|
||||||
//render the log directory.
|
|
||||||
ContainerId containerId =
|
|
||||||
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
|
|
||||||
ContainerId.class);
|
|
||||||
containerId.setId(-1);
|
|
||||||
ApplicationAttemptId applicationAttemptId =
|
|
||||||
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
|
|
||||||
ApplicationAttemptId.class);
|
|
||||||
applicationAttemptId.setAttemptId(-1);
|
|
||||||
ApplicationId applicationId =
|
|
||||||
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
|
|
||||||
ApplicationId.class);
|
|
||||||
applicationId.setClusterTimestamp(-1);
|
|
||||||
applicationId.setId(-1);
|
|
||||||
applicationAttemptId.setApplicationId(applicationId);
|
|
||||||
containerId.setApplicationAttemptId(applicationAttemptId);
|
|
||||||
return containerId;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getAssignedContainerMgrAddress() {
|
public String getAssignedContainerMgrAddress() {
|
||||||
// TODO Verify this is correct.
|
return attemptInfo.getHostname();
|
||||||
return attemptInfo.getTrackerName();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getNodeHttpAddress() {
|
public String getNodeHttpAddress() {
|
||||||
return attemptInfo.getHostname() + ":" + attemptInfo.getHttpPort();
|
return attemptInfo.getTrackerName() + ":" + attemptInfo.getHttpPort();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.mapreduce.JobACL;
|
import org.apache.hadoop.mapreduce.JobACL;
|
||||||
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
||||||
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.JobReport;
|
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||||
|
@ -159,4 +160,9 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
|
||||||
throw new IllegalStateException("Not implemented yet");
|
throw new IllegalStateException("Not implemented yet");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AMInfo> getAMInfos() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,8 +41,10 @@ import org.apache.hadoop.mapreduce.v2.app.MRApp;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
import org.apache.hadoop.yarn.service.Service;
|
import org.apache.hadoop.yarn.service.Service;
|
||||||
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
public class TestJobHistoryEvents {
|
public class TestJobHistoryEvents {
|
||||||
|
@ -159,6 +161,10 @@ public class TestJobHistoryEvents {
|
||||||
private void verifyAttempt(TaskAttempt attempt) {
|
private void verifyAttempt(TaskAttempt attempt) {
|
||||||
Assert.assertEquals("TaskAttempt state not currect",
|
Assert.assertEquals("TaskAttempt state not currect",
|
||||||
TaskAttemptState.SUCCEEDED, attempt.getState());
|
TaskAttemptState.SUCCEEDED, attempt.getState());
|
||||||
|
Assert.assertNotNull(attempt.getAssignedContainerID());
|
||||||
|
//Verify the wrong ctor is not being used. Remove after mrv1 is removed.
|
||||||
|
ContainerId fakeCid = BuilderUtils.newContainerId(-1, -1, -1, -1);
|
||||||
|
Assert.assertFalse(attempt.getAssignedContainerID().equals(fakeCid));
|
||||||
}
|
}
|
||||||
|
|
||||||
static class MRAppWithHistory extends MRApp {
|
static class MRAppWithHistory extends MRApp {
|
||||||
|
|
|
@ -34,8 +34,9 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
|
||||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
||||||
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.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.app.MRApp;
|
import org.apache.hadoop.mapreduce.v2.app.MRApp;
|
||||||
|
@ -46,7 +47,9 @@ import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory;
|
||||||
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
|
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.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.service.Service;
|
import org.apache.hadoop.yarn.service.Service;
|
||||||
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
public class TestJobHistoryParsing {
|
public class TestJobHistoryParsing {
|
||||||
|
@ -54,6 +57,7 @@ public class TestJobHistoryParsing {
|
||||||
@Test
|
@Test
|
||||||
public void testHistoryParsing() throws Exception {
|
public void testHistoryParsing() throws Exception {
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
|
long amStartTimeEst = System.currentTimeMillis();
|
||||||
MRApp app = new MRAppWithHistory(2, 1, true, this.getClass().getName(), true);
|
MRApp app = new MRAppWithHistory(2, 1, true, this.getClass().getName(), true);
|
||||||
app.submit(conf);
|
app.submit(conf);
|
||||||
Job job = app.getContext().getAllJobs().values().iterator().next();
|
Job job = app.getContext().getAllJobs().values().iterator().next();
|
||||||
|
@ -102,12 +106,30 @@ public class TestJobHistoryParsing {
|
||||||
job.isUber(), jobInfo.getUberized());
|
job.isUber(), jobInfo.getUberized());
|
||||||
int totalTasks = jobInfo.getAllTasks().size();
|
int totalTasks = jobInfo.getAllTasks().size();
|
||||||
Assert.assertEquals("total number of tasks is incorrect ", 3, totalTasks);
|
Assert.assertEquals("total number of tasks is incorrect ", 3, totalTasks);
|
||||||
|
|
||||||
|
// Verify aminfo
|
||||||
|
Assert.assertEquals(1, jobInfo.getAMInfos().size());
|
||||||
|
Assert.assertEquals("testhost", jobInfo.getAMInfos().get(0)
|
||||||
|
.getNodeManagerHost());
|
||||||
|
AMInfo amInfo = jobInfo.getAMInfos().get(0);
|
||||||
|
Assert.assertEquals(3333, amInfo.getNodeManagerHttpPort());
|
||||||
|
Assert.assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
|
||||||
|
Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
|
||||||
|
.getApplicationAttemptId());
|
||||||
|
Assert.assertTrue(amInfo.getStartTime() <= System.currentTimeMillis()
|
||||||
|
&& amInfo.getStartTime() >= amStartTimeEst);
|
||||||
|
|
||||||
|
ContainerId fakeCid = BuilderUtils.newContainerId(-1, -1, -1, -1);
|
||||||
//Assert at taskAttempt level
|
//Assert at taskAttempt level
|
||||||
for (TaskInfo taskInfo : jobInfo.getAllTasks().values()) {
|
for (TaskInfo taskInfo : jobInfo.getAllTasks().values()) {
|
||||||
int taskAttemptCount = taskInfo.getAllTaskAttempts().size();
|
int taskAttemptCount = taskInfo.getAllTaskAttempts().size();
|
||||||
Assert.assertEquals("total number of task attempts ",
|
Assert.assertEquals("total number of task attempts ",
|
||||||
1, taskAttemptCount);
|
1, taskAttemptCount);
|
||||||
|
TaskAttemptInfo taInfo =
|
||||||
|
taskInfo.getAllTaskAttempts().values().iterator().next();
|
||||||
|
Assert.assertNotNull(taInfo.getContainerId());
|
||||||
|
//Verify the wrong ctor is not being used. Remove after mrv1 is removed.
|
||||||
|
Assert.assertFalse(taInfo.getContainerId().equals(fakeCid));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Deep compare Job and JobInfo
|
// Deep compare Job and JobInfo
|
||||||
|
|
|
@ -36,12 +36,24 @@ public interface ApplicationConstants {
|
||||||
// TODO: They say tokens via env isn't good.
|
// TODO: They say tokens via env isn't good.
|
||||||
public static final String APPLICATION_CLIENT_SECRET_ENV_NAME =
|
public static final String APPLICATION_CLIENT_SECRET_ENV_NAME =
|
||||||
"AppClientTokenEnv";
|
"AppClientTokenEnv";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The environmental variable for APPLICATION_ATTEMPT_ID. Set in
|
* The environment variable for CONTAINER_ID. Set in AppMaster environment
|
||||||
* ApplicationMaster's environment only.
|
* only
|
||||||
*/
|
*/
|
||||||
public static final String APPLICATION_ATTEMPT_ID_ENV = "APPLICATION_ATTEMPT_ID";
|
public static final String AM_CONTAINER_ID_ENV = "AM_CONTAINER_ID";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The environment variable for NM_HTTP_ADDRESS. Set in AppMaster environment
|
||||||
|
* only
|
||||||
|
*/
|
||||||
|
public static final String NM_HTTP_ADDRESS_ENV = "NM_HTTP_ADDRESS";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The environment variable for APP_SUBMIT_TIME. Set in AppMaster environment
|
||||||
|
* only
|
||||||
|
*/
|
||||||
|
public static final String APP_SUBMIT_TIME_ENV = "APP_SUBMIT_TIME_ENV";
|
||||||
|
|
||||||
public static final String CONTAINER_TOKEN_FILE_ENV_NAME =
|
public static final String CONTAINER_TOKEN_FILE_ENV_NAME =
|
||||||
UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION;
|
UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION;
|
||||||
|
|
|
@ -126,7 +126,7 @@ public interface ApplicationSubmissionContext {
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
public void setUser(String user);
|
public void setUser(String user);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the <code>ContainerLaunchContext</code> to describe the
|
* Get the <code>ContainerLaunchContext</code> to describe the
|
||||||
* <code>Container</code> with which the <code>ApplicationMaster</code> is
|
* <code>Container</code> with which the <code>ApplicationMaster</code> is
|
||||||
|
|
|
@ -287,7 +287,7 @@ public class ApplicationMaster {
|
||||||
Map<String, String> envs = System.getenv();
|
Map<String, String> envs = System.getenv();
|
||||||
|
|
||||||
appAttemptID = Records.newRecord(ApplicationAttemptId.class);
|
appAttemptID = Records.newRecord(ApplicationAttemptId.class);
|
||||||
if (!envs.containsKey(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV)) {
|
if (!envs.containsKey(ApplicationConstants.AM_CONTAINER_ID_ENV)) {
|
||||||
if (cliParser.hasOption("app_attempt_id")) {
|
if (cliParser.hasOption("app_attempt_id")) {
|
||||||
String appIdStr = cliParser.getOptionValue("app_attempt_id", "");
|
String appIdStr = cliParser.getOptionValue("app_attempt_id", "");
|
||||||
appAttemptID = ConverterUtils.toApplicationAttemptId(appIdStr);
|
appAttemptID = ConverterUtils.toApplicationAttemptId(appIdStr);
|
||||||
|
@ -296,7 +296,8 @@ public class ApplicationMaster {
|
||||||
throw new IllegalArgumentException("Application Attempt Id not set in the environment");
|
throw new IllegalArgumentException("Application Attempt Id not set in the environment");
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
appAttemptID = ConverterUtils.toApplicationAttemptId(envs.get(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV));
|
ContainerId containerId = ConverterUtils.toContainerId(envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV));
|
||||||
|
appAttemptID = containerId.getApplicationAttemptId();
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Application master for app"
|
LOG.info("Application master for app"
|
||||||
|
|
|
@ -20,10 +20,8 @@ package org.apache.hadoop.yarn.util;
|
||||||
|
|
||||||
import static org.apache.hadoop.yarn.util.StringHelper._split;
|
import static org.apache.hadoop.yarn.util.StringHelper._split;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.text.NumberFormat;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -33,9 +31,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
||||||
import org.apache.hadoop.yarn.api.records.URL;
|
import org.apache.hadoop.yarn.api.records.URL;
|
||||||
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
||||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
|
||||||
|
@ -97,27 +93,8 @@ public class ConverterUtils {
|
||||||
return url;
|
return url;
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: Why thread local?
|
|
||||||
// ^ NumberFormat instances are not threadsafe
|
|
||||||
private static final ThreadLocal<NumberFormat> appIdFormat =
|
|
||||||
new ThreadLocal<NumberFormat>() {
|
|
||||||
@Override
|
|
||||||
public NumberFormat initialValue() {
|
|
||||||
NumberFormat fmt = NumberFormat.getInstance();
|
|
||||||
fmt.setGroupingUsed(false);
|
|
||||||
fmt.setMinimumIntegerDigits(4);
|
|
||||||
return fmt;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
public static String toString(ApplicationId appId) {
|
public static String toString(ApplicationId appId) {
|
||||||
StringBuilder sb = new StringBuilder();
|
return appId.toString();
|
||||||
sb.append(APPLICATION_PREFIX + "_").append(appId.getClusterTimestamp())
|
|
||||||
.append("_");
|
|
||||||
sb.append(appIdFormat.get().format(appId.getId()));
|
|
||||||
return sb.toString();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ApplicationId toApplicationId(RecordFactory recordFactory,
|
public static ApplicationId toApplicationId(RecordFactory recordFactory,
|
||||||
|
@ -152,11 +129,11 @@ public class ConverterUtils {
|
||||||
return cId.toString();
|
return cId.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ContainerId toContainerId(String containerIdStr)
|
public static ContainerId toContainerId(String containerIdStr) {
|
||||||
throws IOException {
|
|
||||||
Iterator<String> it = _split(containerIdStr).iterator();
|
Iterator<String> it = _split(containerIdStr).iterator();
|
||||||
if (!it.next().equals(CONTAINER_PREFIX)) {
|
if (!it.next().equals(CONTAINER_PREFIX)) {
|
||||||
throw new IOException("Invalid ContainerId prefix: " + containerIdStr);
|
throw new IllegalArgumentException("Invalid ContainerId prefix: "
|
||||||
|
+ containerIdStr);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
ApplicationAttemptId appAttemptID = toApplicationAttemptId(it);
|
ApplicationAttemptId appAttemptID = toApplicationAttemptId(it);
|
||||||
|
@ -165,21 +142,22 @@ public class ConverterUtils {
|
||||||
containerId.setId(Integer.parseInt(it.next()));
|
containerId.setId(Integer.parseInt(it.next()));
|
||||||
return containerId;
|
return containerId;
|
||||||
} catch (NumberFormatException n) {
|
} catch (NumberFormatException n) {
|
||||||
throw new IOException("Invalid ContainerId: " + containerIdStr, n);
|
throw new IllegalArgumentException("Invalid ContainerId: "
|
||||||
|
+ containerIdStr, n);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ApplicationAttemptId toApplicationAttemptId(
|
public static ApplicationAttemptId toApplicationAttemptId(
|
||||||
String applicationAttmeptIdStr) throws IOException {
|
String applicationAttmeptIdStr) {
|
||||||
Iterator<String> it = _split(applicationAttmeptIdStr).iterator();
|
Iterator<String> it = _split(applicationAttmeptIdStr).iterator();
|
||||||
if (!it.next().equals(APPLICATION_ATTEMPT_PREFIX)) {
|
if (!it.next().equals(APPLICATION_ATTEMPT_PREFIX)) {
|
||||||
throw new IOException("Invalid AppAttemptId prefix: "
|
throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
|
||||||
+ applicationAttmeptIdStr);
|
+ applicationAttmeptIdStr);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
return toApplicationAttemptId(it);
|
return toApplicationAttemptId(it);
|
||||||
} catch (NumberFormatException n) {
|
} catch (NumberFormatException n) {
|
||||||
throw new IOException("Invalid AppAttemptId: "
|
throw new IllegalArgumentException("Invalid AppAttemptId: "
|
||||||
+ applicationAttmeptIdStr, n);
|
+ applicationAttmeptIdStr, n);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -69,7 +69,7 @@ public class ContainerLogsPage extends NMView {
|
||||||
ContainerId containerId;
|
ContainerId containerId;
|
||||||
try {
|
try {
|
||||||
containerId = ConverterUtils.toContainerId($(CONTAINER_ID));
|
containerId = ConverterUtils.toContainerId($(CONTAINER_ID));
|
||||||
} catch (IOException e) {
|
} catch (IllegalArgumentException e) {
|
||||||
div.h1("Invalid containerId " + $(CONTAINER_ID))._();
|
div.h1("Invalid containerId " + $(CONTAINER_ID))._();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,8 +22,6 @@ import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
|
||||||
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
|
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
|
||||||
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
|
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
@ -66,7 +64,7 @@ public class ContainerPage extends NMView implements NMWebParams {
|
||||||
ContainerId containerID;
|
ContainerId containerID;
|
||||||
try {
|
try {
|
||||||
containerID = ConverterUtils.toContainerId($(CONTAINER_ID));
|
containerID = ConverterUtils.toContainerId($(CONTAINER_ID));
|
||||||
} catch (IOException e) {
|
} catch (IllegalArgumentException e) {
|
||||||
html.p()._("Invalid containerId " + $(CONTAINER_ID))._();
|
html.p()._("Invalid containerId " + $(CONTAINER_ID))._();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -206,11 +206,12 @@ public class ClientRMService extends AbstractService implements
|
||||||
|
|
||||||
// Safety
|
// Safety
|
||||||
submissionContext.setUser(user);
|
submissionContext.setUser(user);
|
||||||
|
|
||||||
// This needs to be synchronous as the client can query
|
// This needs to be synchronous as the client can query
|
||||||
// immediately following the submission to get the application status.
|
// immediately following the submission to get the application status.
|
||||||
// So call handle directly and do not send an event.
|
// So call handle directly and do not send an event.
|
||||||
rmAppManager.handle(new RMAppManagerSubmitEvent(submissionContext));
|
rmAppManager.handle(new RMAppManagerSubmitEvent(submissionContext, System
|
||||||
|
.currentTimeMillis()));
|
||||||
|
|
||||||
LOG.info("Application with id " + applicationId.getId() +
|
LOG.info("Application with id " + applicationId.getId() +
|
||||||
" submitted by user " + user + " with " + submissionContext);
|
" submitted by user " + user + " with " + submissionContext);
|
||||||
|
|
|
@ -213,7 +213,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
protected synchronized void submitApplication(
|
protected synchronized void submitApplication(
|
||||||
ApplicationSubmissionContext submissionContext) {
|
ApplicationSubmissionContext submissionContext, long submitTime) {
|
||||||
ApplicationId applicationId = submissionContext.getApplicationId();
|
ApplicationId applicationId = submissionContext.getApplicationId();
|
||||||
RMApp application = null;
|
RMApp application = null;
|
||||||
try {
|
try {
|
||||||
|
@ -241,13 +241,13 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
|
||||||
ApplicationStore appStore = rmContext.getApplicationsStore()
|
ApplicationStore appStore = rmContext.getApplicationsStore()
|
||||||
.createApplicationStore(submissionContext.getApplicationId(),
|
.createApplicationStore(submissionContext.getApplicationId(),
|
||||||
submissionContext);
|
submissionContext);
|
||||||
|
|
||||||
// Create RMApp
|
// Create RMApp
|
||||||
application = new RMAppImpl(applicationId, rmContext,
|
application = new RMAppImpl(applicationId, rmContext,
|
||||||
this.conf, submissionContext.getApplicationName(), user,
|
this.conf, submissionContext.getApplicationName(), user,
|
||||||
submissionContext.getQueue(), submissionContext, clientTokenStr,
|
submissionContext.getQueue(), submissionContext, clientTokenStr,
|
||||||
appStore, this.scheduler,
|
appStore, this.scheduler,
|
||||||
this.masterService);
|
this.masterService, submitTime);
|
||||||
|
|
||||||
if (rmContext.getRMApps().putIfAbsent(applicationId, application) !=
|
if (rmContext.getRMApps().putIfAbsent(applicationId, application) !=
|
||||||
null) {
|
null) {
|
||||||
|
@ -284,8 +284,9 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
|
||||||
case APP_SUBMIT:
|
case APP_SUBMIT:
|
||||||
{
|
{
|
||||||
ApplicationSubmissionContext submissionContext =
|
ApplicationSubmissionContext submissionContext =
|
||||||
((RMAppManagerSubmitEvent)event).getSubmissionContext();
|
((RMAppManagerSubmitEvent)event).getSubmissionContext();
|
||||||
submitApplication(submissionContext);
|
long submitTime = ((RMAppManagerSubmitEvent)event).getSubmitTime();
|
||||||
|
submitApplication(submissionContext, submitTime);
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
|
|
|
@ -23,13 +23,21 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
public class RMAppManagerSubmitEvent extends RMAppManagerEvent {
|
public class RMAppManagerSubmitEvent extends RMAppManagerEvent {
|
||||||
|
|
||||||
private final ApplicationSubmissionContext submissionContext;
|
private final ApplicationSubmissionContext submissionContext;
|
||||||
|
private final long submitTime;
|
||||||
|
|
||||||
public RMAppManagerSubmitEvent(ApplicationSubmissionContext submissionContext) {
|
public RMAppManagerSubmitEvent(
|
||||||
super(submissionContext.getApplicationId(), RMAppManagerEventType.APP_SUBMIT);
|
ApplicationSubmissionContext submissionContext, long submitTime) {
|
||||||
|
super(submissionContext.getApplicationId(),
|
||||||
|
RMAppManagerEventType.APP_SUBMIT);
|
||||||
this.submissionContext = submissionContext;
|
this.submissionContext = submissionContext;
|
||||||
|
this.submitTime = submitTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ApplicationSubmissionContext getSubmissionContext() {
|
public ApplicationSubmissionContext getSubmissionContext() {
|
||||||
return this.submissionContext;
|
return this.submissionContext;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getSubmitTime() {
|
||||||
|
return this.submitTime;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,8 +22,6 @@ import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.security.PrivilegedAction;
|
import java.security.PrivilegedAction;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import javax.crypto.SecretKey;
|
import javax.crypto.SecretKey;
|
||||||
|
@ -37,7 +35,6 @@ import org.apache.hadoop.io.DataOutputBuffer;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.SecurityInfo;
|
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
@ -58,7 +55,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||||
import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
|
import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
|
import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
|
||||||
import org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo;
|
|
||||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
|
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||||
|
@ -83,6 +79,7 @@ public class AMLauncher implements Runnable {
|
||||||
private final ApplicationTokenSecretManager applicationTokenSecretManager;
|
private final ApplicationTokenSecretManager applicationTokenSecretManager;
|
||||||
private final ClientToAMSecretManager clientToAMSecretManager;
|
private final ClientToAMSecretManager clientToAMSecretManager;
|
||||||
private final AMLauncherEventType eventType;
|
private final AMLauncherEventType eventType;
|
||||||
|
private final RMContext rmContext;
|
||||||
|
|
||||||
@SuppressWarnings("rawtypes")
|
@SuppressWarnings("rawtypes")
|
||||||
private final EventHandler handler;
|
private final EventHandler handler;
|
||||||
|
@ -96,6 +93,7 @@ public class AMLauncher implements Runnable {
|
||||||
this.applicationTokenSecretManager = applicationTokenSecretManager;
|
this.applicationTokenSecretManager = applicationTokenSecretManager;
|
||||||
this.clientToAMSecretManager = clientToAMSecretManager;
|
this.clientToAMSecretManager = clientToAMSecretManager;
|
||||||
this.eventType = eventType;
|
this.eventType = eventType;
|
||||||
|
this.rmContext = rmContext;
|
||||||
this.handler = rmContext.getDispatcher().getEventHandler();
|
this.handler = rmContext.getDispatcher().getEventHandler();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -189,9 +187,18 @@ public class AMLauncher implements Runnable {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Map<String, String> environment = container.getEnvironment();
|
Map<String, String> environment = container.getEnvironment();
|
||||||
|
|
||||||
// Set the AppAttemptId to be consumable by the AM.
|
// Set the AppAttemptId, containerId, NMHTTPAdress, AppSubmitTime to be
|
||||||
environment.put(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV,
|
// consumable by the AM.
|
||||||
application.getAppAttemptId().toString());
|
environment.put(ApplicationConstants.AM_CONTAINER_ID_ENV, container
|
||||||
|
.getContainerId().toString());
|
||||||
|
environment.put(ApplicationConstants.NM_HTTP_ADDRESS_ENV, application
|
||||||
|
.getMasterContainer().getNodeHttpAddress());
|
||||||
|
environment.put(
|
||||||
|
ApplicationConstants.APP_SUBMIT_TIME_ENV,
|
||||||
|
String.valueOf(rmContext.getRMApps()
|
||||||
|
.get(application.getAppAttemptId().getApplicationId())
|
||||||
|
.getSubmitTime()));
|
||||||
|
|
||||||
|
|
||||||
if (UserGroupInformation.isSecurityEnabled()) {
|
if (UserGroupInformation.isSecurityEnabled()) {
|
||||||
// TODO: Security enabled/disabled info should come from RM.
|
// TODO: Security enabled/disabled info should come from RM.
|
||||||
|
|
|
@ -115,6 +115,12 @@ public interface RMApp extends EventHandler<RMAppEvent> {
|
||||||
*/
|
*/
|
||||||
long getStartTime();
|
long getStartTime();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* the submit time of the application.
|
||||||
|
* @return the submit time of the application.
|
||||||
|
*/
|
||||||
|
long getSubmitTime();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The tracking url for the application master.
|
* The tracking url for the application master.
|
||||||
* @return the tracking url for the application master.
|
* @return the tracking url for the application master.
|
||||||
|
|
|
@ -84,6 +84,7 @@ public class RMAppImpl implements RMApp {
|
||||||
private final WriteLock writeLock;
|
private final WriteLock writeLock;
|
||||||
private final Map<ApplicationAttemptId, RMAppAttempt> attempts
|
private final Map<ApplicationAttemptId, RMAppAttempt> attempts
|
||||||
= new LinkedHashMap<ApplicationAttemptId, RMAppAttempt>();
|
= new LinkedHashMap<ApplicationAttemptId, RMAppAttempt>();
|
||||||
|
private final long submitTime;
|
||||||
|
|
||||||
// Mutable fields
|
// Mutable fields
|
||||||
private long startTime;
|
private long startTime;
|
||||||
|
@ -163,7 +164,8 @@ public class RMAppImpl implements RMApp {
|
||||||
Configuration config, String name, String user, String queue,
|
Configuration config, String name, String user, String queue,
|
||||||
ApplicationSubmissionContext submissionContext, String clientTokenStr,
|
ApplicationSubmissionContext submissionContext, String clientTokenStr,
|
||||||
ApplicationStore appStore,
|
ApplicationStore appStore,
|
||||||
YarnScheduler scheduler, ApplicationMasterService masterService) {
|
YarnScheduler scheduler, ApplicationMasterService masterService,
|
||||||
|
long submitTime) {
|
||||||
|
|
||||||
this.applicationId = applicationId;
|
this.applicationId = applicationId;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
|
@ -178,6 +180,7 @@ public class RMAppImpl implements RMApp {
|
||||||
this.appStore = appStore;
|
this.appStore = appStore;
|
||||||
this.scheduler = scheduler;
|
this.scheduler = scheduler;
|
||||||
this.masterService = masterService;
|
this.masterService = masterService;
|
||||||
|
this.submitTime = submitTime;
|
||||||
this.startTime = System.currentTimeMillis();
|
this.startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
this.maxRetries = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
|
this.maxRetries = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
|
||||||
|
@ -367,6 +370,11 @@ public class RMAppImpl implements RMApp {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSubmitTime() {
|
||||||
|
return this.submitTime;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getTrackingUrl() {
|
public String getTrackingUrl() {
|
||||||
this.readLock.lock();
|
this.readLock.lock();
|
||||||
|
|
|
@ -42,6 +42,7 @@ public class MockNM {
|
||||||
private final String nodeIdStr;
|
private final String nodeIdStr;
|
||||||
private final int memory;
|
private final int memory;
|
||||||
private final ResourceTrackerService resourceTracker;
|
private final ResourceTrackerService resourceTracker;
|
||||||
|
private final int httpPort = 2;
|
||||||
|
|
||||||
MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
|
MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
|
||||||
this.nodeIdStr = nodeIdStr;
|
this.nodeIdStr = nodeIdStr;
|
||||||
|
@ -53,6 +54,10 @@ public class MockNM {
|
||||||
return nodeId;
|
return nodeId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String getHttpAddress() {
|
||||||
|
return nodeId.getHost() + ":" + String.valueOf(httpPort);
|
||||||
|
}
|
||||||
|
|
||||||
public void containerStatus(Container container) throws Exception {
|
public void containerStatus(Container container) throws Exception {
|
||||||
Map<ApplicationId, List<ContainerStatus>> conts =
|
Map<ApplicationId, List<ContainerStatus>> conts =
|
||||||
new HashMap<ApplicationId, List<ContainerStatus>>();
|
new HashMap<ApplicationId, List<ContainerStatus>>();
|
||||||
|
@ -69,7 +74,7 @@ public class MockNM {
|
||||||
RegisterNodeManagerRequest req = Records.newRecord(
|
RegisterNodeManagerRequest req = Records.newRecord(
|
||||||
RegisterNodeManagerRequest.class);
|
RegisterNodeManagerRequest.class);
|
||||||
req.setNodeId(nodeId);
|
req.setNodeId(nodeId);
|
||||||
req.setHttpPort(2);
|
req.setHttpPort(httpPort);
|
||||||
Resource resource = Records.newRecord(Resource.class);
|
Resource resource = Records.newRecord(Resource.class);
|
||||||
resource.setMemory(memory);
|
resource.setMemory(memory);
|
||||||
req.setResource(resource);
|
req.setResource(resource);
|
||||||
|
|
|
@ -163,7 +163,7 @@ public class TestAppManager{
|
||||||
}
|
}
|
||||||
public void submitApplication(
|
public void submitApplication(
|
||||||
ApplicationSubmissionContext submissionContext) {
|
ApplicationSubmissionContext submissionContext) {
|
||||||
super.submitApplication(submissionContext);
|
super.submitApplication(submissionContext, System.currentTimeMillis());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
|
||||||
|
@ -39,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMaste
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
||||||
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
@ -56,6 +58,9 @@ public class TestApplicationMasterLauncher {
|
||||||
boolean launched = false;
|
boolean launched = false;
|
||||||
boolean cleanedup = false;
|
boolean cleanedup = false;
|
||||||
String attemptIdAtContainerManager = null;
|
String attemptIdAtContainerManager = null;
|
||||||
|
String containerIdAtContainerManager = null;
|
||||||
|
String nmAddressAtContainerManager = null;
|
||||||
|
long submitTimeAtContainerManager;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public StartContainerResponse
|
public StartContainerResponse
|
||||||
|
@ -63,9 +68,20 @@ public class TestApplicationMasterLauncher {
|
||||||
throws YarnRemoteException {
|
throws YarnRemoteException {
|
||||||
LOG.info("Container started by MyContainerManager: " + request);
|
LOG.info("Container started by MyContainerManager: " + request);
|
||||||
launched = true;
|
launched = true;
|
||||||
attemptIdAtContainerManager = request.getContainerLaunchContext()
|
containerIdAtContainerManager =
|
||||||
.getEnvironment().get(
|
request.getContainerLaunchContext().getEnvironment()
|
||||||
ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV);
|
.get(ApplicationConstants.AM_CONTAINER_ID_ENV);
|
||||||
|
ContainerId containerId =
|
||||||
|
ConverterUtils.toContainerId(containerIdAtContainerManager);
|
||||||
|
attemptIdAtContainerManager =
|
||||||
|
containerId.getApplicationAttemptId().toString();
|
||||||
|
nmAddressAtContainerManager =
|
||||||
|
request.getContainerLaunchContext().getEnvironment()
|
||||||
|
.get(ApplicationConstants.NM_HTTP_ADDRESS_ENV);
|
||||||
|
submitTimeAtContainerManager =
|
||||||
|
Long.parseLong(request.getContainerLaunchContext().getEnvironment()
|
||||||
|
.get(ApplicationConstants.APP_SUBMIT_TIME_ENV));
|
||||||
|
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -140,6 +156,13 @@ public class TestApplicationMasterLauncher {
|
||||||
ApplicationAttemptId appAttemptId = attempt.getAppAttemptId();
|
ApplicationAttemptId appAttemptId = attempt.getAppAttemptId();
|
||||||
Assert.assertEquals(appAttemptId.toString(),
|
Assert.assertEquals(appAttemptId.toString(),
|
||||||
containerManager.attemptIdAtContainerManager);
|
containerManager.attemptIdAtContainerManager);
|
||||||
|
Assert.assertEquals(app.getSubmitTime(),
|
||||||
|
containerManager.submitTimeAtContainerManager);
|
||||||
|
Assert.assertEquals(app.getRMAppAttempt(appAttemptId)
|
||||||
|
.getSubmissionContext().getAMContainerSpec().getContainerId()
|
||||||
|
.toString(), containerManager.containerIdAtContainerManager);
|
||||||
|
Assert.assertEquals(nm1.getHttpAddress(),
|
||||||
|
containerManager.nmAddressAtContainerManager);
|
||||||
|
|
||||||
MockAM am = new MockAM(rm.getRMContext(), rm
|
MockAM am = new MockAM(rm.getRMContext(), rm
|
||||||
.getApplicationMasterService(), appAttemptId);
|
.getApplicationMasterService(), appAttemptId);
|
||||||
|
|
|
@ -166,6 +166,11 @@ public abstract class MockAsm extends MockApps {
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSubmitTime() {
|
||||||
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getFinishTime() {
|
public long getFinishTime() {
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
|
|
|
@ -33,6 +33,7 @@ public class MockRMApp implements RMApp {
|
||||||
String name = MockApps.newAppName();
|
String name = MockApps.newAppName();
|
||||||
String queue = MockApps.newQueue();
|
String queue = MockApps.newQueue();
|
||||||
long start = System.currentTimeMillis() - (int) (Math.random() * DT);
|
long start = System.currentTimeMillis() - (int) (Math.random() * DT);
|
||||||
|
long submit = start - (int) (Math.random() * DT);
|
||||||
long finish = 0;
|
long finish = 0;
|
||||||
RMAppState state = RMAppState.NEW;
|
RMAppState state = RMAppState.NEW;
|
||||||
int failCount = 0;
|
int failCount = 0;
|
||||||
|
@ -141,6 +142,11 @@ public class MockRMApp implements RMApp {
|
||||||
return start;
|
return start;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSubmitTime() {
|
||||||
|
return submit;
|
||||||
|
}
|
||||||
|
|
||||||
public void setStartTime(long time) {
|
public void setStartTime(long time) {
|
||||||
this.start = time;
|
this.start = time;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
|
|
||||||
|
@ -51,7 +50,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAlloca
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
||||||
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
|
||||||
|
@ -61,7 +59,7 @@ public class TestRMAppTransitions {
|
||||||
private RMContext rmContext;
|
private RMContext rmContext;
|
||||||
private static int maxRetries = 4;
|
private static int maxRetries = 4;
|
||||||
private static int appId = 1;
|
private static int appId = 1;
|
||||||
private AsyncDispatcher rmDispatcher;
|
// private AsyncDispatcher rmDispatcher;
|
||||||
|
|
||||||
// ignore all the RM application attempt events
|
// ignore all the RM application attempt events
|
||||||
private static final class TestApplicationAttemptEventDispatcher implements
|
private static final class TestApplicationAttemptEventDispatcher implements
|
||||||
|
@ -152,7 +150,7 @@ public class TestRMAppTransitions {
|
||||||
conf, name, user,
|
conf, name, user,
|
||||||
queue, submissionContext, clientTokenStr,
|
queue, submissionContext, clientTokenStr,
|
||||||
appStore, scheduler,
|
appStore, scheduler,
|
||||||
masterService);
|
masterService, System.currentTimeMillis());
|
||||||
|
|
||||||
testAppStartState(applicationId, user, name, queue, application);
|
testAppStartState(applicationId, user, name, queue, application);
|
||||||
return application;
|
return application;
|
||||||
|
|
|
@ -323,25 +323,29 @@ Hadoop MapReduce Next Generation - Writing YARN Applications
|
||||||
multi-tenancy nature, amongst other issues, it cannot make any assumptions
|
multi-tenancy nature, amongst other issues, it cannot make any assumptions
|
||||||
of things like pre-configured ports that it can listen on.
|
of things like pre-configured ports that it can listen on.
|
||||||
|
|
||||||
|
* When the ApplicationMaster starts up, several parameters are made available
|
||||||
|
to it via the environment. These include the ContainerId for the
|
||||||
|
ApplicationMaster container, the application submission time and the HTTP
|
||||||
|
address of the NodeManager running the container. Ref ApplicationConstants
|
||||||
|
for parameter names.
|
||||||
|
|
||||||
* All interactions with the ResourceManager require an ApplicationAttemptId
|
* All interactions with the ResourceManager require an ApplicationAttemptId
|
||||||
(there can be multiple attempts per application in case of failures). When
|
(there can be multiple attempts per application in case of failures). The
|
||||||
the ApplicationMaster starts up, the ApplicationAttemptId associated with
|
ApplicationAttemptId can be obtained from the ApplicationMaster
|
||||||
this particular instance will be set in the environment. There are helper
|
containerId. There are helper apis to convert the value obtained from the
|
||||||
apis to convert the value obtained from the environment into an
|
environment into objects.
|
||||||
ApplicationAttemptId object.
|
|
||||||
|
|
||||||
+---+
|
+---+
|
||||||
Map<String, String> envs = System.getenv();
|
Map<String, String> envs = System.getenv();
|
||||||
ApplicationAttemptId appAttemptID =
|
String containerIdString =
|
||||||
Records.newRecord(ApplicationAttemptId.class);
|
envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV);
|
||||||
if (!envs.containsKey(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV)) {
|
if (containerIdString == null) {
|
||||||
// app attempt id should always be set in the env by the framework
|
// container id should always be set in the env by the framework
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
"ApplicationAttemptId not set in the environment");
|
"ContainerId not set in the environment");
|
||||||
}
|
}
|
||||||
appAttemptID =
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdString);
|
||||||
ConverterUtils.toApplicationAttemptId(
|
ApplicationAttemptId appAttemptID = containerId.getApplicationAttemptId();
|
||||||
envs.get(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV));
|
|
||||||
+---+
|
+---+
|
||||||
|
|
||||||
* After an ApplicationMaster has initialized itself completely, it needs to
|
* After an ApplicationMaster has initialized itself completely, it needs to
|
||||||
|
|
Loading…
Reference in New Issue