|
@@ -41,22 +41,15 @@ import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.io.BytesWritable;
|
|
import org.apache.hadoop.io.BytesWritable;
|
|
import org.apache.hadoop.io.DataInputBuffer;
|
|
import org.apache.hadoop.io.DataInputBuffer;
|
|
-import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
|
import org.apache.hadoop.io.RawComparator;
|
|
import org.apache.hadoop.io.RawComparator;
|
|
-import org.apache.hadoop.io.SequenceFile;
|
|
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
-import org.apache.hadoop.io.SequenceFile.CompressionType;
|
|
|
|
-import org.apache.hadoop.io.SequenceFile.Sorter;
|
|
|
|
-import org.apache.hadoop.io.SequenceFile.ValueBytes;
|
|
|
|
-import org.apache.hadoop.io.SequenceFile.Writer;
|
|
|
|
-import org.apache.hadoop.io.SequenceFile.Sorter.RawKeyValueIterator;
|
|
|
|
-import org.apache.hadoop.io.SequenceFile.Sorter.SegmentDescriptor;
|
|
|
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
-import org.apache.hadoop.io.compress.CompressionOutputStream;
|
|
|
|
-import org.apache.hadoop.io.compress.Compressor;
|
|
|
|
import org.apache.hadoop.io.compress.DefaultCodec;
|
|
import org.apache.hadoop.io.compress.DefaultCodec;
|
|
import org.apache.hadoop.io.serializer.SerializationFactory;
|
|
import org.apache.hadoop.io.serializer.SerializationFactory;
|
|
import org.apache.hadoop.io.serializer.Serializer;
|
|
import org.apache.hadoop.io.serializer.Serializer;
|
|
|
|
+import org.apache.hadoop.mapred.IFile.Writer;
|
|
|
|
+import org.apache.hadoop.mapred.IFile.Reader;
|
|
|
|
+import org.apache.hadoop.mapred.Merger.Segment;
|
|
import org.apache.hadoop.util.IndexedSortable;
|
|
import org.apache.hadoop.util.IndexedSortable;
|
|
import org.apache.hadoop.util.Progress;
|
|
import org.apache.hadoop.util.Progress;
|
|
import org.apache.hadoop.util.QuickSort;
|
|
import org.apache.hadoop.util.QuickSort;
|
|
@@ -64,6 +57,11 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|
|
|
|
|
/** A Map task. */
|
|
/** A Map task. */
|
|
class MapTask extends Task {
|
|
class MapTask extends Task {
|
|
|
|
+ /**
|
|
|
|
+ * The size of each record in the index file for the map-outputs.
|
|
|
|
+ */
|
|
|
|
+ public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
|
|
|
|
+
|
|
|
|
|
|
private BytesWritable split = new BytesWritable();
|
|
private BytesWritable split = new BytesWritable();
|
|
private String splitClass;
|
|
private String splitClass;
|
|
@@ -281,29 +279,23 @@ class MapTask extends Task {
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- class MapOutputBuffer implements MapOutputCollector, IndexedSortable {
|
|
|
|
|
|
+ class MapOutputBuffer<K extends Object, V extends Object>
|
|
|
|
+ implements MapOutputCollector<K, V>, IndexedSortable {
|
|
private final int partitions;
|
|
private final int partitions;
|
|
- private final Partitioner partitioner;
|
|
|
|
|
|
+ private final Partitioner<K, V> partitioner;
|
|
private final JobConf job;
|
|
private final JobConf job;
|
|
private final Reporter reporter;
|
|
private final Reporter reporter;
|
|
- private final Class keyClass;
|
|
|
|
- private final Class valClass;
|
|
|
|
- private final RawComparator comparator;
|
|
|
|
|
|
+ private final Class<K> keyClass;
|
|
|
|
+ private final Class<V> valClass;
|
|
|
|
+ private final RawComparator<K> comparator;
|
|
private final SerializationFactory serializationFactory;
|
|
private final SerializationFactory serializationFactory;
|
|
- private final Serializer keySerializer;
|
|
|
|
- private final Serializer valSerializer;
|
|
|
|
|
|
+ private final Serializer<K> keySerializer;
|
|
|
|
+ private final Serializer<V> valSerializer;
|
|
private final Class<? extends Reducer> combinerClass;
|
|
private final Class<? extends Reducer> combinerClass;
|
|
- private final CombineOutputCollector combineCollector;
|
|
|
|
- private final boolean compressMapOutput;
|
|
|
|
- private final CompressionCodec codec;
|
|
|
|
- private final CompressionType compressionType;
|
|
|
|
-
|
|
|
|
- // used if compressMapOutput && compressionType == RECORD
|
|
|
|
- // DataOutputBuffer req b/c compression codecs req continguous buffer
|
|
|
|
- private final DataOutputBuffer rawBuffer;
|
|
|
|
- private final CompressionOutputStream deflateFilter;
|
|
|
|
- private final DataOutputStream deflateStream;
|
|
|
|
- private final Compressor compressor;
|
|
|
|
|
|
+ private final CombineOutputCollector<K, V> combineCollector;
|
|
|
|
+
|
|
|
|
+ // Compression for map-outputs
|
|
|
|
+ private CompressionCodec codec = null;
|
|
|
|
|
|
// k/v accounting
|
|
// k/v accounting
|
|
private volatile int kvstart = 0; // marks beginning of spill
|
|
private volatile int kvstart = 0; // marks beginning of spill
|
|
@@ -380,8 +372,8 @@ class MapTask extends Task {
|
|
LOG.info("record buffer = " + softRecordLimit + "/" + kvoffsets.length);
|
|
LOG.info("record buffer = " + softRecordLimit + "/" + kvoffsets.length);
|
|
// k/v serialization
|
|
// k/v serialization
|
|
comparator = job.getOutputKeyComparator();
|
|
comparator = job.getOutputKeyComparator();
|
|
- keyClass = job.getMapOutputKeyClass();
|
|
|
|
- valClass = job.getMapOutputValueClass();
|
|
|
|
|
|
+ keyClass = (Class<K>)job.getMapOutputKeyClass();
|
|
|
|
+ valClass = (Class<V>)job.getMapOutputValueClass();
|
|
serializationFactory = new SerializationFactory(job);
|
|
serializationFactory = new SerializationFactory(job);
|
|
keySerializer = serializationFactory.getSerializer(keyClass);
|
|
keySerializer = serializationFactory.getSerializer(keyClass);
|
|
keySerializer.open(bb);
|
|
keySerializer.open(bb);
|
|
@@ -393,45 +385,23 @@ class MapTask extends Task {
|
|
mapOutputRecordCounter = counters.findCounter(MAP_OUTPUT_RECORDS);
|
|
mapOutputRecordCounter = counters.findCounter(MAP_OUTPUT_RECORDS);
|
|
combineInputCounter = counters.findCounter(COMBINE_INPUT_RECORDS);
|
|
combineInputCounter = counters.findCounter(COMBINE_INPUT_RECORDS);
|
|
combineOutputCounter = counters.findCounter(COMBINE_OUTPUT_RECORDS);
|
|
combineOutputCounter = counters.findCounter(COMBINE_OUTPUT_RECORDS);
|
|
- // combiner and compression
|
|
|
|
- compressMapOutput = job.getCompressMapOutput();
|
|
|
|
- combinerClass = job.getCombinerClass();
|
|
|
|
- minSpillsForCombine = job.getInt("min.num.spills.for.combine", 3);
|
|
|
|
- if (compressMapOutput) {
|
|
|
|
- compressionType = job.getMapOutputCompressionType();
|
|
|
|
|
|
+ // compression
|
|
|
|
+ if (job.getCompressMapOutput()) {
|
|
Class<? extends CompressionCodec> codecClass =
|
|
Class<? extends CompressionCodec> codecClass =
|
|
job.getMapOutputCompressorClass(DefaultCodec.class);
|
|
job.getMapOutputCompressorClass(DefaultCodec.class);
|
|
codec = (CompressionCodec)
|
|
codec = (CompressionCodec)
|
|
ReflectionUtils.newInstance(codecClass, job);
|
|
ReflectionUtils.newInstance(codecClass, job);
|
|
- if (CompressionType.RECORD == compressionType
|
|
|
|
- && null == combinerClass) {
|
|
|
|
- compressor = codec.createCompressor();
|
|
|
|
- rawBuffer = new DataOutputBuffer();
|
|
|
|
- deflateFilter = codec.createOutputStream(rawBuffer, compressor);
|
|
|
|
- deflateStream = new DataOutputStream(deflateFilter);
|
|
|
|
- valSerializer.close();
|
|
|
|
- valSerializer.open(deflateStream);
|
|
|
|
- } else {
|
|
|
|
- rawBuffer = null;
|
|
|
|
- compressor = null;
|
|
|
|
- deflateStream = null;
|
|
|
|
- deflateFilter = null;
|
|
|
|
- }
|
|
|
|
- } else {
|
|
|
|
- compressionType = CompressionType.NONE;
|
|
|
|
- codec = null;
|
|
|
|
- rawBuffer = null;
|
|
|
|
- compressor = null;
|
|
|
|
- deflateStream = null;
|
|
|
|
- deflateFilter = null;
|
|
|
|
}
|
|
}
|
|
|
|
+ // combiner
|
|
|
|
+ combinerClass = job.getCombinerClass();
|
|
combineCollector = (null != combinerClass)
|
|
combineCollector = (null != combinerClass)
|
|
? new CombineOutputCollector(combineOutputCounter)
|
|
? new CombineOutputCollector(combineOutputCounter)
|
|
: null;
|
|
: null;
|
|
|
|
+ minSpillsForCombine = job.getInt("min.num.spills.for.combine", 3);
|
|
}
|
|
}
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
- public synchronized void collect(Object key, Object value)
|
|
|
|
|
|
+ public synchronized void collect(K key, V value)
|
|
throws IOException {
|
|
throws IOException {
|
|
reporter.progress();
|
|
reporter.progress();
|
|
if (key.getClass() != keyClass) {
|
|
if (key.getClass() != keyClass) {
|
|
@@ -449,6 +419,7 @@ class MapTask extends Task {
|
|
).initCause(sortSpillException);
|
|
).initCause(sortSpillException);
|
|
}
|
|
}
|
|
try {
|
|
try {
|
|
|
|
+ // serialize key bytes into buffer
|
|
int keystart = bufindex;
|
|
int keystart = bufindex;
|
|
keySerializer.serialize(key);
|
|
keySerializer.serialize(key);
|
|
if (bufindex < keystart) {
|
|
if (bufindex < keystart) {
|
|
@@ -456,27 +427,14 @@ class MapTask extends Task {
|
|
bb.reset();
|
|
bb.reset();
|
|
keystart = 0;
|
|
keystart = 0;
|
|
}
|
|
}
|
|
|
|
+ // serialize value bytes into buffer
|
|
int valstart = bufindex;
|
|
int valstart = bufindex;
|
|
- if (compressMapOutput && CompressionType.RECORD == compressionType
|
|
|
|
- && null == combinerClass) {
|
|
|
|
- // compress serialized value bytes
|
|
|
|
- rawBuffer.reset();
|
|
|
|
- deflateFilter.resetState();
|
|
|
|
- valSerializer.serialize(value);
|
|
|
|
- deflateStream.flush();
|
|
|
|
- deflateFilter.finish();
|
|
|
|
- bb.write(rawBuffer.getData(), 0, rawBuffer.getLength());
|
|
|
|
- bb.markRecord();
|
|
|
|
- mapOutputByteCounter.increment((valstart - keystart) +
|
|
|
|
- compressor.getBytesRead());
|
|
|
|
- } else {
|
|
|
|
- // serialize value bytes into buffer
|
|
|
|
- valSerializer.serialize(value);
|
|
|
|
- int valend = bb.markRecord();
|
|
|
|
- mapOutputByteCounter.increment(valend > keystart
|
|
|
|
- ? valend - keystart
|
|
|
|
- : (bufvoid - keystart) + valend);
|
|
|
|
- }
|
|
|
|
|
|
+ valSerializer.serialize(value);
|
|
|
|
+ int valend = bb.markRecord();
|
|
|
|
+ mapOutputByteCounter.increment(valend > keystart
|
|
|
|
+ ? valend - keystart
|
|
|
|
+ : (bufvoid - keystart) + valend);
|
|
|
|
+
|
|
int partition = partitioner.getPartition(key, value, partitions);
|
|
int partition = partitioner.getPartition(key, value, partitions);
|
|
if (partition < 0 || partition >= partitions) {
|
|
if (partition < 0 || partition >= partitions) {
|
|
throw new IOException("Illegal partition for " + key + " (" +
|
|
throw new IOException("Illegal partition for " + key + " (" +
|
|
@@ -766,31 +724,34 @@ class MapTask extends Task {
|
|
out = localFs.create(filename);
|
|
out = localFs.create(filename);
|
|
// create spill index
|
|
// create spill index
|
|
Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
|
|
Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
|
|
- getTaskID(), numSpills, partitions * 16);
|
|
|
|
|
|
+ getTaskID(), numSpills, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH);
|
|
indexOut = localFs.create(indexFilename);
|
|
indexOut = localFs.create(indexFilename);
|
|
final int endPosition = (kvend > kvstart)
|
|
final int endPosition = (kvend > kvstart)
|
|
? kvend
|
|
? kvend
|
|
: kvoffsets.length + kvend;
|
|
: kvoffsets.length + kvend;
|
|
sorter.sort(MapOutputBuffer.this, kvstart, endPosition, reporter);
|
|
sorter.sort(MapOutputBuffer.this, kvstart, endPosition, reporter);
|
|
int spindex = kvstart;
|
|
int spindex = kvstart;
|
|
- InMemValBytes vbytes = new InMemValBytes();
|
|
|
|
|
|
+ InMemValBytes value = new InMemValBytes();
|
|
for (int i = 0; i < partitions; ++i) {
|
|
for (int i = 0; i < partitions; ++i) {
|
|
- SequenceFile.Writer writer = null;
|
|
|
|
|
|
+ IFile.Writer<K, V> writer = null;
|
|
try {
|
|
try {
|
|
long segmentStart = out.getPos();
|
|
long segmentStart = out.getPos();
|
|
- writer = SequenceFile.createWriter(job, out,
|
|
|
|
- keyClass, valClass, compressionType, codec);
|
|
|
|
|
|
+ writer = new Writer<K, V>(job, out, keyClass, valClass, codec);
|
|
if (null == combinerClass) {
|
|
if (null == combinerClass) {
|
|
// spill directly
|
|
// spill directly
|
|
|
|
+ DataInputBuffer key = new DataInputBuffer();
|
|
|
|
+ long recordNo = 0;
|
|
while (spindex < endPosition &&
|
|
while (spindex < endPosition &&
|
|
kvindices[kvoffsets[spindex % kvoffsets.length]
|
|
kvindices[kvoffsets[spindex % kvoffsets.length]
|
|
+ PARTITION] == i) {
|
|
+ PARTITION] == i) {
|
|
final int kvoff = kvoffsets[spindex % kvoffsets.length];
|
|
final int kvoff = kvoffsets[spindex % kvoffsets.length];
|
|
- getVBytesForOffset(kvoff, vbytes);
|
|
|
|
- writer.appendRaw(kvbuffer, kvindices[kvoff + KEYSTART],
|
|
|
|
- kvindices[kvoff + VALSTART] - kvindices[kvoff + KEYSTART],
|
|
|
|
- vbytes);
|
|
|
|
|
|
+ getVBytesForOffset(kvoff, value);
|
|
|
|
+ key.reset(kvbuffer, kvindices[kvoff + KEYSTART],
|
|
|
|
+ (kvindices[kvoff + VALSTART] -
|
|
|
|
+ kvindices[kvoff + KEYSTART]));
|
|
|
|
+ writer.append(key, value);
|
|
++spindex;
|
|
++spindex;
|
|
|
|
+ ++recordNo;
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
int spstart = spindex;
|
|
int spstart = spindex;
|
|
@@ -812,12 +773,13 @@ class MapTask extends Task {
|
|
combineAndSpill(kvIter, combineInputCounter);
|
|
combineAndSpill(kvIter, combineInputCounter);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- // we need to close the writer to flush buffered data, obtaining
|
|
|
|
- // the correct offset
|
|
|
|
|
|
+
|
|
|
|
+ // close the writer
|
|
writer.close();
|
|
writer.close();
|
|
|
|
+
|
|
|
|
+ // write the index as <offset, raw-length, compressed-length>
|
|
|
|
+ writeIndexRecord(indexOut, out, segmentStart, writer);
|
|
writer = null;
|
|
writer = null;
|
|
- indexOut.writeLong(segmentStart);
|
|
|
|
- indexOut.writeLong(out.getPos() - segmentStart);
|
|
|
|
} finally {
|
|
} finally {
|
|
if (null != writer) writer.close();
|
|
if (null != writer) writer.close();
|
|
}
|
|
}
|
|
@@ -836,7 +798,7 @@ class MapTask extends Task {
|
|
* directly to a spill file. Consider this "losing".
|
|
* directly to a spill file. Consider this "losing".
|
|
*/
|
|
*/
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
- private void spillSingleRecord(final Object key, final Object value)
|
|
|
|
|
|
+ private void spillSingleRecord(final K key, final V value)
|
|
throws IOException {
|
|
throws IOException {
|
|
long size = kvbuffer.length + partitions * APPROX_HEADER_LENGTH;
|
|
long size = kvbuffer.length + partitions * APPROX_HEADER_LENGTH;
|
|
FSDataOutputStream out = null;
|
|
FSDataOutputStream out = null;
|
|
@@ -849,15 +811,16 @@ class MapTask extends Task {
|
|
out = localFs.create(filename);
|
|
out = localFs.create(filename);
|
|
// create spill index
|
|
// create spill index
|
|
Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
|
|
Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
|
|
- getTaskID(), numSpills, partitions * 16);
|
|
|
|
|
|
+ getTaskID(), numSpills, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH);
|
|
indexOut = localFs.create(indexFilename);
|
|
indexOut = localFs.create(indexFilename);
|
|
// we don't run the combiner for a single record
|
|
// we don't run the combiner for a single record
|
|
for (int i = 0; i < partitions; ++i) {
|
|
for (int i = 0; i < partitions; ++i) {
|
|
- SequenceFile.Writer writer = null;
|
|
|
|
|
|
+ IFile.Writer writer = null;
|
|
try {
|
|
try {
|
|
long segmentStart = out.getPos();
|
|
long segmentStart = out.getPos();
|
|
- writer = SequenceFile.createWriter(job, out,
|
|
|
|
- keyClass, valClass, compressionType, codec);
|
|
|
|
|
|
+ // Create a new codec, don't care!
|
|
|
|
+ writer = new IFile.Writer(job, out, keyClass, valClass, codec);
|
|
|
|
+
|
|
if (i == partition) {
|
|
if (i == partition) {
|
|
final long recordStart = out.getPos();
|
|
final long recordStart = out.getPos();
|
|
writer.append(key, value);
|
|
writer.append(key, value);
|
|
@@ -866,8 +829,9 @@ class MapTask extends Task {
|
|
mapOutputByteCounter.increment(out.getPos() - recordStart);
|
|
mapOutputByteCounter.increment(out.getPos() - recordStart);
|
|
}
|
|
}
|
|
writer.close();
|
|
writer.close();
|
|
- indexOut.writeLong(segmentStart);
|
|
|
|
- indexOut.writeLong(out.getPos() - segmentStart);
|
|
|
|
|
|
+
|
|
|
|
+ // index record
|
|
|
|
+ writeIndexRecord(indexOut, out, segmentStart, writer);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
if (null != writer) writer.close();
|
|
if (null != writer) writer.close();
|
|
throw e;
|
|
throw e;
|
|
@@ -891,7 +855,7 @@ class MapTask extends Task {
|
|
int vallen = (nextindex > kvindices[kvoff + VALSTART])
|
|
int vallen = (nextindex > kvindices[kvoff + VALSTART])
|
|
? nextindex - kvindices[kvoff + VALSTART]
|
|
? nextindex - kvindices[kvoff + VALSTART]
|
|
: (bufvoid - kvindices[kvoff + VALSTART]) + nextindex;
|
|
: (bufvoid - kvindices[kvoff + VALSTART]) + nextindex;
|
|
- vbytes.reset(kvindices[kvoff + VALSTART], vallen);
|
|
|
|
|
|
+ vbytes.reset(kvbuffer, kvindices[kvoff + VALSTART], vallen);
|
|
}
|
|
}
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
@@ -917,36 +881,30 @@ class MapTask extends Task {
|
|
/**
|
|
/**
|
|
* Inner class wrapping valuebytes, used for appendRaw.
|
|
* Inner class wrapping valuebytes, used for appendRaw.
|
|
*/
|
|
*/
|
|
- protected class InMemValBytes implements ValueBytes {
|
|
|
|
|
|
+ protected class InMemValBytes extends DataInputBuffer {
|
|
|
|
+ private byte[] buffer;
|
|
private int start;
|
|
private int start;
|
|
- private int len;
|
|
|
|
- public void reset(int start, int len) {
|
|
|
|
|
|
+ private int length;
|
|
|
|
+
|
|
|
|
+ public void reset(byte[] buffer, int start, int length) {
|
|
|
|
+ this.buffer = buffer;
|
|
this.start = start;
|
|
this.start = start;
|
|
- this.len = len;
|
|
|
|
- }
|
|
|
|
- public int getSize() {
|
|
|
|
- return len;
|
|
|
|
- }
|
|
|
|
- public void writeUncompressedBytes(DataOutputStream outStream)
|
|
|
|
- throws IOException {
|
|
|
|
- if (start + len > bufvoid) {
|
|
|
|
|
|
+ this.length = length;
|
|
|
|
+
|
|
|
|
+ if (start + length > bufvoid) {
|
|
|
|
+ this.buffer = new byte[this.length];
|
|
final int taillen = bufvoid - start;
|
|
final int taillen = bufvoid - start;
|
|
- outStream.write(kvbuffer, start, taillen);
|
|
|
|
- outStream.write(kvbuffer, 0, len - taillen);
|
|
|
|
- return;
|
|
|
|
|
|
+ System.arraycopy(buffer, start, this.buffer, 0, taillen);
|
|
|
|
+ System.arraycopy(buffer, 0, this.buffer, taillen, length-taillen);
|
|
|
|
+ this.start = 0;
|
|
}
|
|
}
|
|
- outStream.write(kvbuffer, start, len);
|
|
|
|
- }
|
|
|
|
- public void writeCompressedBytes(DataOutputStream outStream)
|
|
|
|
- throws IOException {
|
|
|
|
- // If writing record-compressed data, kvbuffer vals rec-compressed
|
|
|
|
- // and may be written directly. Note: not contiguous
|
|
|
|
- writeUncompressedBytes(outStream);
|
|
|
|
|
|
+
|
|
|
|
+ super.reset(this.buffer, this.start, this.length);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
protected class MRResultIterator implements RawKeyValueIterator {
|
|
protected class MRResultIterator implements RawKeyValueIterator {
|
|
- private final DataOutputBuffer keybuf = new DataOutputBuffer();
|
|
|
|
|
|
+ private final DataInputBuffer keybuf = new DataInputBuffer();
|
|
private final InMemValBytes vbytes = new InMemValBytes();
|
|
private final InMemValBytes vbytes = new InMemValBytes();
|
|
private final int end;
|
|
private final int end;
|
|
private int current;
|
|
private int current;
|
|
@@ -957,14 +915,13 @@ class MapTask extends Task {
|
|
public boolean next() throws IOException {
|
|
public boolean next() throws IOException {
|
|
return ++current < end;
|
|
return ++current < end;
|
|
}
|
|
}
|
|
- public DataOutputBuffer getKey() throws IOException {
|
|
|
|
|
|
+ public DataInputBuffer getKey() throws IOException {
|
|
final int kvoff = kvoffsets[current % kvoffsets.length];
|
|
final int kvoff = kvoffsets[current % kvoffsets.length];
|
|
- keybuf.reset();
|
|
|
|
- keybuf.write(kvbuffer, kvindices[kvoff + KEYSTART],
|
|
|
|
- kvindices[kvoff + VALSTART] - kvindices[kvoff + KEYSTART]);
|
|
|
|
|
|
+ keybuf.reset(kvbuffer, kvindices[kvoff + KEYSTART],
|
|
|
|
+ kvindices[kvoff + VALSTART] - kvindices[kvoff + KEYSTART]);
|
|
return keybuf;
|
|
return keybuf;
|
|
}
|
|
}
|
|
- public ValueBytes getValue() throws IOException {
|
|
|
|
|
|
+ public DataInputBuffer getValue() throws IOException {
|
|
getVBytesForOffset(kvoffsets[current % kvoffsets.length], vbytes);
|
|
getVBytesForOffset(kvoffsets[current % kvoffsets.length], vbytes);
|
|
return vbytes;
|
|
return vbytes;
|
|
}
|
|
}
|
|
@@ -999,7 +956,7 @@ class MapTask extends Task {
|
|
//lengths for each partition
|
|
//lengths for each partition
|
|
finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
|
|
finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
|
|
|
|
|
|
- finalIndexFileSize = partitions * 16;
|
|
|
|
|
|
+ finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
|
|
|
|
|
|
Path finalOutputFile = mapOutputFile.getOutputFileForWrite(getTaskID(),
|
|
Path finalOutputFile = mapOutputFile.getOutputFileForWrite(getTaskID(),
|
|
finalOutFileSize);
|
|
finalOutFileSize);
|
|
@@ -1009,6 +966,7 @@ class MapTask extends Task {
|
|
//The output stream for the final single output file
|
|
//The output stream for the final single output file
|
|
FSDataOutputStream finalOut = localFs.create(finalOutputFile, true,
|
|
FSDataOutputStream finalOut = localFs.create(finalOutputFile, true,
|
|
4096);
|
|
4096);
|
|
|
|
+
|
|
//The final index file output stream
|
|
//The final index file output stream
|
|
FSDataOutputStream finalIndexOut = localFs.create(finalIndexFile, true,
|
|
FSDataOutputStream finalIndexOut = localFs.create(finalIndexFile, true,
|
|
4096);
|
|
4096);
|
|
@@ -1018,12 +976,11 @@ class MapTask extends Task {
|
|
//create dummy files
|
|
//create dummy files
|
|
for (int i = 0; i < partitions; i++) {
|
|
for (int i = 0; i < partitions; i++) {
|
|
segmentStart = finalOut.getPos();
|
|
segmentStart = finalOut.getPos();
|
|
- Writer writer = SequenceFile.createWriter(job, finalOut,
|
|
|
|
- job.getMapOutputKeyClass(),
|
|
|
|
- job.getMapOutputValueClass(),
|
|
|
|
- compressionType, codec);
|
|
|
|
|
|
+ Writer<K, V> writer = new Writer<K, V>(job, finalOut,
|
|
|
|
+ keyClass, valClass, null);
|
|
finalIndexOut.writeLong(segmentStart);
|
|
finalIndexOut.writeLong(segmentStart);
|
|
finalIndexOut.writeLong(finalOut.getPos() - segmentStart);
|
|
finalIndexOut.writeLong(finalOut.getPos() - segmentStart);
|
|
|
|
+ finalIndexOut.writeLong(finalOut.getPos() - segmentStart);
|
|
writer.close();
|
|
writer.close();
|
|
}
|
|
}
|
|
finalOut.close();
|
|
finalOut.close();
|
|
@@ -1031,48 +988,55 @@ class MapTask extends Task {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
{
|
|
{
|
|
- //create a sorter object as we need access to the SegmentDescriptor
|
|
|
|
- //class and merge methods
|
|
|
|
- Sorter sorter = new Sorter(localFs, job.getOutputKeyComparator(),
|
|
|
|
- keyClass, valClass, job);
|
|
|
|
- sorter.setProgressable(reporter);
|
|
|
|
-
|
|
|
|
for (int parts = 0; parts < partitions; parts++){
|
|
for (int parts = 0; parts < partitions; parts++){
|
|
- List<SegmentDescriptor> segmentList =
|
|
|
|
- new ArrayList<SegmentDescriptor>(numSpills);
|
|
|
|
|
|
+ //create the segments to be merged
|
|
|
|
+ List<Segment<K, V>> segmentList =
|
|
|
|
+ new ArrayList<Segment<K, V>>(numSpills);
|
|
for(int i = 0; i < numSpills; i++) {
|
|
for(int i = 0; i < numSpills; i++) {
|
|
FSDataInputStream indexIn = localFs.open(indexFileName[i]);
|
|
FSDataInputStream indexIn = localFs.open(indexFileName[i]);
|
|
- indexIn.seek(parts * 16);
|
|
|
|
|
|
+ indexIn.seek(parts * MAP_OUTPUT_INDEX_RECORD_LENGTH);
|
|
long segmentOffset = indexIn.readLong();
|
|
long segmentOffset = indexIn.readLong();
|
|
|
|
+ long rawSegmentLength = indexIn.readLong();
|
|
long segmentLength = indexIn.readLong();
|
|
long segmentLength = indexIn.readLong();
|
|
indexIn.close();
|
|
indexIn.close();
|
|
- SegmentDescriptor s = sorter.new SegmentDescriptor(segmentOffset,
|
|
|
|
- segmentLength, filename[i]);
|
|
|
|
- s.preserveInput(true);
|
|
|
|
- s.doSync();
|
|
|
|
|
|
+ FSDataInputStream in = localFs.open(filename[i]);
|
|
|
|
+ in.seek(segmentOffset);
|
|
|
|
+ Segment<K, V> s =
|
|
|
|
+ new Segment<K, V>(new Reader<K, V>(job, in, segmentLength, codec),
|
|
|
|
+ true);
|
|
segmentList.add(i, s);
|
|
segmentList.add(i, s);
|
|
|
|
+
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Index: (" + indexFileName[i] + ", " + segmentOffset +
|
|
|
|
+ rawSegmentLength + ", " + segmentLength + ")");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ //merge
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
|
+ RawKeyValueIterator kvIter =
|
|
|
|
+ Merger.merge(job, localFs,
|
|
|
|
+ keyClass, valClass,
|
|
|
|
+ segmentList, job.getInt("io.sort.factor", 100),
|
|
|
|
+ new Path(getTaskID().toString()),
|
|
|
|
+ job.getOutputKeyComparator(), reporter);
|
|
|
|
+
|
|
|
|
+ //write merged output to disk
|
|
segmentStart = finalOut.getPos();
|
|
segmentStart = finalOut.getPos();
|
|
- RawKeyValueIterator kvIter = sorter.merge(segmentList, new Path(getTaskID().toString()));
|
|
|
|
- SequenceFile.Writer writer = SequenceFile.createWriter(job, finalOut,
|
|
|
|
- job.getMapOutputKeyClass(), job.getMapOutputValueClass(),
|
|
|
|
- compressionType, codec);
|
|
|
|
|
|
+ Writer<K, V> writer =
|
|
|
|
+ new Writer<K, V>(job, finalOut, keyClass, valClass, codec);
|
|
if (null == combinerClass || numSpills < minSpillsForCombine) {
|
|
if (null == combinerClass || numSpills < minSpillsForCombine) {
|
|
- sorter.writeFile(kvIter, writer);
|
|
|
|
|
|
+ Merger.writeFile(kvIter, writer, reporter);
|
|
} else {
|
|
} else {
|
|
combineCollector.setWriter(writer);
|
|
combineCollector.setWriter(writer);
|
|
combineAndSpill(kvIter, combineInputCounter);
|
|
combineAndSpill(kvIter, combineInputCounter);
|
|
}
|
|
}
|
|
- //close the file - required esp. for block compression to ensure
|
|
|
|
- //partition data don't span partition boundaries
|
|
|
|
|
|
+
|
|
|
|
+ //close
|
|
writer.close();
|
|
writer.close();
|
|
- //when we write the offset/length to the final index file, we write
|
|
|
|
- //longs for both. This helps us to reliably seek directly to the
|
|
|
|
- //offset/length for a partition when we start serving the byte-ranges
|
|
|
|
- //to the reduces. We probably waste some space in the file by doing
|
|
|
|
- //this as opposed to writing VLong but it helps us later on.
|
|
|
|
- finalIndexOut.writeLong(segmentStart);
|
|
|
|
- finalIndexOut.writeLong(finalOut.getPos()-segmentStart);
|
|
|
|
|
|
+
|
|
|
|
+ //write index record
|
|
|
|
+ writeIndexRecord(finalIndexOut, finalOut, segmentStart, writer);
|
|
}
|
|
}
|
|
finalOut.close();
|
|
finalOut.close();
|
|
finalIndexOut.close();
|
|
finalIndexOut.close();
|
|
@@ -1084,8 +1048,26 @@ class MapTask extends Task {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ private void writeIndexRecord(FSDataOutputStream indexOut,
|
|
|
|
+ FSDataOutputStream out, long start,
|
|
|
|
+ Writer<K, V> writer)
|
|
|
|
+ throws IOException {
|
|
|
|
+ //when we write the offset/decompressed-length/compressed-length to
|
|
|
|
+ //the final index file, we write longs for both compressed and
|
|
|
|
+ //decompressed lengths. This helps us to reliably seek directly to
|
|
|
|
+ //the offset/length for a partition when we start serving the
|
|
|
|
+ //byte-ranges to the reduces. We probably waste some space in the
|
|
|
|
+ //file by doing this as opposed to writing VLong but it helps us later on.
|
|
|
|
+ // index record: <offset, raw-length, compressed-length>
|
|
|
|
+ //StringBuffer sb = new StringBuffer();
|
|
|
|
+ indexOut.writeLong(start);
|
|
|
|
+ indexOut.writeLong(writer.getRawLength());
|
|
|
|
+ long segmentLength = out.getPos() - start;
|
|
|
|
+ indexOut.writeLong(segmentLength);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ } // MapOutputBuffer
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Exception indicating that the allocated sort buffer is insufficient
|
|
* Exception indicating that the allocated sort buffer is insufficient
|
|
* to hold the current record.
|
|
* to hold the current record.
|