|
@@ -54,6 +54,7 @@ import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobUnsuccessfulCompletionEvent;
|
|
|
import org.apache.hadoop.mapreduce.lib.chain.ChainMapper;
|
|
|
import org.apache.hadoop.mapreduce.lib.chain.ChainReducer;
|
|
|
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
|
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
|
|
|
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
|
@@ -583,13 +584,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
if (getState() == JobState.NEW) {
|
|
|
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
|
|
|
appSubmitTime, startTime, finishTime, setupProgress, 0.0f, 0.0f,
|
|
|
- cleanupProgress, remoteJobConfFile.toString(), amInfos);
|
|
|
+ cleanupProgress, remoteJobConfFile.toString(), amInfos, isUber);
|
|
|
}
|
|
|
|
|
|
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
|
|
|
appSubmitTime, startTime, finishTime, setupProgress,
|
|
|
computeProgress(mapTasks), computeProgress(reduceTasks),
|
|
|
- cleanupProgress, remoteJobConfFile.toString(), amInfos);
|
|
|
+ cleanupProgress, remoteJobConfFile.toString(), amInfos, isUber);
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
|
}
|
|
@@ -812,6 +813,129 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
return amInfos;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Decide whether job can be run in uber mode based on various criteria.
|
|
|
+ * @param dataInputLength Total length for all splits
|
|
|
+ */
|
|
|
+ private void makeUberDecision(long dataInputLength) {
|
|
|
+ //FIXME: need new memory criterion for uber-decision (oops, too late here;
|
|
|
+ // until AM-resizing supported,
|
|
|
+ // must depend on job client to pass fat-slot needs)
|
|
|
+ // these are no longer "system" settings, necessarily; user may override
|
|
|
+ int sysMaxMaps = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
|
|
|
+
|
|
|
+ //FIXME: handling multiple reduces within a single AM does not seem to
|
|
|
+ //work.
|
|
|
+ // int sysMaxReduces =
|
|
|
+ // job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
|
|
|
+ int sysMaxReduces = 1;
|
|
|
+
|
|
|
+ long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
|
|
|
+ conf.getLong("dfs.block.size", 64*1024*1024)); //FIXME: this is
|
|
|
+ // wrong; get FS from [File?]InputFormat and default block size from that
|
|
|
+
|
|
|
+ long sysMemSizeForUberSlot =
|
|
|
+ conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
|
|
|
+ MRJobConfig.DEFAULT_MR_AM_VMEM_MB);
|
|
|
+
|
|
|
+ boolean uberEnabled =
|
|
|
+ conf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
|
|
+ boolean smallNumMapTasks = (numMapTasks <= sysMaxMaps);
|
|
|
+ boolean smallNumReduceTasks = (numReduceTasks <= sysMaxReduces);
|
|
|
+ boolean smallInput = (dataInputLength <= sysMaxBytes);
|
|
|
+ // ignoring overhead due to UberAM and statics as negligible here:
|
|
|
+ boolean smallMemory =
|
|
|
+ ( (Math.max(conf.getLong(MRJobConfig.MAP_MEMORY_MB, 0),
|
|
|
+ conf.getLong(MRJobConfig.REDUCE_MEMORY_MB, 0))
|
|
|
+ <= sysMemSizeForUberSlot)
|
|
|
+ || (sysMemSizeForUberSlot == JobConf.DISABLED_MEMORY_LIMIT));
|
|
|
+ boolean notChainJob = !isChainJob(conf);
|
|
|
+
|
|
|
+ // User has overall veto power over uberization, or user can modify
|
|
|
+ // limits (overriding system settings and potentially shooting
|
|
|
+ // themselves in the head). Note that ChainMapper/Reducer are
|
|
|
+ // fundamentally incompatible with MR-1220; they employ a blocking
|
|
|
+ // queue between the maps/reduces and thus require parallel execution,
|
|
|
+ // while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
|
|
|
+ // and thus requires sequential execution.
|
|
|
+ isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
|
|
|
+ && smallInput && smallMemory && notChainJob;
|
|
|
+
|
|
|
+ if (isUber) {
|
|
|
+ LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
|
|
|
+ + numReduceTasks + "r tasks (" + dataInputLength
|
|
|
+ + " input bytes) will run sequentially on single node.");
|
|
|
+
|
|
|
+ // make sure reduces are scheduled only after all map are completed
|
|
|
+ conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
|
|
|
+ 1.0f);
|
|
|
+ // uber-subtask attempts all get launched on same node; if one fails,
|
|
|
+ // probably should retry elsewhere, i.e., move entire uber-AM: ergo,
|
|
|
+ // limit attempts to 1 (or at most 2? probably not...)
|
|
|
+ conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
|
|
|
+ conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 1);
|
|
|
+
|
|
|
+ // disable speculation
|
|
|
+ conf.setBoolean(MRJobConfig.MAP_SPECULATIVE, false);
|
|
|
+ conf.setBoolean(MRJobConfig.REDUCE_SPECULATIVE, false);
|
|
|
+ } else {
|
|
|
+ StringBuilder msg = new StringBuilder();
|
|
|
+ msg.append("Not uberizing ").append(jobId).append(" because:");
|
|
|
+ if (!uberEnabled)
|
|
|
+ msg.append(" not enabled;");
|
|
|
+ if (!smallNumMapTasks)
|
|
|
+ msg.append(" too many maps;");
|
|
|
+ if (!smallNumReduceTasks)
|
|
|
+ msg.append(" too many reduces;");
|
|
|
+ if (!smallInput)
|
|
|
+ msg.append(" too much input;");
|
|
|
+ if (!smallMemory)
|
|
|
+ msg.append(" too much RAM;");
|
|
|
+ if (!notChainJob)
|
|
|
+ msg.append(" chainjob");
|
|
|
+ LOG.info(msg.toString());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * ChainMapper and ChainReducer must execute in parallel, so they're not
|
|
|
+ * compatible with uberization/LocalContainerLauncher (100% sequential).
|
|
|
+ */
|
|
|
+ private boolean isChainJob(Configuration conf) {
|
|
|
+ boolean isChainJob = false;
|
|
|
+ try {
|
|
|
+ String mapClassName = conf.get(MRJobConfig.MAP_CLASS_ATTR);
|
|
|
+ if (mapClassName != null) {
|
|
|
+ Class<?> mapClass = Class.forName(mapClassName);
|
|
|
+ if (ChainMapper.class.isAssignableFrom(mapClass))
|
|
|
+ isChainJob = true;
|
|
|
+ }
|
|
|
+ } catch (ClassNotFoundException cnfe) {
|
|
|
+ // don't care; assume it's not derived from ChainMapper
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ String reduceClassName = conf.get(MRJobConfig.REDUCE_CLASS_ATTR);
|
|
|
+ if (reduceClassName != null) {
|
|
|
+ Class<?> reduceClass = Class.forName(reduceClassName);
|
|
|
+ if (ChainReducer.class.isAssignableFrom(reduceClass))
|
|
|
+ isChainJob = true;
|
|
|
+ }
|
|
|
+ } catch (ClassNotFoundException cnfe) {
|
|
|
+ // don't care; assume it's not derived from ChainReducer
|
|
|
+ }
|
|
|
+ return isChainJob;
|
|
|
+ }
|
|
|
+
|
|
|
+ /*
|
|
|
+ private int getBlockSize() {
|
|
|
+ String inputClassName = conf.get(MRJobConfig.INPUT_FORMAT_CLASS_ATTR);
|
|
|
+ if (inputClassName != null) {
|
|
|
+ Class<?> inputClass - Class.forName(inputClassName);
|
|
|
+ if (FileInputFormat<K, V>)
|
|
|
+ }
|
|
|
+ }
|
|
|
+ */
|
|
|
+
|
|
|
public static class InitTransition
|
|
|
implements MultipleArcTransition<JobImpl, JobEvent, JobState> {
|
|
|
|
|
@@ -863,81 +987,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
inputLength += taskSplitMetaInfo[i].getInputDataLength();
|
|
|
}
|
|
|
|
|
|
- //FIXME: need new memory criterion for uber-decision (oops, too late here;
|
|
|
- // until AM-resizing supported, must depend on job client to pass fat-slot needs)
|
|
|
- // these are no longer "system" settings, necessarily; user may override
|
|
|
- int sysMaxMaps = job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
|
|
|
- int sysMaxReduces =
|
|
|
- job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
|
|
|
- long sysMaxBytes = job.conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
|
|
|
- job.conf.getLong("dfs.block.size", 64*1024*1024)); //FIXME: this is
|
|
|
- // wrong; get FS from [File?]InputFormat and default block size from that
|
|
|
- //long sysMemSizeForUberSlot = JobTracker.getMemSizeForReduceSlot();
|
|
|
- // FIXME [could use default AM-container memory size...]
|
|
|
-
|
|
|
- boolean uberEnabled =
|
|
|
- job.conf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
|
|
- boolean smallNumMapTasks = (job.numMapTasks <= sysMaxMaps);
|
|
|
- boolean smallNumReduceTasks = (job.numReduceTasks <= sysMaxReduces);
|
|
|
- boolean smallInput = (inputLength <= sysMaxBytes);
|
|
|
- boolean smallMemory = true; //FIXME (see above)
|
|
|
- // ignoring overhead due to UberTask and statics as negligible here:
|
|
|
- // FIXME && (Math.max(memoryPerMap, memoryPerReduce) <= sysMemSizeForUberSlot
|
|
|
- // || sysMemSizeForUberSlot == JobConf.DISABLED_MEMORY_LIMIT)
|
|
|
- boolean notChainJob = !isChainJob(job.conf);
|
|
|
-
|
|
|
- // User has overall veto power over uberization, or user can modify
|
|
|
- // limits (overriding system settings and potentially shooting
|
|
|
- // themselves in the head). Note that ChainMapper/Reducer are
|
|
|
- // fundamentally incompatible with MR-1220; they employ a blocking
|
|
|
-
|
|
|
- // User has overall veto power over uberization, or user can modify
|
|
|
- // limits (overriding system settings and potentially shooting
|
|
|
- // themselves in the head). Note that ChainMapper/Reducer are
|
|
|
- // fundamentally incompatible with MR-1220; they employ a blocking
|
|
|
- // queue between the maps/reduces and thus require parallel execution,
|
|
|
- // while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
|
|
|
- // and thus requires sequential execution.
|
|
|
- job.isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
|
|
|
- && smallInput && smallMemory && notChainJob;
|
|
|
-
|
|
|
- if (job.isUber) {
|
|
|
- LOG.info("Uberizing job " + job.jobId + ": " + job.numMapTasks + "m+"
|
|
|
- + job.numReduceTasks + "r tasks (" + inputLength
|
|
|
- + " input bytes) will run sequentially on single node.");
|
|
|
- //TODO: also note which node?
|
|
|
-
|
|
|
- // make sure reduces are scheduled only after all map are completed
|
|
|
- job.conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
|
|
|
- 1.0f);
|
|
|
- // uber-subtask attempts all get launched on same node; if one fails,
|
|
|
- // probably should retry elsewhere, i.e., move entire uber-AM: ergo,
|
|
|
- // limit attempts to 1 (or at most 2? probably not...)
|
|
|
- job.conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
|
|
|
- job.conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 1);
|
|
|
-
|
|
|
- // disable speculation: makes no sense to speculate an entire job
|
|
|
- //canSpeculateMaps = canSpeculateReduces = false; // [TODO: in old
|
|
|
- //version, ultimately was from conf.getMapSpeculativeExecution(),
|
|
|
- //conf.getReduceSpeculativeExecution()]
|
|
|
- } else {
|
|
|
- StringBuilder msg = new StringBuilder();
|
|
|
- msg.append("Not uberizing ").append(job.jobId).append(" because:");
|
|
|
- if (!uberEnabled)
|
|
|
- msg.append(" not enabled;");
|
|
|
- if (!smallNumMapTasks)
|
|
|
- msg.append(" too many maps;");
|
|
|
- if (!smallNumReduceTasks)
|
|
|
- msg.append(" too many reduces;");
|
|
|
- if (!smallInput)
|
|
|
- msg.append(" too much input;");
|
|
|
- if (!smallMemory)
|
|
|
- msg.append(" too much RAM;");
|
|
|
- if (!notChainJob)
|
|
|
- msg.append(" chainjob");
|
|
|
- LOG.info(msg.toString());
|
|
|
- }
|
|
|
-
|
|
|
+ job.makeUberDecision(inputLength);
|
|
|
+
|
|
|
job.taskAttemptCompletionEvents =
|
|
|
new ArrayList<TaskAttemptCompletionEvent>(
|
|
|
job.numMapTasks + job.numReduceTasks + 10);
|
|
@@ -1008,35 +1059,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * ChainMapper and ChainReducer must execute in parallel, so they're not
|
|
|
- * compatible with uberization/LocalContainerLauncher (100% sequential).
|
|
|
- */
|
|
|
- boolean isChainJob(Configuration conf) {
|
|
|
- boolean isChainJob = false;
|
|
|
- try {
|
|
|
- String mapClassName = conf.get(MRJobConfig.MAP_CLASS_ATTR);
|
|
|
- if (mapClassName != null) {
|
|
|
- Class<?> mapClass = Class.forName(mapClassName);
|
|
|
- if (ChainMapper.class.isAssignableFrom(mapClass))
|
|
|
- isChainJob = true;
|
|
|
- }
|
|
|
- } catch (ClassNotFoundException cnfe) {
|
|
|
- // don't care; assume it's not derived from ChainMapper
|
|
|
- }
|
|
|
- try {
|
|
|
- String reduceClassName = conf.get(MRJobConfig.REDUCE_CLASS_ATTR);
|
|
|
- if (reduceClassName != null) {
|
|
|
- Class<?> reduceClass = Class.forName(reduceClassName);
|
|
|
- if (ChainReducer.class.isAssignableFrom(reduceClass))
|
|
|
- isChainJob = true;
|
|
|
- }
|
|
|
- } catch (ClassNotFoundException cnfe) {
|
|
|
- // don't care; assume it's not derived from ChainReducer
|
|
|
- }
|
|
|
- return isChainJob;
|
|
|
- }
|
|
|
-
|
|
|
private void createMapTasks(JobImpl job, long inputLength,
|
|
|
TaskSplitMetaInfo[] splits) {
|
|
|
for (int i=0; i < job.numMapTasks; ++i) {
|