|
@@ -20,7 +20,9 @@ package org.apache.hadoop.mapred;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
+import java.util.Map;
|
|
|
import java.util.concurrent.BlockingQueue;
|
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
|
@@ -35,6 +37,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
|
import org.apache.hadoop.mapreduce.JobContext;
|
|
|
import org.apache.hadoop.mapreduce.JobCounter;
|
|
|
import org.apache.hadoop.mapreduce.MRConfig;
|
|
|
+import org.apache.hadoop.mapreduce.TaskID;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
@@ -168,6 +171,10 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
public void run() {
|
|
|
ContainerLauncherEvent event = null;
|
|
|
|
|
|
+ // Collect locations of map outputs to give to reduces
|
|
|
+ Map<TaskAttemptID, MapOutputFile> localMapFiles =
|
|
|
+ new HashMap<TaskAttemptID, MapOutputFile>();
|
|
|
+
|
|
|
// _must_ either run subtasks sequentially or accept expense of new JVMs
|
|
|
// (i.e., fork()), else will get weird failures when maps try to create/
|
|
|
// write same dirname or filename: no chdir() in Java
|
|
@@ -223,7 +230,7 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
context.getEventHandler().handle(jce);
|
|
|
}
|
|
|
runSubtask(remoteTask, ytask.getType(), attemptID, numMapTasks,
|
|
|
- (numReduceTasks > 0));
|
|
|
+ (numReduceTasks > 0), localMapFiles);
|
|
|
|
|
|
} catch (RuntimeException re) {
|
|
|
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptID.getTaskId().getJobId());
|
|
@@ -265,7 +272,8 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
final TaskType taskType,
|
|
|
TaskAttemptId attemptID,
|
|
|
final int numMapTasks,
|
|
|
- boolean renameOutputs)
|
|
|
+ boolean renameOutputs,
|
|
|
+ Map<TaskAttemptID, MapOutputFile> localMapFiles)
|
|
|
throws RuntimeException, IOException {
|
|
|
org.apache.hadoop.mapred.TaskAttemptID classicAttemptID =
|
|
|
TypeConverter.fromYarn(attemptID);
|
|
@@ -309,7 +317,9 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
map.run(conf, umbilical);
|
|
|
|
|
|
if (renameOutputs) {
|
|
|
- renameMapOutputForReduce(conf, attemptID, map.getMapOutputFile());
|
|
|
+ MapOutputFile renamed = renameMapOutputForReduce(conf, attemptID,
|
|
|
+ map.getMapOutputFile());
|
|
|
+ localMapFiles.put(classicAttemptID, renamed);
|
|
|
}
|
|
|
relocalize();
|
|
|
|
|
@@ -335,10 +345,11 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
conf.set(MRConfig.MASTER_ADDRESS, "local"); // bypass shuffle
|
|
|
|
|
|
ReduceTask reduce = (ReduceTask)task;
|
|
|
+ reduce.setLocalMapFiles(localMapFiles);
|
|
|
reduce.setConf(conf);
|
|
|
|
|
|
reduce.run(conf, umbilical);
|
|
|
- //relocalize(); // needed only if more than one reducer supported (is MAPREDUCE-434 fixed yet?)
|
|
|
+ relocalize();
|
|
|
}
|
|
|
|
|
|
} catch (FSError e) {
|
|
@@ -387,15 +398,16 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
* so there are no particular compatibility issues.)
|
|
|
*/
|
|
|
@SuppressWarnings("deprecation")
|
|
|
- private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
|
|
|
- MapOutputFile subMapOutputFile)
|
|
|
- throws IOException {
|
|
|
+ private MapOutputFile renameMapOutputForReduce(JobConf conf,
|
|
|
+ TaskAttemptId mapId, MapOutputFile subMapOutputFile) throws IOException {
|
|
|
FileSystem localFs = FileSystem.getLocal(conf);
|
|
|
// move map output to reduce input
|
|
|
Path mapOut = subMapOutputFile.getOutputFile();
|
|
|
FileStatus mStatus = localFs.getFileStatus(mapOut);
|
|
|
Path reduceIn = subMapOutputFile.getInputFileForWrite(
|
|
|
TypeConverter.fromYarn(mapId).getTaskID(), mStatus.getLen());
|
|
|
+ Path mapOutIndex = new Path(mapOut.toString() + ".index");
|
|
|
+ Path reduceInIndex = new Path(reduceIn.toString() + ".index");
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Renaming map output file for task attempt "
|
|
|
+ mapId.toString() + " from original location " + mapOut.toString()
|
|
@@ -407,6 +419,10 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
}
|
|
|
if (!localFs.rename(mapOut, reduceIn))
|
|
|
throw new IOException("Couldn't rename " + mapOut);
|
|
|
+ if (!localFs.rename(mapOutIndex, reduceInIndex))
|
|
|
+ throw new IOException("Couldn't rename " + mapOutIndex);
|
|
|
+
|
|
|
+ return new RenamedMapOutputFile(reduceIn);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -441,5 +457,70 @@ public class LocalContainerLauncher extends AbstractService implements
|
|
|
}
|
|
|
|
|
|
} // end SubtaskRunner
|
|
|
+
|
|
|
+ private static class RenamedMapOutputFile extends MapOutputFile {
|
|
|
+ private Path path;
|
|
|
+
|
|
|
+ public RenamedMapOutputFile(Path path) {
|
|
|
+ this.path = path;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Path getOutputFile() throws IOException {
|
|
|
+ return path;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Path getOutputFileForWrite(long size) throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getOutputFileForWriteInVolume(Path existing) {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getOutputIndexFile() throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getOutputIndexFileForWrite(long size) throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getOutputIndexFileForWriteInVolume(Path existing) {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getSpillFile(int spillNumber) throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getSpillFileForWrite(int spillNumber, long size)
|
|
|
+ throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getSpillIndexFile(int spillNumber) throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getSpillIndexFileForWrite(int spillNumber, long size)
|
|
|
+ throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getInputFile(int mapId) throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public Path getInputFileForWrite(TaskID mapId, long size)
|
|
|
+ throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void removeAll() throws IOException {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
}
|