|
@@ -89,7 +89,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
new TreeSet<InMemoryMapOutput<K,V>>(new MapOutputComparator<K, V>());
|
|
new TreeSet<InMemoryMapOutput<K,V>>(new MapOutputComparator<K, V>());
|
|
private final MergeThread<InMemoryMapOutput<K,V>, K,V> inMemoryMerger;
|
|
private final MergeThread<InMemoryMapOutput<K,V>, K,V> inMemoryMerger;
|
|
|
|
|
|
- Set<Path> onDiskMapOutputs = new TreeSet<Path>();
|
|
|
|
|
|
+ Set<CompressAwarePath> onDiskMapOutputs = new TreeSet<CompressAwarePath>();
|
|
private final OnDiskMerger onDiskMerger;
|
|
private final OnDiskMerger onDiskMerger;
|
|
|
|
|
|
private final long memoryLimit;
|
|
private final long memoryLimit;
|
|
@@ -336,7 +336,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
inMemoryMergedMapOutputs.size());
|
|
inMemoryMergedMapOutputs.size());
|
|
}
|
|
}
|
|
|
|
|
|
- public synchronized void closeOnDiskFile(Path file) {
|
|
|
|
|
|
+ public synchronized void closeOnDiskFile(CompressAwarePath file) {
|
|
onDiskMapOutputs.add(file);
|
|
onDiskMapOutputs.add(file);
|
|
|
|
|
|
if (onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
|
|
if (onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
|
|
@@ -356,7 +356,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
List<InMemoryMapOutput<K, V>> memory =
|
|
List<InMemoryMapOutput<K, V>> memory =
|
|
new ArrayList<InMemoryMapOutput<K, V>>(inMemoryMergedMapOutputs);
|
|
new ArrayList<InMemoryMapOutput<K, V>>(inMemoryMergedMapOutputs);
|
|
memory.addAll(inMemoryMapOutputs);
|
|
memory.addAll(inMemoryMapOutputs);
|
|
- List<Path> disk = new ArrayList<Path>(onDiskMapOutputs);
|
|
|
|
|
|
+ List<CompressAwarePath> disk = new ArrayList<CompressAwarePath>(onDiskMapOutputs);
|
|
return finalMerge(jobConf, rfs, memory, disk);
|
|
return finalMerge(jobConf, rfs, memory, disk);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -456,6 +456,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
codec, null);
|
|
codec, null);
|
|
|
|
|
|
RawKeyValueIterator rIter = null;
|
|
RawKeyValueIterator rIter = null;
|
|
|
|
+ CompressAwarePath compressAwarePath;
|
|
try {
|
|
try {
|
|
LOG.info("Initiating in-memory merge with " + noInMemorySegments +
|
|
LOG.info("Initiating in-memory merge with " + noInMemorySegments +
|
|
" segments...");
|
|
" segments...");
|
|
@@ -474,6 +475,8 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
combineCollector.setWriter(writer);
|
|
combineCollector.setWriter(writer);
|
|
combineAndSpill(rIter, reduceCombineInputCounter);
|
|
combineAndSpill(rIter, reduceCombineInputCounter);
|
|
}
|
|
}
|
|
|
|
+ compressAwarePath = new CompressAwarePath(outputPath,
|
|
|
|
+ writer.getRawLength());
|
|
writer.close();
|
|
writer.close();
|
|
|
|
|
|
LOG.info(reduceId +
|
|
LOG.info(reduceId +
|
|
@@ -489,12 +492,12 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
}
|
|
}
|
|
|
|
|
|
// Note the output of the merge
|
|
// Note the output of the merge
|
|
- closeOnDiskFile(outputPath);
|
|
|
|
|
|
+ closeOnDiskFile(compressAwarePath);
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private class OnDiskMerger extends MergeThread<Path,K,V> {
|
|
|
|
|
|
+ private class OnDiskMerger extends MergeThread<CompressAwarePath,K,V> {
|
|
|
|
|
|
public OnDiskMerger(MergeManagerImpl<K, V> manager) {
|
|
public OnDiskMerger(MergeManagerImpl<K, V> manager) {
|
|
super(manager, Integer.MAX_VALUE, exceptionReporter);
|
|
super(manager, Integer.MAX_VALUE, exceptionReporter);
|
|
@@ -503,7 +506,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void merge(List<Path> inputs) throws IOException {
|
|
|
|
|
|
+ public void merge(List<CompressAwarePath> 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,7 +521,7 @@ public class MergeManagerImpl<K, V> implements 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) {
|
|
|
|
|
|
+ for (CompressAwarePath file : inputs) {
|
|
approxOutputSize += localFS.getFileStatus(file).getLen();
|
|
approxOutputSize += localFS.getFileStatus(file).getLen();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -536,6 +539,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
(Class<V>) jobConf.getMapOutputValueClass(),
|
|
(Class<V>) jobConf.getMapOutputValueClass(),
|
|
codec, null);
|
|
codec, null);
|
|
RawKeyValueIterator iter = null;
|
|
RawKeyValueIterator iter = null;
|
|
|
|
+ CompressAwarePath compressAwarePath;
|
|
Path tmpDir = new Path(reduceId.toString());
|
|
Path tmpDir = new Path(reduceId.toString());
|
|
try {
|
|
try {
|
|
iter = Merger.merge(jobConf, rfs,
|
|
iter = Merger.merge(jobConf, rfs,
|
|
@@ -548,13 +552,15 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
mergedMapOutputsCounter, null);
|
|
mergedMapOutputsCounter, null);
|
|
|
|
|
|
Merger.writeFile(iter, writer, reporter, jobConf);
|
|
Merger.writeFile(iter, writer, reporter, jobConf);
|
|
|
|
+ compressAwarePath = new CompressAwarePath(outputPath,
|
|
|
|
+ writer.getRawLength());
|
|
writer.close();
|
|
writer.close();
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
localFS.delete(outputPath, true);
|
|
localFS.delete(outputPath, true);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
- closeOnDiskFile(outputPath);
|
|
|
|
|
|
+ closeOnDiskFile(compressAwarePath);
|
|
|
|
|
|
LOG.info(reduceId +
|
|
LOG.info(reduceId +
|
|
" Finished merging " + inputs.size() +
|
|
" Finished merging " + inputs.size() +
|
|
@@ -653,7 +659,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
|
|
|
|
private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
|
|
private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
|
|
List<InMemoryMapOutput<K,V>> inMemoryMapOutputs,
|
|
List<InMemoryMapOutput<K,V>> inMemoryMapOutputs,
|
|
- List<Path> onDiskMapOutputs
|
|
|
|
|
|
+ List<CompressAwarePath> onDiskMapOutputs
|
|
) throws IOException {
|
|
) throws IOException {
|
|
LOG.info("finalMerge called with " +
|
|
LOG.info("finalMerge called with " +
|
|
inMemoryMapOutputs.size() + " in-memory map-outputs and " +
|
|
inMemoryMapOutputs.size() + " in-memory map-outputs and " +
|
|
@@ -712,7 +718,8 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
try {
|
|
try {
|
|
Merger.writeFile(rIter, writer, reporter, job);
|
|
Merger.writeFile(rIter, writer, reporter, job);
|
|
// add to list of final disk outputs.
|
|
// add to list of final disk outputs.
|
|
- onDiskMapOutputs.add(outputPath);
|
|
|
|
|
|
+ onDiskMapOutputs.add(new CompressAwarePath(outputPath,
|
|
|
|
+ writer.getRawLength()));
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
if (null != outputPath) {
|
|
if (null != outputPath) {
|
|
try {
|
|
try {
|
|
@@ -742,15 +749,19 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
// segments on disk
|
|
// segments on disk
|
|
List<Segment<K,V>> diskSegments = new ArrayList<Segment<K,V>>();
|
|
List<Segment<K,V>> diskSegments = new ArrayList<Segment<K,V>>();
|
|
long onDiskBytes = inMemToDiskBytes;
|
|
long onDiskBytes = inMemToDiskBytes;
|
|
- Path[] onDisk = onDiskMapOutputs.toArray(new Path[onDiskMapOutputs.size()]);
|
|
|
|
- for (Path file : onDisk) {
|
|
|
|
- onDiskBytes += fs.getFileStatus(file).getLen();
|
|
|
|
- LOG.debug("Disk file: " + file + " Length is " +
|
|
|
|
- fs.getFileStatus(file).getLen());
|
|
|
|
|
|
+ long rawBytes = inMemToDiskBytes;
|
|
|
|
+ CompressAwarePath[] onDisk = onDiskMapOutputs.toArray(
|
|
|
|
+ new CompressAwarePath[onDiskMapOutputs.size()]);
|
|
|
|
+ for (CompressAwarePath file : onDisk) {
|
|
|
|
+ long fileLength = fs.getFileStatus(file).getLen();
|
|
|
|
+ onDiskBytes += fileLength;
|
|
|
|
+ rawBytes += (file.getRawDataLength() > 0) ? file.getRawDataLength() : fileLength;
|
|
|
|
+
|
|
|
|
+ LOG.debug("Disk file: " + file + " Length is " + fileLength);
|
|
diskSegments.add(new Segment<K, V>(job, fs, file, codec, keepInputs,
|
|
diskSegments.add(new Segment<K, V>(job, fs, file, codec, keepInputs,
|
|
(file.toString().endsWith(
|
|
(file.toString().endsWith(
|
|
Task.MERGED_OUTPUT_PREFIX) ?
|
|
Task.MERGED_OUTPUT_PREFIX) ?
|
|
- null : mergedMapOutputsCounter)
|
|
|
|
|
|
+ null : mergedMapOutputsCounter), file.getRawDataLength()
|
|
));
|
|
));
|
|
}
|
|
}
|
|
LOG.info("Merging " + onDisk.length + " files, " +
|
|
LOG.info("Merging " + onDisk.length + " files, " +
|
|
@@ -786,7 +797,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
return diskMerge;
|
|
return diskMerge;
|
|
}
|
|
}
|
|
finalSegments.add(new Segment<K,V>(
|
|
finalSegments.add(new Segment<K,V>(
|
|
- new RawKVIteratorReader(diskMerge, onDiskBytes), true));
|
|
|
|
|
|
+ new RawKVIteratorReader(diskMerge, onDiskBytes), true, rawBytes));
|
|
}
|
|
}
|
|
return Merger.merge(job, fs, keyClass, valueClass,
|
|
return Merger.merge(job, fs, keyClass, valueClass,
|
|
finalSegments, finalSegments.size(), tmpDir,
|
|
finalSegments, finalSegments.size(), tmpDir,
|
|
@@ -794,4 +805,27 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
|
|
null);
|
|
null);
|
|
|
|
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ static class CompressAwarePath extends Path {
|
|
|
|
+ private long rawDataLength;
|
|
|
|
+
|
|
|
|
+ public CompressAwarePath(Path path, long rawDataLength) {
|
|
|
|
+ super(path.toUri());
|
|
|
|
+ this.rawDataLength = rawDataLength;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public long getRawDataLength() {
|
|
|
|
+ return rawDataLength;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public boolean equals(Object other) {
|
|
|
|
+ return super.equals(other);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public int hashCode() {
|
|
|
|
+ return super.hashCode();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|