核心类
核心算法
参数
- 参数:mapred.reduce.parallel.copies(default 5) 说明:每个reduce并行下载map结果的最大线程数
Reduce task在做shuffle时,实际上就是从不同的已经完成的map上去下载属于自己这个reduce的部分数据, 由于map通常有许多个,所以对一个reduce来说,下载也可以是并行的从多个map下载,这个并行度是可以调整的, 调整参数为:mapred.reduce.parallel.copies(default 5)。
默认情况下,每个只会有5个并行的下载线程在从map下数据,如果一个时间段内job完成的map有100个或者更多, 那么reduce也最多只能同时下载5个map的数据
所以这个参数比较适合map很多并且完成的比较快的job的情况下调大,有利于reduce更快的获取属于自己部分的数据。
- 参数:mapred.reduce.copy.backoff(default 300秒)
说明:reduce下载线程最大等待时间(秒)
reduce的每一个下载线程在下载某个map数据的时候,有可能因为那个map中间结果所在机器发生错误, 或者中间结果的文件丢失,或者网络瞬断等等情况,这样reduce的下载就有可能失败,
所以reduce的下载线程并不会无休止的等待下去,当一定时间后下载仍然失败,那么下载线程就会放弃这次下载, 并在随后尝试从另外的地方下载(因为这段时间map可能重跑)。
所以reduce下载线程的这个最大的下载时间段是可以调整的,
调整参数为:mapred.reduce.copy.backoff(default 300秒)。
如果集群环境的网络本身是瓶颈,那么用户可以通过调大这个参数来避免reduce下载线程被误判为失败的情况。 不过在网络环境比较好的情况下,没有必要调整。通常来说专业的集群网络不应该有太大问题,所以这个参数需要调整的情况不多。
- 参数:io.sort.factor
Reduce将map结果下载到本地时,同样也是需要进行merge的,所以io.sort.factor的配置选项同样会影响reduce进行merge时的行为,该参数的详细介绍上文已经提到,当发现reduce在shuffle阶段iowait非常的高的时候,就有可能通过调大这个参数来加大一次merge时的并发吞吐,优化reduce效率。
- 参数:mapred.job.shuffle.input.buffer.percent(default 0.7) 说明:用来缓存shuffle数据的reduce task heap百分比
Reduce在shuffle阶段对下载来的map数据,并不是立刻就写入磁盘的,而是会先缓存在内存中,然后当使用内存达到一定量的时候才刷入磁盘。
这个内存大小的控制就不像map一样可以通过io.sort.mb来设定了,而是通过另外一个参数来设置: mapred.job.shuffle.input.buffer.percent(default 0.7), 这个参数其实是一个百分比,意思是说,shuffile在reduce内存中的数据最多使用内存量为:0.7 × maxHeap of reduce task。
也就是说,如果该reduce task的最大heap使用量(通常通过mapred.child.java.opts来设置,比如设置为-Xmx1024m)的一定比例用来缓存数据。
默认情况下,reduce会使用其heapsize的70%来在内存中缓存数据。 如果reduce的heap由于业务原因调整的比较大,相应的缓存大小也会变大,这也是为什么reduce用来做缓存的参数是一个百分比,而不是一个固定的值了。
- 参数:mapred.job.shuffle.merge.percent(default 0.66) 说明:缓存的内存中多少百分比后开始做merge操作
假设mapred.job.shuffle.input.buffer.percent为0.7,reduce task的max heapsize为1G, 那么用来做下载数据缓存的内存就为大概700MB左右,这700M的内存,跟map端一样, 也不是要等到全部写满才会往磁盘刷的,而是当这700M中被使用到了一定的限度(通常是一个百分比),就会开始往磁盘刷。
这个限度阈值也是可以通过job参数来设定的,设定参数为:mapred.job.shuffle.merge.percent(default 0.66)。
如果下载速度很快,很容易就把内存缓存撑大,那么调整一下这个参数有可能会对reduce的性能有所帮助。
- 参数:mapred.job.reduce.input.buffer.percent(default 0.0) 说明:sort完成后reduce计算阶段用来缓解数据的百分比
当reduce将所有的map上对应自己partition的数据下载完成后,就会开始真正的reduce计算阶段 (中间有个sort阶段通常时间非常短,几秒钟就完成了,因为整个下载阶段就已经是边下载边sort,然后边merge的)。
当reduce task真正进入reduce函数的计算阶段的时候,有一个参数也是可以调整reduce的计算行为。 也就是:mapred.job.reduce.input.buffer.percent(default 0.0)。
由于reduce计算时肯定也是需要消耗内存的,而在读取reduce需要的数据时,同样是需要内存作为buffer, 这个参数是控制,需要多少的内存百分比来作为reduce读已经sort好的数据的buffer百分比。 默认情况下为0,也就是说,默认情况下,reduce是全部从磁盘开始读处理数据。
如果这个参数大于0,那么就会有一定量的数据被缓存在内存并输送给reduce, 当reduce计算逻辑消耗内存很小时,可以分一部分内存用来缓存数据,反正reduce的内存闲着也是闲着。
fetcher
EventFetcher主要是通过getMapCompletionEvents方法的rpc拿到Completion Map,然后放入scheduler的pendingHosts。
//EventFetcher.run()
public void run() {
int failures = 0;
LOG.info(reduce + " Thread started: " + getName());
try {
while (!stopped && !Thread.currentThread().isInterrupted()) {
try {
int numNewMaps = getMapCompletionEvents();
failures = 0;
if (numNewMaps > 0) {
LOG.info(reduce + ": " + "Got " + numNewMaps + " new map-outputs");
}
LOG.debug("GetMapEventsThread about to sleep for " + SLEEP_TIME);
if (!Thread.currentThread().isInterrupted()) {
Thread.sleep(SLEEP_TIME);
}
} catch (InterruptedException e)
...
}
} catch (InterruptedException e)
...
}
protected int getMapCompletionEvents()
throws IOException, InterruptedException {
int numNewMaps = 0;
TaskCompletionEvent events[] = null;
do {
// rpc 调用
MapTaskCompletionEventsUpdate update =
umbilical.getMapCompletionEvents(
(org.apache.hadoop.mapred.JobID)reduce.getJobID(),
fromEventIdx,
maxEventsToFetch,
(org.apache.hadoop.mapred.TaskAttemptID)reduce);
events = update.getMapTaskCompletionEvents();
LOG.debug("Got " + events.length + " map completion events from " +
fromEventIdx);
assert !update.shouldReset() : "Unexpected legacy state";
// Update the last seen event ID
fromEventIdx += events.length;
// Process the TaskCompletionEvents:
// 1. Save the SUCCEEDED maps in knownOutputs to fetch the outputs.
// 2. Save the OBSOLETE/FAILED/KILLED maps in obsoleteOutputs to stop
// fetching from those maps.
// 3. Remove TIPFAILED maps from neededOutputs since we don't need their
// outputs at all.
for (TaskCompletionEvent event : events) {
scheduler.resolve(event);
if (TaskCompletionEvent.Status.SUCCEEDED == event.getTaskStatus()) {
++numNewMaps;
}
}
} while (events.length == maxEventsToFetch);
return numNewMaps;
}
@Override
public void resolve(TaskCompletionEvent event) {
switch (event.getTaskStatus()) {
case SUCCEEDED:
URI u = getBaseURI(reduceId, event.getTaskTrackerHttp());
// 将得到的events放入scheduler对应的pendingHosts中
addKnownMapOutput(u.getHost() + ":" + u.getPort(),
u.toString(),
event.getTaskAttemptId());
maxMapRuntime = Math.max(maxMapRuntime, event.getTaskRunTime());
break;
case FAILED:
case KILLED:
case OBSOLETE:
obsoleteMapOutput(event.getTaskAttemptId());
LOG.info("Ignoring obsolete output of " + event.getTaskStatus() +
" map-task: '" + event.getTaskAttemptId() + "'");
break;
case TIPFAILED:
tipFailed(event.getTaskAttemptId().getTaskID());
LOG.info("Ignoring output of failed map TIP: '" +
event.getTaskAttemptId() + "'");
break;
}
}
在Fetcher线程的run方法中先判断是否有merge,如果有则block当前Fetcher线程,这里只判断是否存在内存到磁盘的merge。随后从Host列表中_随机_选出一个Host(随机方法可查看scheduler.getHost()代码),进行copy,copy的入口是copyFromHost,copyFromHost的主要工作是建立http连接,然后循环调用copyMapOutput对某个map的输出进行copy。在copyMapOutput中会判断当前的内容是输出到mem还是disk,此处的逻辑判断在merger.reserve()中
public void run() {
try {
while (!stopped && !Thread.currentThread().isInterrupted()) {
MapHost host = null;
try {
// If merge is on, block
// 存在内存到磁盘的merge,则Fetcher被block
// 不像mapper端的环形缓冲区一样,可以一边向磁盘写,一边继续向内存写数据
// 这里当存在MemToDisk的merge时,fetch操作被阻塞
// memToDisk触发的条件是内存中的占比超过90%
merger.waitForResource();
// Get a host to shuffle from
// 从上文eventFetcher中获得的pendingHosts中随机取出一个
host = scheduler.getHost();
metrics.threadBusy();
// Shuffle
copyFromHost(host);
}
...
}
}
...
}
protected void copyFromHost(MapHost host) throws IOException {
...
// Get completed maps on 'host'
// 得到map list,最多拿到MAX_MAPS_AT_ONCE(默认是20)
List<TaskAttemptID> maps = scheduler.getMapsForHost(host);
// Sanity check to catch hosts with only 'OBSOLETE' maps,
// especially at the tail of large jobs
if (maps.size() == 0) {
return;
}
...
// List of maps to be fetched yet
Set<TaskAttemptID> remaining = new HashSet<TaskAttemptID>(maps);
// Construct the url and connect
DataInputStream input = null;
URL url = getMapOutputURL(host, maps);//eg: http://hadoop3:13562/mapOutput?job=job_1613991969743_0002&reduce=0&map=attempt_1613991969743_0002_m_000000_0
try {
// 通过jetty服务器创建http连接
setupConnectionsWithRetry(host, remaining, url);
if (stopped) {
abortConnect(host, remaining);
return;
}
} catch (IOException ie) {
...
return;
}
// 根据http创建一个input流
input = new DataInputStream(connection.getInputStream());
try {
// Loop through available map-outputs and fetch them
// On any error, faildTasks is not null and we exit
// after putting back the remaining maps to the
// yet_to_be_fetched list and marking the failed tasks.
TaskAttemptID[] failedTasks = null;
while (!remaining.isEmpty() && failedTasks == null) {
try {
// 复制map
failedTasks = copyMapOutput(host, input, remaining, fetchRetryEnabled);
} catch (IOException e) {
//
// Setup connection again if disconnected by NM
connection.disconnect();
// Get map output from remaining tasks only.
url = getMapOutputURL(host, remaining);
// Connect with retry as expecting host's recovery take sometime.
setupConnectionsWithRetry(host, remaining, url);
if (stopped) {
abortConnect(host, remaining);
return;
}
input = new DataInputStream(connection.getInputStream());
}
}
...
} finally {
...
}
}
private TaskAttemptID[] copyMapOutput(MapHost host,
DataInputStream input,
Set<TaskAttemptID> remaining,
boolean canRetry) throws IOException {
...
try {
long startTime = Time.monotonicNow();
int forReduce = -1;
//Read the shuffle header
try {
ShuffleHeader header = new ShuffleHeader();
// 读取内容到input流中
header.readFields(input);
mapId = TaskAttemptID.forName(header.mapId);
compressedLength = header.compressedLength;
decompressedLength = header.uncompressedLength;
forReduce = header.forReduce;
} catch (IllegalArgumentException e) {
...
}
InputStream is = input;
is = CryptoUtils.wrapIfNecessary(jobConf, is, compressedLength);
compressedLength -= CryptoUtils.cryptoPadding(jobConf);
decompressedLength -= CryptoUtils.cryptoPadding(jobConf);
...
// Get the location for the map output - either in-memory or on-disk
try {
// 根据内容大小申请资源,此处决定写入的目的地 mem or disk
mapOutput = merger.reserve(mapId, decompressedLength, id);
} catch (IOException ioe) {
...
}
// Check if we can shuffle *now* ...
// 如果当前内存使用超过memoryLimit,则merger.reserve 返回null
// 此时不能shuffle
if (mapOutput == null) {
LOG.info("fetcher#" + id + " - MergeManager returned status WAIT ...");
//Not an error but wait to process data.
return EMPTY_ATTEMPT_ID_ARRAY;
}
// The codec for lz0,lz4,snappy,bz2,etc. throw java.lang.InternalError
// on decompression failures. Catching and re-throwing as IOException
// to allow fetch failure logic to be processed
try {
// Go!
LOG.info("fetcher#" + id + " about to shuffle output of map "
+ mapOutput.getMapId() + " decomp: " + decompressedLength
+ " len: " + compressedLength + " to " + mapOutput.getDescription());
// 将数据写入mem 或者 disk
mapOutput.shuffle(host, is, compressedLength, decompressedLength,
metrics, reporter);
} catch (java.lang.InternalError e) {
LOG.warn("Failed to shuffle for fetcher#"+id, e);
throw new IOException(e);
}
// Inform the shuffle scheduler
long endTime = Time.monotonicNow();
// Reset retryStartTime as map task make progress if retried before.
retryStartTime = 0;
//
scheduler.copySucceeded(mapId, host, compressedLength,
startTime, endTime, mapOutput);
// Note successful shuffle
remaining.remove(mapId);
metrics.successFetch();
return null;
} catch (IOException ioe) {
...
}
}
merge
- 通过createMergeManagernew出一个MergeManagerImpl对象,该对象的构造函数主要构建几个Merge线程,并为一些参数设置阈值。
// Shuffle.java
public void init(ShuffleConsumerPlugin.Context context) {
...
// 主要用来存放map等相关信息,也包含fetcher相关的调度
scheduler = new ShuffleSchedulerImpl<K, V>(jobConf, taskStatus, reduceId,
this, copyPhase, context.getShuffledMapsCounter(),
context.getReduceShuffleBytes(), context.getFailedShuffleCounter());
// new 出一个MergeManagerImpl对象,在该对象中创建
// merge线程,包含inMemoryMerger(内存merge到磁盘)、onDiskMerger(磁盘merge)
merger = createMergeManager(context);
}
public MergeManagerImpl(TaskAttemptID reduceId, JobConf jobConf,
FileSystem localFS,
LocalDirAllocator localDirAllocator,
Reporter reporter,
CompressionCodec codec,
Class<? extends Reducer> combinerClass,
CombineOutputCollector<K,V> combineCollector,
Counters.Counter spilledRecordsCounter,
Counters.Counter reduceCombineInputCounter,
Counters.Counter mergedMapOutputsCounter,
ExceptionReporter exceptionReporter,
Progress mergePhase, MapOutputFile mapOutputFile) {
...
// SHUFFLE_INPUT_BUFFER_PERCENT = "mapreduce.reduce.shuffle.input.buffer.percent"
// 默认是0.7 Reduce阶段内存缓冲区中用于shuffle的百分比
final float maxInMemCopyUse =
jobConf.getFloat(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT,
MRJobConfig.DEFAULT_SHUFFLE_INPUT_BUFFER_PERCENT);
if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
throw new IllegalArgumentException("Invalid value for " +
MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
maxInMemCopyUse);
}
// Allow unit tests to fix Runtime memory
// REDUCE_MEMORY_TOTAL_BYTES = "mapreduce.reduce.memory.totalbytes"
// 用于shuffle的buffer大小
// 此值是由Reduce运行时的内存*shuffle百分比得到的
this.memoryLimit =
(long)(jobConf.getLong(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES,
Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
* maxInMemCopyUse);
this.ioSortFactor = jobConf.getInt(MRJobConfig.IO_SORT_FACTOR, 100);
// SHUFFLE_MEMORY_LIMIT_PERCENT = "mapreduce.reduce.shuffle.memory.limit.percent"
// 单个shuffle的内存限制,默认是0.25
final float singleShuffleMemoryLimitPercent =
jobConf.getFloat(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT,
DEFAULT_SHUFFLE_MEMORY_LIMIT_PERCENT);
if (singleShuffleMemoryLimitPercent <= 0.0f
|| singleShuffleMemoryLimitPercent > 1.0f) {
throw new IllegalArgumentException("Invalid value for "
+ MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+ singleShuffleMemoryLimitPercent);
}
usedMemory = 0L;
commitMemory = 0L;
// singleShuffleMemoryLimitPercent = mapreduce.reduce.shuffle.memory.limit.percent
// 每次写入时的最大限制,超过则直接写入disk
this.maxSingleShuffleLimit =
(long)(memoryLimit * singleShuffleMemoryLimitPercent);
// 在内存中merge的阈值,该阈值只有在开启内存merge功能才有用
// 内存merge的开关是mapreduce.reduce.merge.memtomem.enabled,默认关闭
this.memToMemMergeOutputsThreshold =
jobConf.getInt(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, ioSortFactor);
// SHUFFLE_MERGE_PERCENT = "mapreduce.reduce.shuffle.merge.percent"
// merge阈值,用于shuffle的内存*mapreduce.reduce.shuffle.merge.percent
this.mergeThreshold = (long)(this.memoryLimit *
jobConf.getFloat(MRJobConfig.SHUFFLE_MERGE_PERCENT,
0.90f));
...
// 内存merge功能是否开启
boolean allowMemToMemMerge =
jobConf.getBoolean(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, false);
if (allowMemToMemMerge) {
this.memToMemMerger =
new IntermediateMemoryToMemoryMerger(this,
memToMemMergeOutputsThreshold);
this.memToMemMerger.start();
} else {
this.memToMemMerger = null;
}
// 启动内存merge到Disk的merger线程
this.inMemoryMerger = createInMemoryMerger();
this.inMemoryMerger.start();
// 启动disk merge线程
this.onDiskMerger = new OnDiskMerger(this);
this.onDiskMerger.start();
this.mergePhase = mergePhase;
}
- merger.resolve()会判断当前的内容是输出到mem还是disk
public synchronized MapOutput<K,V> reserve(TaskAttemptID mapId,
long requestedSize,
int fetcher
) throws IOException {
// 判断当前内容大小是否能写入Memory,不能则直接写入disk
// requestedSize < maxSingleShuffleLimit 为true
// memoryLimit * mapreduce.reduce.shuffle.memory.limit.percent
if (!canShuffleToMemory(requestedSize)) {
LOG.info(mapId + ": Shuffling to disk since " + requestedSize +
" is greater than maxSingleShuffleLimit (" +
maxSingleShuffleLimit + ")");
return new OnDiskMapOutput<K,V>(mapId, reduceId, this, requestedSize,
jobConf, mapOutputFile, fetcher, true);
}
// Stall shuffle if we are above the memory limit
// 下面这段英语的意思是说:
// 如果将(usedMemory > memoryLimit) 改为 (usedMemory + requestedSize > memoryLimit)
// 会出现所有的Fetcher线程被死循环
// 当used size < mergeThreshold && requested size < singleShuffleLimit
// && usedMemory + requestedSize > memoryLimit 则写操作暂停,但此时并没有
// 超过mergeThreshold,并不会触发merge Thread,则假如所有的Fetcher的requested size 都满足
// usedMemory + requestedSize > memoryLimit,则此时all threads could just be stalling and not
// make progress at all.
// It is possible that all threads could just be stalling and not make
// progress at all. This could happen when:
//
// requested size is causing the used memory to go above limit &&
// requested size < singleShuffleLimit &&
// current used size < mergeThreshold (merge will not get triggered)
//
// To avoid this from happening, we allow exactly one thread to go past
// the memory limit. We check (usedMemory > memoryLimit) and not
// (usedMemory + requestedSize > memoryLimit). When this thread is done
// fetching, this will automatically trigger a merge thereby unlocking
// all the stalled threads
// 如果usedMemory > memoryLimit 则返回null,暂定shuffle,等待memToDisk
if (usedMemory > memoryLimit) {
LOG.debug(mapId + ": Stalling shuffle since usedMemory (" + usedMemory
+ ") is greater than memoryLimit (" + memoryLimit + ")." +
" CommitMemory is (" + commitMemory + ")");
return null;
}
// Allow the in-memory shuffle to progress
LOG.debug(mapId + ": Proceeding with shuffle since usedMemory ("
+ usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")."
+ "CommitMemory is (" + commitMemory + ")");
// usedMemory += requestedSize and new InMemoryMapOutput
return unconditionalReserve(mapId, requestedSize, true);
}
由merger.reserve()得到OnDiskMapOutput或者InMemoryMapOutput之后,调用mapOutput.shuffle将内容通过输出流写入Memory或者Disk。随后由scheduler.copySucceeded进行收尾工作,主要包括将已完成copy的map状态设置为true(map copy的状态存储在finishedMaps的boolean数组中,mapIndex为数组的id,完成则为true),完成一些统计信息,以及由output.commit()提交并关闭mapOutput流,在关闭的过程中会判断是否需要merge。
sort
真正的sort逻辑是在merger.close(),之所以把把此阶段称为SORT阶段,可能是因为在此阶段是纯粹的SORT吧,而不掺杂别的操作,这就可以解释为什么在之前的COPY阶段虽然也存在SORT,但并没有将SORT阶段从此处开始。下面跟下代码:
public RawKeyValueIterator close() throws Throwable {
...
return finalMerge(jobConf, rfs, memory, disk);
}
private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
List<InMemoryMapOutput<K,V>> inMemoryMapOutputs,
List<CompressAwarePath> onDiskMapOutputs
) throws IOException {
...
// 每个reduce最大的buffer占百分比
final float maxRedPer =
job.getFloat(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, 0f);
...
// reduce用于存储buffer的最大内存
int maxInMemReduce = (int)Math.min(
Runtime.getRuntime().maxMemory() * maxRedPer, Integer.MAX_VALUE);
// merge config params
Class<K> keyClass = (Class<K>)job.getMapOutputKeyClass();
Class<V> valueClass = (Class<V>)job.getMapOutputValueClass();
boolean keepInputs = job.getKeepFailedTaskFiles();
final Path tmpDir = new Path(reduceId.toString());
final RawComparator<K> comparator =
(RawComparator<K>)job.getOutputKeyComparator();
// segments required to vacate memory
List<Segment<K,V>> memDiskSegments = new ArrayList<Segment<K,V>>();
long inMemToDiskBytes = 0;
// 标识是否进行mergePhase是否完成,是否执行内部merge?????
boolean mergePhaseFinished = false;
if (inMemoryMapOutputs.size() > 0) {
TaskID mapId = inMemoryMapOutputs.get(0).getMapId().getTaskID();
inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs,
memDiskSegments,
maxInMemReduce);
final int numMemDiskSegments = memDiskSegments.size();
// 不符合条件就走到了if底
// 内存中有数据,磁盘中的文件少,
if (numMemDiskSegments > 0 &&
ioSortFactor > onDiskMapOutputs.size()) {
// If we reach here, it implies that we have less than io.sort.factor
// disk segments and this will be incremented by 1 (result of the
// memory segments merge). Since this total would still be
// <= io.sort.factor, we will not do any more intermediate merges,
// the merge of all these disk segments would be directly fed to the
// reduce method
mergePhaseFinished = true;
// must spill to disk, but can't retain in-mem for intermediate merge
final Path outputPath =
mapOutputFile.getInputFileForWrite(mapId,
inMemToDiskBytes).suffix(
Task.MERGED_OUTPUT_PREFIX);
final RawKeyValueIterator rIter = Merger.merge(job, fs,
keyClass, valueClass, memDiskSegments, numMemDiskSegments,
tmpDir, comparator, reporter, spilledRecordsCounter, null,
mergePhase);
FSDataOutputStream out = CryptoUtils.wrapIfNecessary(job, fs.create(outputPath));
Writer<K, V> writer = new Writer<K, V>(job, out, keyClass, valueClass,
codec, null, true);
try {
Merger.writeFile(rIter, writer, reporter, job);
writer.close();
onDiskMapOutputs.add(new CompressAwarePath(outputPath,
writer.getRawLength(), writer.getCompressedLength()));
writer = null;
// add to list of final disk outputs.
} catch (IOException e) {
if (null != outputPath) {
try {
fs.delete(outputPath, true);
} catch (IOException ie) {
// NOTHING
}
}
throw e;
} finally {
if (null != writer) {
writer.close();
}
}
LOG.info("Merged " + numMemDiskSegments + " segments, " +
inMemToDiskBytes + " bytes to disk to satisfy " +
"reduce memory limit");
inMemToDiskBytes = 0;
memDiskSegments.clear();
} else if (inMemToDiskBytes != 0) {
LOG.info("Keeping " + numMemDiskSegments + " segments, " +
inMemToDiskBytes + " bytes in memory for " +
"intermediate, on-disk merge");
}
}
// segments on disk
List<Segment<K,V>> diskSegments = new ArrayList<Segment<K,V>>();
long onDiskBytes = inMemToDiskBytes;
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,
(file.toString().endsWith(
Task.MERGED_OUTPUT_PREFIX) ?
null : mergedMapOutputsCounter), file.getRawDataLength()
));
}
LOG.info("Merging " + onDisk.length + " files, " +
onDiskBytes + " bytes from disk");
// 安装长度大小进行排序,方便进行最小堆排序??????
Collections.sort(diskSegments, new Comparator<Segment<K,V>>() {
public int compare(Segment<K, V> o1, Segment<K, V> o2) {
if (o1.getLength() == o2.getLength()) {
return 0;
}
return o1.getLength() < o2.getLength() ? -1 : 1;
}
});
// build final list of segments from merged backed by disk + in-mem
List<Segment<K,V>> finalSegments = new ArrayList<Segment<K,V>>();
long inMemBytes = createInMemorySegments(inMemoryMapOutputs,
finalSegments, 0);
LOG.info("Merging " + finalSegments.size() + " segments, " +
inMemBytes + " bytes from memory into reduce");
if (0 != onDiskBytes) {
final int numInMemSegments = memDiskSegments.size();
diskSegments.addAll(0, memDiskSegments);
memDiskSegments.clear();
// Pass mergePhase only if there is a going to be intermediate
// merges. See comment where mergePhaseFinished is being set
Progress thisPhase = (mergePhaseFinished) ? null : mergePhase;
RawKeyValueIterator diskMerge = Merger.merge(
job, fs, keyClass, valueClass, codec, diskSegments,
ioSortFactor, numInMemSegments, tmpDir, comparator,
reporter, false, spilledRecordsCounter, null, thisPhase);
diskSegments.clear();
if (0 == finalSegments.size()) {
return diskMerge;
}
finalSegments.add(new Segment<K,V>(
new RawKVIteratorReader(diskMerge, onDiskBytes), true, rawBytes));
}
return Merger.merge(job, fs, keyClass, valueClass,
finalSegments, finalSegments.size(), tmpDir,
comparator, reporter, spilledRecordsCounter, null,
null);
}
reduce
此阶段的主要逻辑在finalMerge中实现,这里的代码逻辑是先判断内存中的数据和磁盘中的文件个数的多少(ioSortFactor > onDiskMapOutputs.size()),符合条件则进行内存中的文件进行merge,输出到disk中。随后对disk中的文件进行merge,此处夹杂一行代码Collections.sort,对disk中的文件进行长度的排序,形成一个小顶堆进行merge,将内存和磁盘最终的merge文件放入finalSegments中进行最终的merge。