|
@@ -56,12 +56,14 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.local.LocalContainerAllocator;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.recover.Recovery;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
|
|
@@ -83,6 +85,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
+import org.apache.hadoop.yarn.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.service.CompositeService;
|
|
|
import org.apache.hadoop.yarn.service.Service;
|
|
|
|
|
@@ -126,6 +129,7 @@ public class MRAppMaster extends CompositeService {
|
|
|
private TaskAttemptListener taskAttemptListener;
|
|
|
private JobTokenSecretManager jobTokenSecretManager =
|
|
|
new JobTokenSecretManager();
|
|
|
+ private JobEventDispatcher jobEventDispatcher;
|
|
|
|
|
|
private Job job;
|
|
|
|
|
@@ -148,7 +152,7 @@ public class MRAppMaster extends CompositeService {
|
|
|
|
|
|
@Override
|
|
|
public void init(final Configuration conf) {
|
|
|
- context = new RunningAppContext();
|
|
|
+ context = new RunningAppContext(conf);
|
|
|
|
|
|
// Job name is the same as the app name util we support DAG of jobs
|
|
|
// for an app later
|
|
@@ -182,18 +186,17 @@ public class MRAppMaster extends CompositeService {
|
|
|
//service to log job history events
|
|
|
EventHandler<JobHistoryEvent> historyService =
|
|
|
createJobHistoryHandler(context);
|
|
|
- addIfService(historyService);
|
|
|
+ dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
|
|
|
+ historyService);
|
|
|
|
|
|
- JobEventDispatcher synchronousJobEventDispatcher = new JobEventDispatcher();
|
|
|
+ this.jobEventDispatcher = new JobEventDispatcher();
|
|
|
|
|
|
//register the event dispatchers
|
|
|
- dispatcher.register(JobEventType.class, synchronousJobEventDispatcher);
|
|
|
+ dispatcher.register(JobEventType.class, jobEventDispatcher);
|
|
|
dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
|
|
|
dispatcher.register(TaskAttemptEventType.class,
|
|
|
new TaskAttemptEventDispatcher());
|
|
|
dispatcher.register(TaskCleaner.EventType.class, taskCleaner);
|
|
|
- dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
|
|
|
- historyService);
|
|
|
|
|
|
if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|
|
|
|| conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
|
|
@@ -203,10 +206,34 @@ public class MRAppMaster extends CompositeService {
|
|
|
}
|
|
|
|
|
|
dispatcher.register(Speculator.EventType.class,
|
|
|
- new SpeculatorEventDispatcher());
|
|
|
+ new SpeculatorEventDispatcher(conf));
|
|
|
|
|
|
- Credentials fsTokens = new Credentials();
|
|
|
+ // service to allocate containers from RM (if non-uber) or to fake it (uber)
|
|
|
+ containerAllocator = createContainerAllocator(clientService, context);
|
|
|
+ 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 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);
|
|
|
+ } // end of init()
|
|
|
+
|
|
|
+ /** Create and initialize (but don't start) a single job. */
|
|
|
+ protected Job createJob(Configuration conf) {
|
|
|
+
|
|
|
+ // ////////// Obtain the tokens needed by the job. //////////
|
|
|
+ Credentials fsTokens = new Credentials();
|
|
|
UserGroupInformation currentUser = null;
|
|
|
|
|
|
try {
|
|
@@ -234,66 +261,12 @@ public class MRAppMaster extends CompositeService {
|
|
|
} catch (IOException e) {
|
|
|
throw new YarnException(e);
|
|
|
}
|
|
|
-
|
|
|
- super.init(conf);
|
|
|
-
|
|
|
- //---- start of what used to be startJobs() code:
|
|
|
-
|
|
|
- Configuration config = getConfig();
|
|
|
-
|
|
|
- job = createJob(config, fsTokens, currentUser.getUserName());
|
|
|
-
|
|
|
- /** 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) */
|
|
|
- synchronousJobEventDispatcher.handle(initJobEvent);
|
|
|
-
|
|
|
- // send init to speculator. This won't yest start as dispatcher isn't
|
|
|
- // started yet.
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new SpeculatorEvent(job.getID(), clock.getTime()));
|
|
|
-
|
|
|
- // 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()) {
|
|
|
- LOG.info("MRAppMaster uberizing job " + job.getID()
|
|
|
- + " in local container (\"uber-AM\").");
|
|
|
- } else {
|
|
|
- LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
|
|
|
- + "job " + job.getID() + ".");
|
|
|
- }
|
|
|
-
|
|
|
- // service to allocate containers from RM (if non-uber) or to fake it (uber)
|
|
|
- containerAllocator =
|
|
|
- createContainerAllocator(clientService, context, job.isUber());
|
|
|
- addIfService(containerAllocator);
|
|
|
- dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
|
|
|
- if (containerAllocator instanceof Service) {
|
|
|
- ((Service) containerAllocator).init(config);
|
|
|
- }
|
|
|
-
|
|
|
- // corresponding service to launch allocated containers via NodeManager
|
|
|
- containerLauncher = createContainerLauncher(context, job.isUber());
|
|
|
- addIfService(containerLauncher);
|
|
|
- dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
|
|
|
- if (containerLauncher instanceof Service) {
|
|
|
- ((Service) containerLauncher).init(config);
|
|
|
- }
|
|
|
-
|
|
|
- } // end of init()
|
|
|
-
|
|
|
- /** Create and initialize (but don't start) a single job.
|
|
|
- * @param fsTokens */
|
|
|
- protected Job createJob(Configuration conf, Credentials fsTokens,
|
|
|
- String user) {
|
|
|
+ // ////////// End of obtaining the tokens needed by the job. //////////
|
|
|
|
|
|
// create single job
|
|
|
Job newJob = new JobImpl(appID, conf, dispatcher.getEventHandler(),
|
|
|
taskAttemptListener, jobTokenSecretManager, fsTokens, clock, startCount,
|
|
|
- completedTasksFromPreviousRun, metrics, user);
|
|
|
+ completedTasksFromPreviousRun, metrics, currentUser.getUserName());
|
|
|
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
|
|
|
|
|
|
dispatcher.register(JobFinishEvent.Type.class,
|
|
@@ -388,19 +361,13 @@ public class MRAppMaster extends CompositeService {
|
|
|
}
|
|
|
|
|
|
protected ContainerAllocator createContainerAllocator(
|
|
|
- ClientService clientService, AppContext context, boolean isLocal) {
|
|
|
- //return new StaticContainerAllocator(context);
|
|
|
- return isLocal
|
|
|
- ? new LocalContainerAllocator(clientService, context)
|
|
|
- : new RMContainerAllocator(clientService, context);
|
|
|
+ final ClientService clientService, final AppContext context) {
|
|
|
+ return new ContainerAllocatorRouter(clientService, context);
|
|
|
}
|
|
|
|
|
|
- protected ContainerLauncher createContainerLauncher(AppContext context,
|
|
|
- boolean isLocal) {
|
|
|
- return isLocal
|
|
|
- ? new LocalContainerLauncher(context,
|
|
|
- (TaskUmbilicalProtocol) taskAttemptListener)
|
|
|
- : new ContainerLauncherImpl(context);
|
|
|
+ protected ContainerLauncher
|
|
|
+ createContainerLauncher(final AppContext context) {
|
|
|
+ return new ContainerLauncherRouter(context);
|
|
|
}
|
|
|
|
|
|
//TODO:should have an interface for MRClientService
|
|
@@ -440,9 +407,96 @@ public class MRAppMaster extends CompositeService {
|
|
|
return taskAttemptListener;
|
|
|
}
|
|
|
|
|
|
- class RunningAppContext implements AppContext {
|
|
|
+ /**
|
|
|
+ * By the time life-cycle of this router starts, job-init would have already
|
|
|
+ * happened.
|
|
|
+ */
|
|
|
+ private final class ContainerAllocatorRouter extends AbstractService
|
|
|
+ implements ContainerAllocator {
|
|
|
+ private final ClientService clientService;
|
|
|
+ private final AppContext context;
|
|
|
+ private ContainerAllocator containerAllocator;
|
|
|
+
|
|
|
+ ContainerAllocatorRouter(ClientService clientService,
|
|
|
+ AppContext context) {
|
|
|
+ super(ContainerAllocatorRouter.class.getName());
|
|
|
+ this.clientService = clientService;
|
|
|
+ this.context = context;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void start() {
|
|
|
+ if (job.isUber()) {
|
|
|
+ this.containerAllocator = new LocalContainerAllocator(
|
|
|
+ this.clientService, this.context);
|
|
|
+ } else {
|
|
|
+ this.containerAllocator = new RMContainerAllocator(
|
|
|
+ this.clientService, this.context);
|
|
|
+ }
|
|
|
+ ((Service)this.containerAllocator).init(getConfig());
|
|
|
+ ((Service)this.containerAllocator).start();
|
|
|
+ super.start();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void stop() {
|
|
|
+ ((Service)this.containerAllocator).stop();
|
|
|
+ super.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void handle(ContainerAllocatorEvent event) {
|
|
|
+ this.containerAllocator.handle(event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * By the time life-cycle of this router starts, job-init would have already
|
|
|
+ * happened.
|
|
|
+ */
|
|
|
+ private final class ContainerLauncherRouter extends AbstractService
|
|
|
+ implements ContainerLauncher {
|
|
|
+ private final AppContext context;
|
|
|
+ private ContainerLauncher containerLauncher;
|
|
|
+
|
|
|
+ ContainerLauncherRouter(AppContext context) {
|
|
|
+ super(ContainerLauncherRouter.class.getName());
|
|
|
+ this.context = context;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void start() {
|
|
|
+ if (job.isUber()) {
|
|
|
+ this.containerLauncher = new LocalContainerLauncher(context,
|
|
|
+ (TaskUmbilicalProtocol) taskAttemptListener);
|
|
|
+ } else {
|
|
|
+ this.containerLauncher = new ContainerLauncherImpl(context);
|
|
|
+ }
|
|
|
+ ((Service)this.containerLauncher).init(getConfig());
|
|
|
+ ((Service)this.containerLauncher).start();
|
|
|
+ super.start();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void handle(ContainerLauncherEvent event) {
|
|
|
+ this.containerLauncher.handle(event);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void stop() {
|
|
|
+ ((Service)this.containerLauncher).stop();
|
|
|
+ super.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private class RunningAppContext implements AppContext {
|
|
|
|
|
|
- private Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
|
|
|
+ private final Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
|
|
|
+ private final Configuration conf;
|
|
|
+
|
|
|
+ public RunningAppContext(Configuration config) {
|
|
|
+ this.conf = config;
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
|
public ApplicationAttemptId getApplicationAttemptId() {
|
|
@@ -481,7 +535,7 @@ public class MRAppMaster extends CompositeService {
|
|
|
|
|
|
@Override
|
|
|
public CharSequence getUser() {
|
|
|
- return getConfig().get(MRJobConfig.USER_NAME);
|
|
|
+ return this.conf.get(MRJobConfig.USER_NAME);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -492,13 +546,45 @@ public class MRAppMaster extends CompositeService {
|
|
|
|
|
|
@Override
|
|
|
public void start() {
|
|
|
+
|
|
|
+ ///////////////////// Create the job itself.
|
|
|
+ job = createJob(getConfig());
|
|
|
+ // End of creating the job.
|
|
|
+
|
|
|
// metrics system init is really init & start.
|
|
|
// It's more test friendly to put it here.
|
|
|
DefaultMetricsSystem.initialize("MRAppMaster");
|
|
|
|
|
|
- startJobs();
|
|
|
+ /** 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);
|
|
|
+
|
|
|
+ // send init to speculator. This won't yest start as dispatcher isn't
|
|
|
+ // started yet.
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new SpeculatorEvent(job.getID(), clock.getTime()));
|
|
|
+
|
|
|
+ // 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()) {
|
|
|
+ LOG.info("MRAppMaster uberizing job " + job.getID()
|
|
|
+ + " in local container (\"uber-AM\").");
|
|
|
+ } else {
|
|
|
+ LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
|
|
|
+ + "job " + job.getID() + ".");
|
|
|
+ }
|
|
|
+
|
|
|
//start all the components
|
|
|
super.start();
|
|
|
+
|
|
|
+ // All components have started, start the job.
|
|
|
+ startJobs();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -546,10 +632,14 @@ public class MRAppMaster extends CompositeService {
|
|
|
|
|
|
private class SpeculatorEventDispatcher implements
|
|
|
EventHandler<SpeculatorEvent> {
|
|
|
+ private final Configuration conf;
|
|
|
+ public SpeculatorEventDispatcher(Configuration config) {
|
|
|
+ this.conf = config;
|
|
|
+ }
|
|
|
@Override
|
|
|
public void handle(SpeculatorEvent event) {
|
|
|
- if (getConfig().getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|
|
|
- || getConfig().getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
|
|
|
+ if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|
|
|
+ || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
|
|
|
// Speculator IS enabled, direct the event to there.
|
|
|
speculator.handle(event);
|
|
|
}
|