|
@@ -51,6 +51,7 @@ import org.apache.hadoop.mapreduce.jobhistory.AMStartedEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.EventReader;
|
|
import org.apache.hadoop.mapreduce.jobhistory.EventReader;
|
|
import org.apache.hadoop.mapreduce.jobhistory.EventType;
|
|
import org.apache.hadoop.mapreduce.jobhistory.EventType;
|
|
import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
|
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryCopyService;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
|
@@ -66,6 +67,7 @@ import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler;
|
|
import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler;
|
|
import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventType;
|
|
import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventType;
|
|
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.JobStateInternal;
|
|
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.mapreduce.v2.app.job.event.JobEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|
@@ -91,6 +93,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
|
|
|
|
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -110,6 +113,7 @@ 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;
|
|
|
|
+import org.apache.hadoop.yarn.event.Event;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.service.AbstractService;
|
|
import org.apache.hadoop.yarn.service.AbstractService;
|
|
import org.apache.hadoop.yarn.service.CompositeService;
|
|
import org.apache.hadoop.yarn.service.CompositeService;
|
|
@@ -179,9 +183,13 @@ public class MRAppMaster extends CompositeService {
|
|
|
|
|
|
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
|
|
|
|
|
|
+ protected UserGroupInformation currentUser; // Will be setup during init
|
|
|
|
|
|
private volatile boolean isLastAMRetry = false;
|
|
private volatile boolean isLastAMRetry = false;
|
|
|
|
+ //Something happened and we should shut down right after we start up.
|
|
|
|
+ boolean errorHappenedShutDown = false;
|
|
|
|
+ private String shutDownMessage = null;
|
|
|
|
+ JobStateInternal forcedState = null;
|
|
|
|
|
|
public MRAppMaster(ApplicationAttemptId applicationAttemptId,
|
|
public MRAppMaster(ApplicationAttemptId applicationAttemptId,
|
|
ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
|
|
ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
|
|
@@ -242,94 +250,175 @@ public class MRAppMaster extends CompositeService {
|
|
newApiCommitter = true;
|
|
newApiCommitter = true;
|
|
LOG.info("Using mapred newApiCommitter.");
|
|
LOG.info("Using mapred newApiCommitter.");
|
|
}
|
|
}
|
|
-
|
|
|
|
- committer = createOutputCommitter(conf);
|
|
|
|
- boolean recoveryEnabled = conf.getBoolean(
|
|
|
|
- MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
|
|
|
- boolean recoverySupportedByCommitter = committer.isRecoverySupported();
|
|
|
|
- if (recoveryEnabled && recoverySupportedByCommitter
|
|
|
|
- && appAttemptID.getAttemptId() > 1) {
|
|
|
|
- LOG.info("Recovery is enabled. "
|
|
|
|
- + "Will try to recover from previous life on best effort basis.");
|
|
|
|
- recoveryServ = createRecoveryService(context);
|
|
|
|
- addIfService(recoveryServ);
|
|
|
|
- dispatcher = recoveryServ.getDispatcher();
|
|
|
|
- clock = recoveryServ.getClock();
|
|
|
|
- inRecovery = true;
|
|
|
|
- } else {
|
|
|
|
- LOG.info("Not starting RecoveryService: recoveryEnabled: "
|
|
|
|
- + recoveryEnabled + " recoverySupportedByCommitter: "
|
|
|
|
- + recoverySupportedByCommitter + " ApplicationAttemptID: "
|
|
|
|
- + appAttemptID.getAttemptId());
|
|
|
|
|
|
+
|
|
|
|
+ boolean copyHistory = false;
|
|
|
|
+ try {
|
|
|
|
+ String user = UserGroupInformation.getCurrentUser().getShortUserName();
|
|
|
|
+ Path stagingDir = MRApps.getStagingAreaDir(conf, user);
|
|
|
|
+ FileSystem fs = getFileSystem(conf);
|
|
|
|
+ boolean stagingExists = fs.exists(stagingDir);
|
|
|
|
+ Path startCommitFile = MRApps.getStartJobCommitFile(conf, user, jobId);
|
|
|
|
+ boolean commitStarted = fs.exists(startCommitFile);
|
|
|
|
+ Path endCommitSuccessFile = MRApps.getEndJobCommitSuccessFile(conf, user, jobId);
|
|
|
|
+ boolean commitSuccess = fs.exists(endCommitSuccessFile);
|
|
|
|
+ Path endCommitFailureFile = MRApps.getEndJobCommitFailureFile(conf, user, jobId);
|
|
|
|
+ boolean commitFailure = fs.exists(endCommitFailureFile);
|
|
|
|
+ if(!stagingExists) {
|
|
|
|
+ isLastAMRetry = true;
|
|
|
|
+ errorHappenedShutDown = true;
|
|
|
|
+ forcedState = JobStateInternal.ERROR;
|
|
|
|
+ shutDownMessage = "Staging dir does not exist " + stagingDir;
|
|
|
|
+ LOG.fatal(shutDownMessage);
|
|
|
|
+ } else if (commitStarted) {
|
|
|
|
+ //A commit was started so this is the last time, we just need to know
|
|
|
|
+ // what result we will use to notify, and how we will unregister
|
|
|
|
+ errorHappenedShutDown = true;
|
|
|
|
+ isLastAMRetry = true;
|
|
|
|
+ copyHistory = true;
|
|
|
|
+ if (commitSuccess) {
|
|
|
|
+ shutDownMessage = "We crashed after successfully committing. Recovering.";
|
|
|
|
+ forcedState = JobStateInternal.SUCCEEDED;
|
|
|
|
+ } else if (commitFailure) {
|
|
|
|
+ shutDownMessage = "We crashed after a commit failure.";
|
|
|
|
+ forcedState = JobStateInternal.FAILED;
|
|
|
|
+ } else {
|
|
|
|
+ //The commit is still pending, commit error
|
|
|
|
+ shutDownMessage = "We crashed durring a commit";
|
|
|
|
+ forcedState = JobStateInternal.ERROR;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ throw new YarnException("Error while initializing", e);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (errorHappenedShutDown) {
|
|
dispatcher = createDispatcher();
|
|
dispatcher = createDispatcher();
|
|
addIfService(dispatcher);
|
|
addIfService(dispatcher);
|
|
- }
|
|
|
|
-
|
|
|
|
- //service to handle requests from JobClient
|
|
|
|
- clientService = createClientService(context);
|
|
|
|
- addIfService(clientService);
|
|
|
|
|
|
+
|
|
|
|
+ EventHandler<JobHistoryEvent> historyService = null;
|
|
|
|
+ if (copyHistory) {
|
|
|
|
+ historyService =
|
|
|
|
+ createJobHistoryHandler(context);
|
|
|
|
+ dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
|
|
|
|
+ historyService);
|
|
|
|
+ }
|
|
|
|
+ NoopEventHandler eater = new NoopEventHandler();
|
|
|
|
+ //We do not have a JobEventDispatcher in this path
|
|
|
|
+ dispatcher.register(JobEventType.class, eater);
|
|
|
|
+
|
|
|
|
+ // service to allocate containers from RM (if non-uber) or to fake it (uber)
|
|
|
|
+ containerAllocator = createContainerAllocator(null, context);
|
|
|
|
+ addIfService(containerAllocator);
|
|
|
|
+ dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
|
|
|
|
+
|
|
|
|
+ if (copyHistory) {
|
|
|
|
+ // Add the staging directory cleaner before the history server but after
|
|
|
|
+ // the container allocator so the staging directory is cleaned after
|
|
|
|
+ // the history has been flushed but before unregistering with the RM.
|
|
|
|
+ addService(createStagingDirCleaningService());
|
|
|
|
+
|
|
|
|
+ // Add the JobHistoryEventHandler last so that it is properly stopped first.
|
|
|
|
+ // This will guarantee that all history-events are flushed before AM goes
|
|
|
|
+ // ahead with shutdown.
|
|
|
|
+ // Note: Even though JobHistoryEventHandler is started last, if any
|
|
|
|
+ // component creates a JobHistoryEvent in the meanwhile, it will be just be
|
|
|
|
+ // queued inside the JobHistoryEventHandler
|
|
|
|
+ addIfService(historyService);
|
|
|
|
+
|
|
|
|
|
|
- containerAllocator = createContainerAllocator(clientService, context);
|
|
|
|
|
|
+ JobHistoryCopyService cpHist = new JobHistoryCopyService(appAttemptID,
|
|
|
|
+ dispatcher.getEventHandler());
|
|
|
|
+ addIfService(cpHist);
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ committer = createOutputCommitter(conf);
|
|
|
|
+ boolean recoveryEnabled = conf.getBoolean(
|
|
|
|
+ MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
|
|
|
+ boolean recoverySupportedByCommitter = committer.isRecoverySupported();
|
|
|
|
+ if (recoveryEnabled && recoverySupportedByCommitter
|
|
|
|
+ && appAttemptID.getAttemptId() > 1) {
|
|
|
|
+ LOG.info("Recovery is enabled. "
|
|
|
|
+ + "Will try to recover from previous life on best effort basis.");
|
|
|
|
+ recoveryServ = createRecoveryService(context);
|
|
|
|
+ addIfService(recoveryServ);
|
|
|
|
+ dispatcher = recoveryServ.getDispatcher();
|
|
|
|
+ clock = recoveryServ.getClock();
|
|
|
|
+ inRecovery = true;
|
|
|
|
+ } else {
|
|
|
|
+ LOG.info("Not starting RecoveryService: recoveryEnabled: "
|
|
|
|
+ + recoveryEnabled + " recoverySupportedByCommitter: "
|
|
|
|
+ + recoverySupportedByCommitter + " ApplicationAttemptID: "
|
|
|
|
+ + appAttemptID.getAttemptId());
|
|
|
|
+ dispatcher = createDispatcher();
|
|
|
|
+ addIfService(dispatcher);
|
|
|
|
+ }
|
|
|
|
|
|
- //service to handle requests to TaskUmbilicalProtocol
|
|
|
|
- taskAttemptListener = createTaskAttemptListener(context);
|
|
|
|
- addIfService(taskAttemptListener);
|
|
|
|
|
|
+ //service to handle requests from JobClient
|
|
|
|
+ clientService = createClientService(context);
|
|
|
|
+ addIfService(clientService);
|
|
|
|
+
|
|
|
|
+ containerAllocator = createContainerAllocator(clientService, context);
|
|
|
|
+
|
|
|
|
+ //service to handle the output committer
|
|
|
|
+ committerEventHandler = createCommitterEventHandler(context, committer);
|
|
|
|
+ addIfService(committerEventHandler);
|
|
|
|
|
|
- //service to handle the output committer
|
|
|
|
- committerEventHandler = createCommitterEventHandler(context, committer);
|
|
|
|
- addIfService(committerEventHandler);
|
|
|
|
|
|
+ //service to handle requests to TaskUmbilicalProtocol
|
|
|
|
+ taskAttemptListener = createTaskAttemptListener(context);
|
|
|
|
+ addIfService(taskAttemptListener);
|
|
|
|
|
|
- //service to log job history events
|
|
|
|
- EventHandler<JobHistoryEvent> historyService =
|
|
|
|
|
|
+ //service to log job history events
|
|
|
|
+ EventHandler<JobHistoryEvent> historyService =
|
|
createJobHistoryHandler(context);
|
|
createJobHistoryHandler(context);
|
|
- dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
|
|
|
|
- historyService);
|
|
|
|
-
|
|
|
|
- this.jobEventDispatcher = new JobEventDispatcher();
|
|
|
|
-
|
|
|
|
- //register the event dispatchers
|
|
|
|
- dispatcher.register(JobEventType.class, jobEventDispatcher);
|
|
|
|
- dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
|
|
|
|
- dispatcher.register(TaskAttemptEventType.class,
|
|
|
|
- new TaskAttemptEventDispatcher());
|
|
|
|
- dispatcher.register(CommitterEventType.class, committerEventHandler);
|
|
|
|
-
|
|
|
|
- if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|
|
|
|
- || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
|
|
|
|
- //optional service to speculate on task attempts' progress
|
|
|
|
- speculator = createSpeculator(conf, context);
|
|
|
|
- addIfService(speculator);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- speculatorEventDispatcher = new SpeculatorEventDispatcher(conf);
|
|
|
|
- dispatcher.register(Speculator.EventType.class,
|
|
|
|
- speculatorEventDispatcher);
|
|
|
|
-
|
|
|
|
- // service to allocate containers from RM (if non-uber) or to fake it (uber)
|
|
|
|
- addIfService(containerAllocator);
|
|
|
|
- dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
|
|
|
|
-
|
|
|
|
- // corresponding service to launch allocated containers via NodeManager
|
|
|
|
- containerLauncher = createContainerLauncher(context);
|
|
|
|
- addIfService(containerLauncher);
|
|
|
|
- dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
|
|
|
|
-
|
|
|
|
- // Add the staging directory cleaner before the history server but after
|
|
|
|
- // the container allocator so the staging directory is cleaned after
|
|
|
|
- // the history has been flushed but before unregistering with the RM.
|
|
|
|
- addService(createStagingDirCleaningService());
|
|
|
|
-
|
|
|
|
- // Add the JobHistoryEventHandler last so that it is properly stopped first.
|
|
|
|
- // This will guarantee that all history-events are flushed before AM goes
|
|
|
|
- // ahead with shutdown.
|
|
|
|
- // Note: Even though JobHistoryEventHandler is started last, if any
|
|
|
|
- // component creates a JobHistoryEvent in the meanwhile, it will be just be
|
|
|
|
- // queued inside the JobHistoryEventHandler
|
|
|
|
- addIfService(historyService);
|
|
|
|
|
|
+ dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
|
|
|
|
+ historyService);
|
|
|
|
+
|
|
|
|
+ this.jobEventDispatcher = new JobEventDispatcher();
|
|
|
|
+
|
|
|
|
+ //register the event dispatchers
|
|
|
|
+ dispatcher.register(JobEventType.class, jobEventDispatcher);
|
|
|
|
+ dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
|
|
|
|
+ dispatcher.register(TaskAttemptEventType.class,
|
|
|
|
+ new TaskAttemptEventDispatcher());
|
|
|
|
+ dispatcher.register(CommitterEventType.class, committerEventHandler);
|
|
|
|
+
|
|
|
|
+ if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|
|
|
|
+ || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
|
|
|
|
+ //optional service to speculate on task attempts' progress
|
|
|
|
+ speculator = createSpeculator(conf, context);
|
|
|
|
+ addIfService(speculator);
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ speculatorEventDispatcher = new SpeculatorEventDispatcher(conf);
|
|
|
|
+ dispatcher.register(Speculator.EventType.class,
|
|
|
|
+ speculatorEventDispatcher);
|
|
|
|
+
|
|
|
|
+ // service to allocate containers from RM (if non-uber) or to fake it (uber)
|
|
|
|
+ addIfService(containerAllocator);
|
|
|
|
+ dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
|
|
|
|
+
|
|
|
|
+ // corresponding service to launch allocated containers via NodeManager
|
|
|
|
+ containerLauncher = createContainerLauncher(context);
|
|
|
|
+ addIfService(containerLauncher);
|
|
|
|
+ dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
|
|
|
|
+
|
|
|
|
+ // Add the staging directory cleaner before the history server but after
|
|
|
|
+ // the container allocator so the staging directory is cleaned after
|
|
|
|
+ // the history has been flushed but before unregistering with the RM.
|
|
|
|
+ addService(createStagingDirCleaningService());
|
|
|
|
+
|
|
|
|
+ // Add the JobHistoryEventHandler last so that it is properly stopped first.
|
|
|
|
+ // This will guarantee that all history-events are flushed before AM goes
|
|
|
|
+ // ahead with shutdown.
|
|
|
|
+ // Note: Even though JobHistoryEventHandler is started last, if any
|
|
|
|
+ // component creates a JobHistoryEvent in the meanwhile, it will be just be
|
|
|
|
+ // queued inside the JobHistoryEventHandler
|
|
|
|
+ addIfService(historyService);
|
|
|
|
+ }
|
|
|
|
+
|
|
super.init(conf);
|
|
super.init(conf);
|
|
} // end of init()
|
|
} // end of init()
|
|
-
|
|
|
|
|
|
+
|
|
protected Dispatcher createDispatcher() {
|
|
protected Dispatcher createDispatcher() {
|
|
return new AsyncDispatcher();
|
|
return new AsyncDispatcher();
|
|
}
|
|
}
|
|
@@ -489,15 +578,20 @@ public class MRAppMaster extends CompositeService {
|
|
appContext.getClock(), getCommitter());
|
|
appContext.getClock(), getCommitter());
|
|
}
|
|
}
|
|
|
|
|
|
- /** Create and initialize (but don't start) a single job. */
|
|
|
|
- protected Job createJob(Configuration conf) {
|
|
|
|
|
|
+ /** Create and initialize (but don't start) a single job.
|
|
|
|
+ * @param forcedState a state to force the job into or null for normal operation.
|
|
|
|
+ * @param diagnostic a diagnostic message to include with the job.
|
|
|
|
+ */
|
|
|
|
+ protected Job createJob(Configuration conf, JobStateInternal forcedState,
|
|
|
|
+ String diagnostic) {
|
|
|
|
|
|
// create single job
|
|
// create single job
|
|
Job newJob =
|
|
Job newJob =
|
|
new JobImpl(jobId, appAttemptID, conf, dispatcher.getEventHandler(),
|
|
new JobImpl(jobId, appAttemptID, conf, dispatcher.getEventHandler(),
|
|
taskAttemptListener, jobTokenSecretManager, fsTokens, clock,
|
|
taskAttemptListener, jobTokenSecretManager, fsTokens, clock,
|
|
completedTasksFromPreviousRun, metrics, newApiCommitter,
|
|
completedTasksFromPreviousRun, metrics, newApiCommitter,
|
|
- currentUser.getUserName(), appSubmitTime, amInfos, context);
|
|
|
|
|
|
+ currentUser.getUserName(), appSubmitTime, amInfos, context,
|
|
|
|
+ forcedState, diagnostic);
|
|
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
|
|
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
|
|
|
|
|
|
dispatcher.register(JobFinishEvent.Type.class,
|
|
dispatcher.register(JobFinishEvent.Type.class,
|
|
@@ -874,7 +968,7 @@ public class MRAppMaster extends CompositeService {
|
|
amInfos.add(amInfo);
|
|
amInfos.add(amInfo);
|
|
|
|
|
|
// /////////////////// Create the job itself.
|
|
// /////////////////// Create the job itself.
|
|
- job = createJob(getConfig());
|
|
|
|
|
|
+ job = createJob(getConfig(), forcedState, shutDownMessage);
|
|
|
|
|
|
// End of creating the job.
|
|
// End of creating the job.
|
|
|
|
|
|
@@ -891,31 +985,33 @@ public class MRAppMaster extends CompositeService {
|
|
// It's more test friendly to put it here.
|
|
// It's more test friendly to put it here.
|
|
DefaultMetricsSystem.initialize("MRAppMaster");
|
|
DefaultMetricsSystem.initialize("MRAppMaster");
|
|
|
|
|
|
- // create a job event for job intialization
|
|
|
|
- JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
|
|
|
|
- // Send init to the job (this does NOT trigger job execution)
|
|
|
|
- // This is a synchronous call, not an event through dispatcher. We want
|
|
|
|
- // job-init to be done completely here.
|
|
|
|
- jobEventDispatcher.handle(initJobEvent);
|
|
|
|
|
|
+ if (!errorHappenedShutDown) {
|
|
|
|
+ // create a job event for job intialization
|
|
|
|
+ JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
|
|
|
|
+ // Send init to the job (this does NOT trigger job execution)
|
|
|
|
+ // This is a synchronous call, not an event through dispatcher. We want
|
|
|
|
+ // job-init to be done completely here.
|
|
|
|
+ jobEventDispatcher.handle(initJobEvent);
|
|
|
|
|
|
|
|
|
|
- // JobImpl's InitTransition is done (call above is synchronous), so the
|
|
|
|
- // "uber-decision" (MR-1220) has been made. Query job and switch to
|
|
|
|
- // ubermode if appropriate (by registering different container-allocator
|
|
|
|
- // and container-launcher services/event-handlers).
|
|
|
|
|
|
+ // JobImpl's InitTransition is done (call above is synchronous), so the
|
|
|
|
+ // "uber-decision" (MR-1220) has been made. Query job and switch to
|
|
|
|
+ // ubermode if appropriate (by registering different container-allocator
|
|
|
|
+ // and container-launcher services/event-handlers).
|
|
|
|
|
|
- if (job.isUber()) {
|
|
|
|
- speculatorEventDispatcher.disableSpeculation();
|
|
|
|
- LOG.info("MRAppMaster uberizing job " + job.getID()
|
|
|
|
- + " in local container (\"uber-AM\") on node "
|
|
|
|
- + nmHost + ":" + nmPort + ".");
|
|
|
|
- } else {
|
|
|
|
- // send init to speculator only for non-uber jobs.
|
|
|
|
- // This won't yet start as dispatcher isn't started yet.
|
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
|
- new SpeculatorEvent(job.getID(), clock.getTime()));
|
|
|
|
- LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
|
|
|
|
- + "job " + job.getID() + ".");
|
|
|
|
|
|
+ if (job.isUber()) {
|
|
|
|
+ speculatorEventDispatcher.disableSpeculation();
|
|
|
|
+ LOG.info("MRAppMaster uberizing job " + job.getID()
|
|
|
|
+ + " in local container (\"uber-AM\") on node "
|
|
|
|
+ + nmHost + ":" + nmPort + ".");
|
|
|
|
+ } else {
|
|
|
|
+ // send init to speculator only for non-uber jobs.
|
|
|
|
+ // This won't yet start as dispatcher isn't started yet.
|
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
|
+ new SpeculatorEvent(job.getID(), clock.getTime()));
|
|
|
|
+ LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
|
|
|
|
+ + "job " + job.getID() + ".");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
//start all the components
|
|
//start all the components
|
|
@@ -1062,6 +1158,17 @@ public class MRAppMaster extends CompositeService {
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Eats events that are not needed in some error cases.
|
|
|
|
+ */
|
|
|
|
+ private static class NoopEventHandler implements EventHandler<Event> {
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void handle(Event event) {
|
|
|
|
+ //Empty
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private static void validateInputParam(String value, String param)
|
|
private static void validateInputParam(String value, String param)
|
|
throws IOException {
|
|
throws IOException {
|
|
if (value == null) {
|
|
if (value == null) {
|
|
@@ -1158,6 +1265,9 @@ public class MRAppMaster extends CompositeService {
|
|
public Object run() throws Exception {
|
|
public Object run() throws Exception {
|
|
appMaster.init(conf);
|
|
appMaster.init(conf);
|
|
appMaster.start();
|
|
appMaster.start();
|
|
|
|
+ if(appMaster.errorHappenedShutDown) {
|
|
|
|
+ throw new IOException("Was asked to shut down.");
|
|
|
|
+ }
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
});
|
|
});
|