|
@@ -24,6 +24,7 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.io.OutputStream;
|
|
|
+import java.lang.Math;
|
|
|
import java.net.URI;
|
|
|
import java.net.URL;
|
|
|
import java.net.URLClassLoader;
|
|
@@ -335,7 +336,8 @@ class ReduceTask extends Task {
|
|
|
}
|
|
|
|
|
|
FileSystem lfs = FileSystem.getLocal(job);
|
|
|
-
|
|
|
+ FileSystem rfs = ((LocalFileSystem)lfs).getRaw();
|
|
|
+
|
|
|
// Initialize the codec
|
|
|
codec = initCodec();
|
|
|
|
|
@@ -362,7 +364,7 @@ class ReduceTask extends Task {
|
|
|
LOG.info("Initiating final on-disk merge with " + mapFiles.length +
|
|
|
" files");
|
|
|
RawKeyValueIterator rIter =
|
|
|
- Merger.merge(job, lfs,
|
|
|
+ Merger.merge(job,rfs,
|
|
|
job.getMapOutputKeyClass(), job.getMapOutputValueClass(),
|
|
|
codec, mapFiles, !conf.getKeepFailedTaskFiles(),
|
|
|
job.getInt("io.sort.factor", 100), tempDir,
|
|
@@ -509,6 +511,7 @@ class ReduceTask extends Task {
|
|
|
*/
|
|
|
private FileSystem localFileSys;
|
|
|
|
|
|
+ private FileSystem rfs;
|
|
|
/**
|
|
|
* Number of files to merge at a time
|
|
|
*/
|
|
@@ -1215,13 +1218,16 @@ class ReduceTask extends Task {
|
|
|
compressedLength + " raw bytes) " +
|
|
|
"into RAM from " + mapOutputLoc.getTaskAttemptId());
|
|
|
|
|
|
- mapOutput = shuffleInMemory(mapOutputLoc, connection, input, (int)decompressedLength);
|
|
|
+ mapOutput = shuffleInMemory(mapOutputLoc, connection, input,
|
|
|
+ (int)decompressedLength,
|
|
|
+ (int)compressedLength);
|
|
|
} else {
|
|
|
LOG.info("Shuffling " + decompressedLength + " bytes (" +
|
|
|
compressedLength + " raw bytes) " +
|
|
|
"into Local-FS from " + mapOutputLoc.getTaskAttemptId());
|
|
|
|
|
|
- mapOutput = shuffleToDisk(mapOutputLoc, input, filename, compressedLength);
|
|
|
+ mapOutput = shuffleToDisk(mapOutputLoc, input, filename,
|
|
|
+ compressedLength);
|
|
|
}
|
|
|
|
|
|
return mapOutput;
|
|
@@ -1266,7 +1272,8 @@ class ReduceTask extends Task {
|
|
|
private MapOutput shuffleInMemory(MapOutputLocation mapOutputLoc,
|
|
|
URLConnection connection,
|
|
|
InputStream input,
|
|
|
- int mapOutputLength)
|
|
|
+ int mapOutputLength,
|
|
|
+ int compressedLength)
|
|
|
throws IOException, InterruptedException {
|
|
|
// Reserve ram for the map-output
|
|
|
boolean createdNow = ramManager.reserve(mapOutputLength, input);
|
|
@@ -1289,6 +1296,11 @@ class ReduceTask extends Task {
|
|
|
throw ioe;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ IFileInputStream checksumIn =
|
|
|
+ new IFileInputStream(input,compressedLength);
|
|
|
+
|
|
|
+ input = checksumIn;
|
|
|
|
|
|
// Are map-outputs compressed?
|
|
|
if (codec != null) {
|
|
@@ -1402,7 +1414,7 @@ class ReduceTask extends Task {
|
|
|
OutputStream output = null;
|
|
|
long bytesRead = 0;
|
|
|
try {
|
|
|
- output = localFileSys.create(localFilename);
|
|
|
+ output = rfs.create(localFilename);
|
|
|
|
|
|
byte[] buf = new byte[64 * 1024];
|
|
|
int n = input.read(buf, 0, buf.length);
|
|
@@ -1541,7 +1553,9 @@ class ReduceTask extends Task {
|
|
|
(long)(MAX_INMEM_FILESYS_USE * ramManager.getMemoryLimit());
|
|
|
|
|
|
localFileSys = FileSystem.getLocal(conf);
|
|
|
-
|
|
|
+
|
|
|
+ rfs = ((LocalFileSystem)localFileSys).getRaw();
|
|
|
+
|
|
|
// hosts -> next contact time
|
|
|
this.penaltyBox = new LinkedHashMap<String, Long>();
|
|
|
|
|
@@ -2187,7 +2201,7 @@ class ReduceTask extends Task {
|
|
|
approxOutputSize, conf)
|
|
|
.suffix(".merged");
|
|
|
Writer writer =
|
|
|
- new Writer(conf, localFileSys, outputPath,
|
|
|
+ new Writer(conf,rfs, outputPath,
|
|
|
conf.getMapOutputKeyClass(),
|
|
|
conf.getMapOutputValueClass(),
|
|
|
codec);
|
|
@@ -2195,7 +2209,7 @@ class ReduceTask extends Task {
|
|
|
Path tmpDir = new Path(reduceTask.getTaskID().toString());
|
|
|
final Reporter reporter = getReporter(umbilical);
|
|
|
try {
|
|
|
- iter = Merger.merge(conf, localFileSys,
|
|
|
+ iter = Merger.merge(conf, rfs,
|
|
|
conf.getMapOutputKeyClass(),
|
|
|
conf.getMapOutputValueClass(),
|
|
|
codec, mapFiles.toArray(new Path[mapFiles.size()]),
|
|
@@ -2275,7 +2289,7 @@ class ReduceTask extends Task {
|
|
|
reduceTask.getTaskID(), ramfsMergeOutputSize);
|
|
|
|
|
|
Writer writer =
|
|
|
- new Writer(conf, localFileSys, outputPath,
|
|
|
+ new Writer(conf, rfs, outputPath,
|
|
|
conf.getMapOutputKeyClass(),
|
|
|
conf.getMapOutputValueClass(),
|
|
|
codec);
|
|
@@ -2289,7 +2303,7 @@ class ReduceTask extends Task {
|
|
|
LOG.info("Initiating in-memory merge with " + noInMemorySegments +
|
|
|
" segments...");
|
|
|
|
|
|
- rIter = Merger.merge(conf, localFileSys,
|
|
|
+ rIter = Merger.merge(conf, rfs,
|
|
|
(Class<K>)conf.getMapOutputKeyClass(),
|
|
|
(Class<V>)conf.getMapOutputValueClass(),
|
|
|
inMemorySegments, inMemorySegments.size(),
|