|
@@ -89,7 +89,8 @@ public class MergeManager<K, V> {
|
|
new TreeSet<MapOutput<K,V>>(new MapOutputComparator<K, V>());
|
|
new TreeSet<MapOutput<K,V>>(new MapOutputComparator<K, V>());
|
|
private final MergeThread<MapOutput<K,V>, K,V> inMemoryMerger;
|
|
private final MergeThread<MapOutput<K,V>, K,V> inMemoryMerger;
|
|
|
|
|
|
- Set<Path> onDiskMapOutputs = new TreeSet<Path>();
|
|
|
|
|
|
+ Set<MapOutput<K, V>> onDiskMapOutputs =
|
|
|
|
+ new TreeSet<MapOutput<K,V>>(new MapOutputComparator<K, V>());
|
|
private final OnDiskMerger onDiskMerger;
|
|
private final OnDiskMerger onDiskMerger;
|
|
|
|
|
|
private final long memoryLimit;
|
|
private final long memoryLimit;
|
|
@@ -337,7 +338,7 @@ public class MergeManager<K, V> {
|
|
inMemoryMergedMapOutputs.size());
|
|
inMemoryMergedMapOutputs.size());
|
|
}
|
|
}
|
|
|
|
|
|
- public synchronized void closeOnDiskFile(Path file) {
|
|
|
|
|
|
+ public synchronized void closeOnDiskFile(MapOutput<K,V> file) {
|
|
onDiskMapOutputs.add(file);
|
|
onDiskMapOutputs.add(file);
|
|
|
|
|
|
if (onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
|
|
if (onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
|
|
@@ -356,9 +357,17 @@ public class MergeManager<K, V> {
|
|
List<MapOutput<K, V>> memory =
|
|
List<MapOutput<K, V>> memory =
|
|
new ArrayList<MapOutput<K, V>>(inMemoryMergedMapOutputs);
|
|
new ArrayList<MapOutput<K, V>>(inMemoryMergedMapOutputs);
|
|
memory.addAll(inMemoryMapOutputs);
|
|
memory.addAll(inMemoryMapOutputs);
|
|
- List<Path> disk = new ArrayList<Path>(onDiskMapOutputs);
|
|
|
|
|
|
+ List<Path> disk = getDiskMapOutputs();
|
|
return finalMerge(jobConf, rfs, memory, disk);
|
|
return finalMerge(jobConf, rfs, memory, disk);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private List<Path> getDiskMapOutputs(){
|
|
|
|
+ List<Path> result = new ArrayList<Path>();
|
|
|
|
+ for (MapOutput<K, V> item : onDiskMapOutputs){
|
|
|
|
+ result.add(item.getOutputPath());
|
|
|
|
+ }
|
|
|
|
+ return result;
|
|
|
|
+ }
|
|
|
|
|
|
private class IntermediateMemoryToMemoryMerger
|
|
private class IntermediateMemoryToMemoryMerger
|
|
extends MergeThread<MapOutput<K, V>, K, V> {
|
|
extends MergeThread<MapOutput<K, V>, K, V> {
|
|
@@ -456,6 +465,7 @@ public class MergeManager<K, V> {
|
|
codec, null);
|
|
codec, null);
|
|
|
|
|
|
RawKeyValueIterator rIter = null;
|
|
RawKeyValueIterator rIter = null;
|
|
|
|
+ long compressedSize = -1;
|
|
try {
|
|
try {
|
|
LOG.info("Initiating in-memory merge with " + noInMemorySegments +
|
|
LOG.info("Initiating in-memory merge with " + noInMemorySegments +
|
|
" segments...");
|
|
" segments...");
|
|
@@ -475,7 +485,7 @@ public class MergeManager<K, V> {
|
|
combineAndSpill(rIter, reduceCombineInputCounter);
|
|
combineAndSpill(rIter, reduceCombineInputCounter);
|
|
}
|
|
}
|
|
writer.close();
|
|
writer.close();
|
|
-
|
|
|
|
|
|
+ compressedSize = writer.getCompressedLength();
|
|
LOG.info(reduceId +
|
|
LOG.info(reduceId +
|
|
" Merge of the " + noInMemorySegments +
|
|
" Merge of the " + noInMemorySegments +
|
|
" files in-memory complete." +
|
|
" files in-memory complete." +
|
|
@@ -489,21 +499,21 @@ public class MergeManager<K, V> {
|
|
}
|
|
}
|
|
|
|
|
|
// Note the output of the merge
|
|
// Note the output of the merge
|
|
- closeOnDiskFile(outputPath);
|
|
|
|
|
|
+ closeOnDiskFile(new MapOutput<K, V>(outputPath, compressedSize));
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private class OnDiskMerger extends MergeThread<Path,K,V> {
|
|
|
|
|
|
+ private class OnDiskMerger extends MergeThread<MapOutput<K,V>,K,V> {
|
|
|
|
|
|
public OnDiskMerger(MergeManager<K, V> manager) {
|
|
public OnDiskMerger(MergeManager<K, V> manager) {
|
|
- super(manager, Integer.MAX_VALUE, exceptionReporter);
|
|
|
|
|
|
+ super(manager, ioSortFactor, exceptionReporter);
|
|
setName("OnDiskMerger - Thread to merge on-disk map-outputs");
|
|
setName("OnDiskMerger - Thread to merge on-disk map-outputs");
|
|
setDaemon(true);
|
|
setDaemon(true);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void merge(List<Path> inputs) throws IOException {
|
|
|
|
|
|
+ public void merge(List<MapOutput<K,V>> inputs) throws IOException {
|
|
// sanity check
|
|
// sanity check
|
|
if (inputs == null || inputs.isEmpty()) {
|
|
if (inputs == null || inputs.isEmpty()) {
|
|
LOG.info("No ondisk files to merge...");
|
|
LOG.info("No ondisk files to merge...");
|
|
@@ -518,8 +528,8 @@ public class MergeManager<K, V> {
|
|
" map outputs on disk. Triggering merge...");
|
|
" map outputs on disk. Triggering merge...");
|
|
|
|
|
|
// 1. Prepare the list of files to be merged.
|
|
// 1. Prepare the list of files to be merged.
|
|
- for (Path file : inputs) {
|
|
|
|
- approxOutputSize += localFS.getFileStatus(file).getLen();
|
|
|
|
|
|
+ for (MapOutput<K,V> file : inputs) {
|
|
|
|
+ approxOutputSize += localFS.getFileStatus(file.getOutputPath()).getLen();
|
|
}
|
|
}
|
|
|
|
|
|
// add the checksum length
|
|
// add the checksum length
|
|
@@ -536,12 +546,18 @@ public class MergeManager<K, V> {
|
|
(Class<V>) jobConf.getMapOutputValueClass(),
|
|
(Class<V>) jobConf.getMapOutputValueClass(),
|
|
codec, null);
|
|
codec, null);
|
|
RawKeyValueIterator iter = null;
|
|
RawKeyValueIterator iter = null;
|
|
|
|
+ long compressedSize = -1;
|
|
Path tmpDir = new Path(reduceId.toString());
|
|
Path tmpDir = new Path(reduceId.toString());
|
|
|
|
+ Path[] tmpInputs = new Path[inputs.size()];
|
|
|
|
+ int index = 0;
|
|
|
|
+ for (MapOutput<K, V> item : inputs){
|
|
|
|
+ tmpInputs[index++] = item.getOutputPath();
|
|
|
|
+ }
|
|
try {
|
|
try {
|
|
iter = Merger.merge(jobConf, rfs,
|
|
iter = Merger.merge(jobConf, rfs,
|
|
(Class<K>) jobConf.getMapOutputKeyClass(),
|
|
(Class<K>) jobConf.getMapOutputKeyClass(),
|
|
(Class<V>) jobConf.getMapOutputValueClass(),
|
|
(Class<V>) jobConf.getMapOutputValueClass(),
|
|
- codec, inputs.toArray(new Path[inputs.size()]),
|
|
|
|
|
|
+ codec, tmpInputs,
|
|
true, ioSortFactor, tmpDir,
|
|
true, ioSortFactor, tmpDir,
|
|
(RawComparator<K>) jobConf.getOutputKeyComparator(),
|
|
(RawComparator<K>) jobConf.getOutputKeyComparator(),
|
|
reporter, spilledRecordsCounter, null,
|
|
reporter, spilledRecordsCounter, null,
|
|
@@ -549,12 +565,13 @@ public class MergeManager<K, V> {
|
|
|
|
|
|
Merger.writeFile(iter, writer, reporter, jobConf);
|
|
Merger.writeFile(iter, writer, reporter, jobConf);
|
|
writer.close();
|
|
writer.close();
|
|
|
|
+ compressedSize = writer.getCompressedLength();
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
localFS.delete(outputPath, true);
|
|
localFS.delete(outputPath, true);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
- closeOnDiskFile(outputPath);
|
|
|
|
|
|
+ closeOnDiskFile(new MapOutput<K, V>(outputPath, compressedSize));
|
|
|
|
|
|
LOG.info(reduceId +
|
|
LOG.info(reduceId +
|
|
" Finished merging " + inputs.size() +
|
|
" Finished merging " + inputs.size() +
|
|
@@ -778,7 +795,7 @@ public class MergeManager<K, V> {
|
|
// merges. See comment where mergePhaseFinished is being set
|
|
// merges. See comment where mergePhaseFinished is being set
|
|
Progress thisPhase = (mergePhaseFinished) ? null : mergePhase;
|
|
Progress thisPhase = (mergePhaseFinished) ? null : mergePhase;
|
|
RawKeyValueIterator diskMerge = Merger.merge(
|
|
RawKeyValueIterator diskMerge = Merger.merge(
|
|
- job, fs, keyClass, valueClass, diskSegments,
|
|
|
|
|
|
+ job, fs, keyClass, valueClass,codec, diskSegments,
|
|
ioSortFactor, numInMemSegments, tmpDir, comparator,
|
|
ioSortFactor, numInMemSegments, tmpDir, comparator,
|
|
reporter, false, spilledRecordsCounter, null, thisPhase);
|
|
reporter, false, spilledRecordsCounter, null, thisPhase);
|
|
diskSegments.clear();
|
|
diskSegments.clear();
|