|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.text.NumberFormat;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Random;
|
|
@@ -104,11 +105,15 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
splits = job.getInputFormat().getSplits(job, 1);
|
|
|
String jobId = profile.getJobId();
|
|
|
|
|
|
- // run a map task for each split
|
|
|
- job.setNumReduceTasks(1); // force a single reduce task
|
|
|
+ int numReduceTasks = job.getNumReduceTasks();
|
|
|
+ if (numReduceTasks > 1 || numReduceTasks < 0) {
|
|
|
+ // we only allow 0 or 1 reducer in local mode
|
|
|
+ numReduceTasks = 1;
|
|
|
+ job.setNumReduceTasks(1);
|
|
|
+ }
|
|
|
DataOutputBuffer buffer = new DataOutputBuffer();
|
|
|
for (int i = 0; i < splits.length; i++) {
|
|
|
- String mapId = "map_" + newId();
|
|
|
+ String mapId = "map_" + idFormat.format(i);
|
|
|
mapIds.add(mapId);
|
|
|
buffer.reset();
|
|
|
splits[i].write(buffer);
|
|
@@ -129,38 +134,38 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
map_tasks -= 1;
|
|
|
updateCounters(map);
|
|
|
}
|
|
|
-
|
|
|
- // move map output to reduce input
|
|
|
- String reduceId = "reduce_" + newId();
|
|
|
- for (int i = 0; i < mapIds.size(); i++) {
|
|
|
- String mapId = mapIds.get(i);
|
|
|
- Path mapOut = this.mapoutputFile.getOutputFile(mapId);
|
|
|
- Path reduceIn = this.mapoutputFile.getInputFile(i, reduceId);
|
|
|
- if (!localFs.mkdirs(reduceIn.getParent())) {
|
|
|
- throw new IOException("Mkdirs failed to create " +
|
|
|
- reduceIn.getParent().toString());
|
|
|
+ if (numReduceTasks > 0) {
|
|
|
+ // move map output to reduce input
|
|
|
+ String reduceId = "reduce_" + newId();
|
|
|
+ for (int i = 0; i < mapIds.size(); i++) {
|
|
|
+ String mapId = mapIds.get(i);
|
|
|
+ Path mapOut = this.mapoutputFile.getOutputFile(mapId);
|
|
|
+ Path reduceIn = this.mapoutputFile.getInputFile(i, reduceId);
|
|
|
+ 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);
|
|
|
+ this.mapoutputFile.removeAll(mapId);
|
|
|
}
|
|
|
- if (!localFs.rename(mapOut, reduceIn))
|
|
|
- throw new IOException("Couldn't rename " + mapOut);
|
|
|
- this.mapoutputFile.removeAll(mapId);
|
|
|
- }
|
|
|
|
|
|
- {
|
|
|
- ReduceTask reduce = new ReduceTask(jobId, file,
|
|
|
- "tip_r_0001", reduceId, 0, mapIds.size());
|
|
|
- JobConf localConf = new JobConf(job);
|
|
|
- reduce.localizeConfiguration(localConf);
|
|
|
- reduce.setConf(localConf);
|
|
|
- reduce_tasks += 1;
|
|
|
- myMetrics.launchReduce();
|
|
|
- reduce.run(localConf, this);
|
|
|
- reduce.saveTaskOutput();
|
|
|
- myMetrics.completeReduce();
|
|
|
- reduce_tasks -= 1;
|
|
|
- updateCounters(reduce);
|
|
|
+ {
|
|
|
+ ReduceTask reduce = new ReduceTask(jobId, file, "tip_r_0001",
|
|
|
+ reduceId, 0, mapIds.size());
|
|
|
+ JobConf localConf = new JobConf(job);
|
|
|
+ reduce.localizeConfiguration(localConf);
|
|
|
+ reduce.setConf(localConf);
|
|
|
+ reduce_tasks += 1;
|
|
|
+ myMetrics.launchReduce();
|
|
|
+ reduce.run(localConf, this);
|
|
|
+ reduce.saveTaskOutput();
|
|
|
+ myMetrics.completeReduce();
|
|
|
+ reduce_tasks -= 1;
|
|
|
+ updateCounters(reduce);
|
|
|
+ }
|
|
|
+ this.mapoutputFile.removeAll(reduceId);
|
|
|
}
|
|
|
- this.mapoutputFile.removeAll(reduceId);
|
|
|
-
|
|
|
this.status.setRunState(JobStatus.SUCCEEDED);
|
|
|
|
|
|
JobEndNotifier.localRunnerNotification(job, status);
|
|
@@ -293,4 +298,13 @@ class LocalJobRunner implements JobSubmissionProtocol {
|
|
|
return TaskCompletionEvent.EMPTY_ARRAY;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Used for formatting the id numbers
|
|
|
+ */
|
|
|
+ private static NumberFormat idFormat = NumberFormat.getInstance();
|
|
|
+ static {
|
|
|
+ idFormat.setMinimumIntegerDigits(4);
|
|
|
+ idFormat.setGroupingUsed(false);
|
|
|
+ }
|
|
|
+
|
|
|
}
|