|
@@ -79,11 +79,15 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
public static final String LOCAL_MAX_MAPS =
|
|
|
"mapreduce.local.map.tasks.maximum";
|
|
|
|
|
|
+ /** The maximum number of reduce tasks to run in parallel in LocalJobRunner */
|
|
|
+ public static final String LOCAL_MAX_REDUCES =
|
|
|
+ "mapreduce.local.reduce.tasks.maximum";
|
|
|
+
|
|
|
private FileSystem fs;
|
|
|
private HashMap<JobID, Job> jobs = new HashMap<JobID, Job>();
|
|
|
private JobConf conf;
|
|
|
private AtomicInteger map_tasks = new AtomicInteger(0);
|
|
|
- private int reduce_tasks = 0;
|
|
|
+ private AtomicInteger reduce_tasks = new AtomicInteger(0);
|
|
|
final Random rand = new Random();
|
|
|
|
|
|
private LocalJobRunnerMetrics myMetrics = null;
|
|
@@ -115,9 +119,11 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
private JobConf job;
|
|
|
|
|
|
private int numMapTasks;
|
|
|
+ private int numReduceTasks;
|
|
|
private float [] partialMapProgress;
|
|
|
+ private float [] partialReduceProgress;
|
|
|
private Counters [] mapCounters;
|
|
|
- private Counters reduceCounters;
|
|
|
+ private Counters [] reduceCounters;
|
|
|
|
|
|
private JobStatus status;
|
|
|
private List<TaskAttemptID> mapIds = Collections.synchronizedList(
|
|
@@ -184,10 +190,14 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
this.start();
|
|
|
}
|
|
|
|
|
|
+ protected abstract class RunnableWithThrowable implements Runnable {
|
|
|
+ public volatile Throwable storedException;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* A Runnable instance that handles a map task to be run by an executor.
|
|
|
*/
|
|
|
- protected class MapTaskRunnable implements Runnable {
|
|
|
+ protected class MapTaskRunnable extends RunnableWithThrowable {
|
|
|
private final int taskId;
|
|
|
private final TaskSplitMetaInfo info;
|
|
|
private final JobID jobId;
|
|
@@ -198,8 +208,6 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
// where to fetch mapper outputs.
|
|
|
private final Map<TaskAttemptID, MapOutputFile> mapOutputFiles;
|
|
|
|
|
|
- public volatile Throwable storedException;
|
|
|
-
|
|
|
public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId,
|
|
|
Map<TaskAttemptID, MapOutputFile> mapOutputFiles) {
|
|
|
this.info = info;
|
|
@@ -253,12 +261,13 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
* @param mapOutputFiles a mapping from task attempts to output files
|
|
|
* @return a List of Runnables, one per map task.
|
|
|
*/
|
|
|
- protected List<MapTaskRunnable> getMapTaskRunnables(
|
|
|
+ protected List<RunnableWithThrowable> getMapTaskRunnables(
|
|
|
TaskSplitMetaInfo [] taskInfo, JobID jobId,
|
|
|
Map<TaskAttemptID, MapOutputFile> mapOutputFiles) {
|
|
|
|
|
|
int numTasks = 0;
|
|
|
- ArrayList<MapTaskRunnable> list = new ArrayList<MapTaskRunnable>();
|
|
|
+ ArrayList<RunnableWithThrowable> list =
|
|
|
+ new ArrayList<RunnableWithThrowable>();
|
|
|
for (TaskSplitMetaInfo task : taskInfo) {
|
|
|
list.add(new MapTaskRunnable(task, numTasks++, jobId,
|
|
|
mapOutputFiles));
|
|
@@ -267,12 +276,89 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
return list;
|
|
|
}
|
|
|
|
|
|
+ protected class ReduceTaskRunnable extends RunnableWithThrowable {
|
|
|
+ private final int taskId;
|
|
|
+ private final JobID jobId;
|
|
|
+ private final JobConf localConf;
|
|
|
+
|
|
|
+ // This is a reference to a shared object passed in by the
|
|
|
+ // external context; this delivers state to the reducers regarding
|
|
|
+ // where to fetch mapper outputs.
|
|
|
+ private final Map<TaskAttemptID, MapOutputFile> mapOutputFiles;
|
|
|
+
|
|
|
+ public ReduceTaskRunnable(int taskId, JobID jobId,
|
|
|
+ Map<TaskAttemptID, MapOutputFile> mapOutputFiles) {
|
|
|
+ this.taskId = taskId;
|
|
|
+ this.jobId = jobId;
|
|
|
+ this.mapOutputFiles = mapOutputFiles;
|
|
|
+ this.localConf = new JobConf(job);
|
|
|
+ this.localConf.set("mapreduce.jobtracker.address", "local");
|
|
|
+ }
|
|
|
+
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ TaskAttemptID reduceId = new TaskAttemptID(new TaskID(
|
|
|
+ jobId, TaskType.REDUCE, taskId), 0);
|
|
|
+ LOG.info("Starting task: " + reduceId);
|
|
|
+
|
|
|
+ ReduceTask reduce = new ReduceTask(systemJobFile.toString(),
|
|
|
+ reduceId, taskId, mapIds.size(), 1);
|
|
|
+ reduce.setUser(UserGroupInformation.getCurrentUser().
|
|
|
+ getShortUserName());
|
|
|
+ setupChildMapredLocalDirs(localJobDir, reduce, localConf);
|
|
|
+ reduce.setLocalMapFiles(mapOutputFiles);
|
|
|
+
|
|
|
+ if (!Job.this.isInterrupted()) {
|
|
|
+ reduce.setJobFile(localJobFile.toString());
|
|
|
+ localConf.setUser(reduce.getUser());
|
|
|
+ reduce.localizeConfiguration(localConf);
|
|
|
+ reduce.setConf(localConf);
|
|
|
+ try {
|
|
|
+ reduce_tasks.getAndIncrement();
|
|
|
+ myMetrics.launchReduce(reduce.getTaskID());
|
|
|
+ reduce.run(localConf, Job.this);
|
|
|
+ myMetrics.completeReduce(reduce.getTaskID());
|
|
|
+ } finally {
|
|
|
+ reduce_tasks.getAndDecrement();
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("Finishing task: " + reduceId);
|
|
|
+ } else {
|
|
|
+ throw new InterruptedException();
|
|
|
+ }
|
|
|
+ } catch (Throwable t) {
|
|
|
+ // store this to be rethrown in the initial thread context.
|
|
|
+ this.storedException = t;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create Runnables to encapsulate reduce tasks for use by the executor
|
|
|
+ * service.
|
|
|
+ * @param jobId the job id
|
|
|
+ * @param mapOutputFiles a mapping from task attempts to output files
|
|
|
+ * @return a List of Runnables, one per reduce task.
|
|
|
+ */
|
|
|
+ protected List<RunnableWithThrowable> getReduceTaskRunnables(
|
|
|
+ JobID jobId, Map<TaskAttemptID, MapOutputFile> mapOutputFiles) {
|
|
|
+
|
|
|
+ int taskId = 0;
|
|
|
+ ArrayList<RunnableWithThrowable> list =
|
|
|
+ new ArrayList<RunnableWithThrowable>();
|
|
|
+ for (int i = 0; i < this.numReduceTasks; i++) {
|
|
|
+ list.add(new ReduceTaskRunnable(taskId++, jobId, mapOutputFiles));
|
|
|
+ }
|
|
|
+
|
|
|
+ return list;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Initialize the counters that will hold partial-progress from
|
|
|
* the various task attempts.
|
|
|
* @param numMaps the number of map tasks in this job.
|
|
|
*/
|
|
|
- private synchronized void initCounters(int numMaps) {
|
|
|
+ private synchronized void initCounters(int numMaps, int numReduces) {
|
|
|
// Initialize state trackers for all map tasks.
|
|
|
this.partialMapProgress = new float[numMaps];
|
|
|
this.mapCounters = new Counters[numMaps];
|
|
@@ -280,16 +366,22 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
this.mapCounters[i] = new Counters();
|
|
|
}
|
|
|
|
|
|
- this.reduceCounters = new Counters();
|
|
|
+ this.partialReduceProgress = new float[numReduces];
|
|
|
+ this.reduceCounters = new Counters[numReduces];
|
|
|
+ for (int i = 0; i < numReduces; i++) {
|
|
|
+ this.reduceCounters[i] = new Counters();
|
|
|
+ }
|
|
|
+
|
|
|
+ this.numMapTasks = numMaps;
|
|
|
+ this.numReduceTasks = numReduces;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Creates the executor service used to run map tasks.
|
|
|
*
|
|
|
- * @param numMapTasks the total number of map tasks to be run
|
|
|
* @return an ExecutorService instance that handles map tasks
|
|
|
*/
|
|
|
- protected ExecutorService createMapExecutor(int numMapTasks) {
|
|
|
+ protected synchronized ExecutorService createMapExecutor() {
|
|
|
|
|
|
// Determine the size of the thread pool to use
|
|
|
int maxMapThreads = job.getInt(LOCAL_MAX_MAPS, 1);
|
|
@@ -297,13 +389,10 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
throw new IllegalArgumentException(
|
|
|
"Configured " + LOCAL_MAX_MAPS + " must be >= 1");
|
|
|
}
|
|
|
- this.numMapTasks = numMapTasks;
|
|
|
maxMapThreads = Math.min(maxMapThreads, this.numMapTasks);
|
|
|
maxMapThreads = Math.max(maxMapThreads, 1); // In case of no tasks.
|
|
|
|
|
|
- initCounters(this.numMapTasks);
|
|
|
-
|
|
|
- LOG.debug("Starting thread pool executor.");
|
|
|
+ LOG.debug("Starting mapper thread pool executor.");
|
|
|
LOG.debug("Max local threads: " + maxMapThreads);
|
|
|
LOG.debug("Map tasks to process: " + this.numMapTasks);
|
|
|
|
|
@@ -315,6 +404,65 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
|
|
|
return executor;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Creates the executor service used to run reduce tasks.
|
|
|
+ *
|
|
|
+ * @return an ExecutorService instance that handles reduce tasks
|
|
|
+ */
|
|
|
+ protected synchronized ExecutorService createReduceExecutor() {
|
|
|
+
|
|
|
+ // Determine the size of the thread pool to use
|
|
|
+ int maxReduceThreads = job.getInt(LOCAL_MAX_REDUCES, 1);
|
|
|
+ if (maxReduceThreads < 1) {
|
|
|
+ throw new IllegalArgumentException(
|
|
|
+ "Configured " + LOCAL_MAX_REDUCES + " must be >= 1");
|
|
|
+ }
|
|
|
+ maxReduceThreads = Math.min(maxReduceThreads, this.numReduceTasks);
|
|
|
+ maxReduceThreads = Math.max(maxReduceThreads, 1); // In case of no tasks.
|
|
|
+
|
|
|
+ LOG.debug("Starting reduce thread pool executor.");
|
|
|
+ LOG.debug("Max local threads: " + maxReduceThreads);
|
|
|
+ LOG.debug("Reduce tasks to process: " + this.numReduceTasks);
|
|
|
+
|
|
|
+ // Create a new executor service to drain the work queue.
|
|
|
+ ExecutorService executor = Executors.newFixedThreadPool(maxReduceThreads);
|
|
|
+
|
|
|
+ return executor;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Run a set of tasks and waits for them to complete. */
|
|
|
+ private void runTasks(List<RunnableWithThrowable> runnables,
|
|
|
+ ExecutorService service, String taskType) throws Exception {
|
|
|
+ // Start populating the executor with work units.
|
|
|
+ // They may begin running immediately (in other threads).
|
|
|
+ for (Runnable r : runnables) {
|
|
|
+ service.submit(r);
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ service.shutdown(); // Instructs queue to drain.
|
|
|
+
|
|
|
+ // Wait for tasks to finish; do not use a time-based timeout.
|
|
|
+ // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024)
|
|
|
+ LOG.info("Waiting for " + taskType + " tasks");
|
|
|
+ service.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ // Cancel all threads.
|
|
|
+ service.shutdownNow();
|
|
|
+ throw ie;
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info(taskType + " task executor complete.");
|
|
|
+
|
|
|
+ // After waiting for the tasks to complete, if any of these
|
|
|
+ // have thrown an exception, rethrow it now in the main thread context.
|
|
|
+ for (RunnableWithThrowable r : runnables) {
|
|
|
+ if (r.storedException != null) {
|
|
|
+ throw new Exception(r.storedException);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
private org.apache.hadoop.mapreduce.OutputCommitter
|
|
|
createOutputCommitter(boolean newApiCommitter, JobID jobId, Configuration conf) throws Exception {
|
|
@@ -360,94 +508,25 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir);
|
|
|
|
|
|
int numReduceTasks = job.getNumReduceTasks();
|
|
|
- if (numReduceTasks > 1 || numReduceTasks < 0) {
|
|
|
- // we only allow 0 or 1 reducer in local mode
|
|
|
- numReduceTasks = 1;
|
|
|
- job.setNumReduceTasks(1);
|
|
|
- }
|
|
|
outputCommitter.setupJob(jContext);
|
|
|
status.setSetupProgress(1.0f);
|
|
|
|
|
|
Map<TaskAttemptID, MapOutputFile> mapOutputFiles =
|
|
|
Collections.synchronizedMap(new HashMap<TaskAttemptID, MapOutputFile>());
|
|
|
+
|
|
|
+ List<RunnableWithThrowable> mapRunnables = getMapTaskRunnables(
|
|
|
+ taskSplitMetaInfos, jobId, mapOutputFiles);
|
|
|
+
|
|
|
+ initCounters(mapRunnables.size(), numReduceTasks);
|
|
|
+ ExecutorService mapService = createMapExecutor();
|
|
|
+ runTasks(mapRunnables, mapService, "map");
|
|
|
|
|
|
- List<MapTaskRunnable> taskRunnables = getMapTaskRunnables(taskSplitMetaInfos,
|
|
|
- jobId, mapOutputFiles);
|
|
|
- ExecutorService mapService = createMapExecutor(taskRunnables.size());
|
|
|
-
|
|
|
- // Start populating the executor with work units.
|
|
|
- // They may begin running immediately (in other threads).
|
|
|
- for (Runnable r : taskRunnables) {
|
|
|
- mapService.submit(r);
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- mapService.shutdown(); // Instructs queue to drain.
|
|
|
-
|
|
|
- // Wait for tasks to finish; do not use a time-based timeout.
|
|
|
- // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024)
|
|
|
- LOG.info("Waiting for map tasks");
|
|
|
- mapService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
|
|
|
- } catch (InterruptedException ie) {
|
|
|
- // Cancel all threads.
|
|
|
- mapService.shutdownNow();
|
|
|
- throw ie;
|
|
|
- }
|
|
|
-
|
|
|
- LOG.info("Map task executor complete.");
|
|
|
-
|
|
|
- // After waiting for the map tasks to complete, if any of these
|
|
|
- // have thrown an exception, rethrow it now in the main thread context.
|
|
|
- for (MapTaskRunnable r : taskRunnables) {
|
|
|
- if (r.storedException != null) {
|
|
|
- throw new Exception(r.storedException);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- TaskAttemptID reduceId =
|
|
|
- new TaskAttemptID(new TaskID(jobId, TaskType.REDUCE, 0), 0);
|
|
|
try {
|
|
|
if (numReduceTasks > 0) {
|
|
|
- ReduceTask reduce = new ReduceTask(systemJobFile.toString(),
|
|
|
- reduceId, 0, mapIds.size(), 1);
|
|
|
- reduce.setUser(UserGroupInformation.getCurrentUser().
|
|
|
- getShortUserName());
|
|
|
- JobConf localConf = new JobConf(job);
|
|
|
- localConf.set("mapreduce.jobtracker.address", "local");
|
|
|
- setupChildMapredLocalDirs(localJobDir, reduce, localConf);
|
|
|
- // move map output to reduce input
|
|
|
- for (int i = 0; i < mapIds.size(); i++) {
|
|
|
- if (!this.isInterrupted()) {
|
|
|
- TaskAttemptID mapId = mapIds.get(i);
|
|
|
- Path mapOut = mapOutputFiles.get(mapId).getOutputFile();
|
|
|
- MapOutputFile localOutputFile = new MROutputFiles();
|
|
|
- localOutputFile.setConf(localConf);
|
|
|
- Path reduceIn =
|
|
|
- localOutputFile.getInputFileForWrite(mapId.getTaskID(),
|
|
|
- localFs.getFileStatus(mapOut).getLen());
|
|
|
- if (!localFs.mkdirs(reduceIn.getParent())) {
|
|
|
- throw new IOException("Mkdirs failed to create "
|
|
|
- + reduceIn.getParent().toString());
|
|
|
- }
|
|
|
- if (!localFs.rename(mapOut, reduceIn))
|
|
|
- throw new IOException("Couldn't rename " + mapOut);
|
|
|
- } else {
|
|
|
- throw new InterruptedException();
|
|
|
- }
|
|
|
- }
|
|
|
- if (!this.isInterrupted()) {
|
|
|
- reduce.setJobFile(localJobFile.toString());
|
|
|
- localConf.setUser(reduce.getUser());
|
|
|
- reduce.localizeConfiguration(localConf);
|
|
|
- reduce.setConf(localConf);
|
|
|
- reduce_tasks += 1;
|
|
|
- myMetrics.launchReduce(reduce.getTaskID());
|
|
|
- reduce.run(localConf, this);
|
|
|
- myMetrics.completeReduce(reduce.getTaskID());
|
|
|
- reduce_tasks -= 1;
|
|
|
- } else {
|
|
|
- throw new InterruptedException();
|
|
|
- }
|
|
|
+ List<RunnableWithThrowable> reduceRunnables = getReduceTaskRunnables(
|
|
|
+ jobId, mapOutputFiles);
|
|
|
+ ExecutorService reduceService = createReduceExecutor();
|
|
|
+ runTasks(reduceRunnables, reduceService, "reduce");
|
|
|
}
|
|
|
} finally {
|
|
|
for (MapOutputFile output : mapOutputFiles.values()) {
|
|
@@ -465,7 +544,6 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
}
|
|
|
|
|
|
JobEndNotifier.localRunnerNotification(job, status);
|
|
|
-
|
|
|
} catch (Throwable t) {
|
|
|
try {
|
|
|
outputCommitter.abortJob(jContext,
|
|
@@ -511,12 +589,13 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
new ByteArrayInputStream(baos.toByteArray())));
|
|
|
|
|
|
LOG.info(taskStatus.getStateString());
|
|
|
- int taskIndex = mapIds.indexOf(taskId);
|
|
|
- if (taskIndex >= 0) { // mapping
|
|
|
+ int mapTaskIndex = mapIds.indexOf(taskId);
|
|
|
+ if (mapTaskIndex >= 0) {
|
|
|
+ // mapping
|
|
|
float numTasks = (float) this.numMapTasks;
|
|
|
|
|
|
- partialMapProgress[taskIndex] = taskStatus.getProgress();
|
|
|
- mapCounters[taskIndex] = taskStatus.getCounters();
|
|
|
+ partialMapProgress[mapTaskIndex] = taskStatus.getProgress();
|
|
|
+ mapCounters[mapTaskIndex] = taskStatus.getCounters();
|
|
|
|
|
|
float partialProgress = 0.0f;
|
|
|
for (float f : partialMapProgress) {
|
|
@@ -524,8 +603,18 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
}
|
|
|
status.setMapProgress(partialProgress / numTasks);
|
|
|
} else {
|
|
|
- reduceCounters = taskStatus.getCounters();
|
|
|
- status.setReduceProgress(taskStatus.getProgress());
|
|
|
+ // reducing
|
|
|
+ int reduceTaskIndex = taskId.getTaskID().getId();
|
|
|
+ float numTasks = (float) this.numReduceTasks;
|
|
|
+
|
|
|
+ partialReduceProgress[reduceTaskIndex] = taskStatus.getProgress();
|
|
|
+ reduceCounters[reduceTaskIndex] = taskStatus.getCounters();
|
|
|
+
|
|
|
+ float partialProgress = 0.0f;
|
|
|
+ for (float f : partialReduceProgress) {
|
|
|
+ partialProgress += f;
|
|
|
+ }
|
|
|
+ status.setReduceProgress(partialProgress / numTasks);
|
|
|
}
|
|
|
|
|
|
// ignore phase
|
|
@@ -545,7 +634,13 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
for (Counters c : mapCounters) {
|
|
|
current = Counters.sum(current, c);
|
|
|
}
|
|
|
- current = Counters.sum(current, reduceCounters);
|
|
|
+
|
|
|
+ if (null != reduceCounters && reduceCounters.length > 0) {
|
|
|
+ for (Counters c : reduceCounters) {
|
|
|
+ current = Counters.sum(current, c);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
return current;
|
|
|
}
|
|
|
|
|
@@ -684,8 +779,9 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
|
|
|
public ClusterMetrics getClusterMetrics() {
|
|
|
int numMapTasks = map_tasks.get();
|
|
|
- return new ClusterMetrics(numMapTasks, reduce_tasks, numMapTasks,
|
|
|
- reduce_tasks, 0, 0, 1, 1, jobs.size(), 1, 0, 0);
|
|
|
+ int numReduceTasks = reduce_tasks.get();
|
|
|
+ return new ClusterMetrics(numMapTasks, numReduceTasks, numMapTasks,
|
|
|
+ numReduceTasks, 0, 0, 1, 1, jobs.size(), 1, 0, 0);
|
|
|
}
|
|
|
|
|
|
public JobTrackerStatus getJobTrackerStatus() {
|
|
@@ -816,6 +912,27 @@ public class LocalJobRunner implements ClientProtocol {
|
|
|
return job.getConfiguration().getInt(LOCAL_MAX_MAPS, 1);
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Set the max number of reduce tasks to run concurrently in the LocalJobRunner.
|
|
|
+ * @param job the job to configure
|
|
|
+ * @param maxReduces the maximum number of reduce tasks to allow.
|
|
|
+ */
|
|
|
+ public static void setLocalMaxRunningReduces(
|
|
|
+ org.apache.hadoop.mapreduce.JobContext job,
|
|
|
+ int maxReduces) {
|
|
|
+ job.getConfiguration().setInt(LOCAL_MAX_REDUCES, maxReduces);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @return the max number of reduce tasks to run concurrently in the
|
|
|
+ * LocalJobRunner.
|
|
|
+ */
|
|
|
+ public static int getLocalMaxRunningReduces(
|
|
|
+ org.apache.hadoop.mapreduce.JobContext job) {
|
|
|
+ return job.getConfiguration().getInt(LOCAL_MAX_REDUCES, 1);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
|
|
|
) throws IOException,
|