MapReduce的MapTask任务的运行源码级分析

  TaskTracker任务初始化及启动task源码级分析 这篇文章中分析了任务的启动,每个task都会使用一个进程占用一个JVM来执行,org.apache.hadoop.mapred.Child方法是具体的JVM启动类,其main方法中的taskFinal.run(job, umbilical)会启动具体的Task。

  Task分为两种类型:MapTask和ReduceTask,很明显,前者对应于Map任务,后者对应于Reduce任务。且MapTask分为4种:Job-setup Task、Job-cleanup Task、Task-cleanup Task和 Map Task。Job-setup Task、Job-cleanup Task分别是作业运行时启动的第一个任务和最后一个任务,主要工作分别是进行一些作业初始化和收尾工作,比如创建和删除作业临时输出目录;Task-cleanup Task则是任务失败或者被杀死后,用于清理已写入临时目录中数据的任务;最后一种Map Task则是处理数据并将结果存到本地磁盘上。

  本节先看MapTask,Child类调用run()方法,此类任务的run()方法代码如下:  

 1  @Override
 2   public void run(final JobConf job, final TaskUmbilicalProtocol umbilical) 
 3     throws IOException, ClassNotFoundException, InterruptedException {
 4       //负责与TaskTracker的通信,通过该对象可以获得必要的对象 
 5     this.umbilical = umbilical;
 6 
 7     // start thread that will handle communication with parent
 8     // 启动Reporter线程,用来和TaskTracker交互目前运行的状态
 9     TaskReporter reporter = new TaskReporter(getProgress(), umbilical,
10         jvmContext);
11     reporter.startCommunicationThread();
12     boolean useNewApi = job.getUseNewMapper();
13     /*用来初始化任务,主要是进行一些和任务输出相关的设置,比如创 建commiter,设置工作目录等*/ 
14     initialize(job, getJobID(), reporter, useNewApi);
15 
16     // check if it is a cleanupJobTask
17     /*以下4个if语句均是根据任务类型的不同进行相应的操作,这些方 法均是Task类的方法,所以与任务是MapTask还是ReduceTask无关*/ 
18     if (jobCleanup) {
19       runJobCleanupTask(umbilical, reporter);
20       return;
21     }
22     if (jobSetup) {
23         //主要是创建工作目录的FileSystem对象 
24       runJobSetupTask(umbilical, reporter);
25       return;
26     }
27     if (taskCleanup) {
28         //设置任务目前所处的阶段为结束阶段,并且删除工作目录 
29       runTaskCleanupTask(umbilical, reporter);
30       return;
31     }
32     //如果不是上述四种类型,则真正运行任务
33     if (useNewApi) {
34       runNewMapper(job, splitMetaInfo, umbilical, reporter);
35     } else {
36       runOldMapper(job, splitMetaInfo, umbilical, reporter);
37     }
38     done(umbilical, reporter);//等待JobTracker的commit命令
39   }

  (1)参数TaskUmbilicalProtocol,这个协议用于Child和TaskTracker之间的通信。Child通过此协议,查看TaskTracker是否存在,取得任务,报告任务的进度,状态,出错信息,Commit文件到HDFS,并取得map结果给reduce;TaskTracker接收任务并监控任务的进度。

  (2)TaskReporter类是是Task类的内部私有类。Task.TaskReporter用于向TaskTracker提交计数器报告和状态报告,它实现了计数器报告Reporter和状态报告StatusReporter。为了不影响主线程的工作,TaskReporter有一个独立的线程,该线程通过TaskUmbilicalProtocol接口,向TaskTracker报告Task执行情况。startCommunicationThread()方法会启动线程。

  (3)useNewApi = job.getUseNewMapper()获取这个Task使用的新的API还是旧的API。mapreduce job提交流程源码级分析(一)(原创)这篇文章有讲在Job提交的时候就设置了使用新的API(包括新的Mapper和新的Reducer)。

  (4)initialize(job, getJobID(), reporter, useNewApi)该方法在父类Task中。这个方法会将Task的状态设置为RUNNING,表示正在运行;然后如果是新API会获取对应的OutputFormatClass默认是TextOutputFormat.class,新API会获取mapreduce.FileOutputCommitter旧API会获取mapred.FileOutputCommitter;再获取在MapReduce程序中通过FileOutputFormat.setOutputPath设置的输出目录,如果这个输出目录不为null且是新的API会执行else语句FileOutputFormat.setWorkOutputPath(conf, outputPath)(这个是旧版mapred.FileOutputFormat)设置工作目录,比如hdfs://IP:8020/user/XXX,IP指的是namenode,XXX指的是用户名;然后构造一个资源计算器ResourceCalculatorPlugin对象,来获取内存、CPU等资源信息。

  (5)如果jobCleanup==true(是在TaskInProgress类中设置的)表明这个task是清理Job的。直接运行runJobCleanupTask(umbilical, reporter)方法,这个方法是清理Job,包括步骤状态设置,更新状态到TaskTracker,调用org.apache.hadoop.mapreduce.OutputCommitter的相关方法,删除目录,通过done,通知TaskTracker任务完成等待commit命令。

  (6)如果jobSetup==true(是在TaskInProgress类中设置的)表明要初始化Job,直接运行runJobSetupTask(umbilical, reporter)为建立Job做准备,执行状态设置,然后调用org.apache.hadoop.mapreduce.OutputCommitter的setupJob,最后通过done,通知TaskTracker任务完成等待commit命令。

  (7)如果是taskCleanup==true(是在TaskInProgress类中设置的)表明是清理task的任务,直接运行runTaskCleanupTask(umbilical, reporter),清理Task任务,和上面(5)中runJobCleanupTask类似。

  (8)接下来才是执行Mapper的步骤,如果不是上面的5,6,7三种,如果是启用新的API(实际上是启用的,我们也只分析新API),就执行runNewMapper(job, splitMetaInfo, umbilical, reporter)方法。

  (9)done(umbilical, reporter)这个方法也被上面的5,6,7调用了,这个方法用于做结束任务的一些清理工作:更新计数器updateCounters();如果任务需要提交,设置Taks状态为COMMIT_PENDING,并利用TaskUmbilicalProtocol,汇报Task完成,等待提交,然后调用commit提交任务;设置任务结束标志位;结束Reporter通信线程;发送最后一次统计报告(通过sendLastUpdate方法);利用TaskUmbilicalProtocol报告结束状态(通过sendDone方法)。

  下面我们来看(8)中的runNewMapper(job, splitMetaInfo, umbilical, reporter)方法方法,这个方法将会构造一系列的对象来辅助执行Mapper。其代码如下:

 1 private <INKEY,INVALUE,OUTKEY,OUTVALUE>
 2   void runNewMapper(final JobConf job,
 3                     final TaskSplitIndex splitIndex,
 4                     final TaskUmbilicalProtocol umbilical,
 5                     TaskReporter reporter
 6                     ) throws IOException, ClassNotFoundException,
 7                              InterruptedException {
 8       /*TaskAttemptContext类继承于JobContext类,相对于JobContext类增加了一些有关task的信息。
 9        * 通过taskContext对象可以获得很多与任务执行相关的类,比如用户定义的Mapper类,InputFormat类等等 */ 
10     // make a task context so we can get the classes
11     org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
12       new org.apache.hadoop.mapreduce.TaskAttemptContext(job, getTaskID());
13     // make a mapper//创建用户自定义的Mapper类的实例
14     org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE> mapper =
15       (org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>)
16         ReflectionUtils.newInstance(taskContext.getMapperClass(), job);
17     // make the input format 创建用户指定的InputFormat类的实例 
18     org.apache.hadoop.mapreduce.InputFormat<INKEY,INVALUE> inputFormat =
19       (org.apache.hadoop.mapreduce.InputFormat<INKEY,INVALUE>)
20         ReflectionUtils.newInstance(taskContext.getInputFormatClass(), job);
21     // rebuild the input split  重新生成InputSplit 
22     org.apache.hadoop.mapreduce.InputSplit split = null;
23     split = getSplitDetails(new Path(splitIndex.getSplitLocation()),
24         splitIndex.getStartOffset());
25   //根据InputFormat对象创建RecordReader对象,默认是LineRecordReader 
26     org.apache.hadoop.mapreduce.RecordReader<INKEY,INVALUE> input =
27       new NewTrackingRecordReader<INKEY,INVALUE>
28           (split, inputFormat, reporter, job, taskContext);
29 
30     job.setBoolean("mapred.skip.on", isSkipping());
31   //生成RecordWriter对象
32     org.apache.hadoop.mapreduce.RecordWriter output = null;
33     org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context 
34          mapperContext = null;
35     try {
36       Constructor<org.apache.hadoop.mapreduce.Mapper.Context> contextConstructor =
37         org.apache.hadoop.mapreduce.Mapper.Context.class.getConstructor
38         (new Class[]{org.apache.hadoop.mapreduce.Mapper.class,
39                      Configuration.class,
40                      org.apache.hadoop.mapreduce.TaskAttemptID.class,
41                      org.apache.hadoop.mapreduce.RecordReader.class,
42                      org.apache.hadoop.mapreduce.RecordWriter.class,
43                      org.apache.hadoop.mapreduce.OutputCommitter.class,
44                      org.apache.hadoop.mapreduce.StatusReporter.class,
45                      org.apache.hadoop.mapreduce.InputSplit.class});
46 
47       // get an output object
48       if (job.getNumReduceTasks() == 0) {
49          output =
50            new NewDirectOutputCollector(taskContext, job, umbilical, reporter);
51       } else {
52         output = new NewOutputCollector(taskContext, job, umbilical, reporter);
53       }
54 
55       mapperContext = contextConstructor.newInstance(mapper, job, getTaskID(),
56                                                      input, output, committer,
57                                                      reporter, split);
58       /*初始化,在默认情况下调用的是LineRecordReader的initialize方 法,主要是打开输入文件并且将文件指针指向文件头*/ 
59       input.initialize(split, mapperContext);
60       mapper.run(mapperContext);    //Mapper的执行
61       input.close();
62       output.close(mapperContext);
63     } catch (NoSuchMethodException e) {
64       throw new IOException("Can't find Context constructor", e);
65     } catch (InstantiationException e) {
66       throw new IOException("Can't create Context", e);
67     } catch (InvocationTargetException e) {
68       throw new IOException("Can't invoke Context constructor", e);
69     } catch (IllegalAccessException e) {
70       throw new IOException("Can't invoke Context constructor", e);
71     }
72   }
View Code

  (1)会获取配置信息类对象taskContext、自己开发的Mapper的实例mapper、用户指定的InputFormat对象inputFormat(默认是TextInputFormat)、任务对应的分片信息split

  (2)根据inputFormat构建一个NewTrackingRecordReader对象,这个对象中的RecordReader<K,V> real是LineRecordReader。这个类是读取分片中的内容的。

  (3)然后创建mapreduce.RecordWriter output,如果没有reducer(满足job.getNumReduceTasks() == 0),就output =new NewDirectOutputCollector(taskContext, job, umbilical, reporter)直接输出到HDFS上;如果有reducer,就output = new NewOutputCollector(taskContext, job, umbilical, reporter)作为输出,这俩都继承自org.apache.hadoop.mapreduce.RecordWriter类。output是map任务的输出。

  (4)input.initialize(split, mapperContext)初始化,在默认情况下调用的是LineRecordReader的initialize方法,主要是打开输入文件(构建一个LineReader对象,在这实现文件内容的具体读)并且将文件指针指向文件头。

  (5)mapper.run(mapperContext)这里是具体执行mapper的地方,下面再讲。

  (6)最后mapper执行完毕之后,就会关闭输入输出流:input.close();output.close(mapperContext)。

  上面这些就是MapTask的执行过程。还有一些地方需要再详细解读一下:

  一、NewDirectOutputCollector是没有reducer的作业,直接将map的输出写入HDFS中。输出流mapreduce.RecordWriter out = outputFormat.getRecordWriter(taskContext),默认是TextOutputFormat.getRecordWriter(taskContext)这个方法会判断有无压缩配置项,然后通过Path file = getDefaultWorkFile(job, extension),extension这个参数如果没有压缩选项会为空,获取输出文件的写入目录和文件名,形"$output/_temporary/_$taskid/part-[mr]-$id",这个$output是你MR程序设置的输出目录,_temporary/_$taskid这个是临时目录,part-[mr]-$id这一部分是通过getUniqueFile获取的文件名,其中的mr是看具体的task任务类型而定,id就是taskid;getRecordWriter方法最终会返回LineRecordWriter<K, V>(fileOut, keyValueSeparator),fileOut是FSDataOutputStream指向要写入的文件,keyValueSeparator是数据的分隔符,可通过"mapred.textoutputformat.separator"来配置,默认是" "表示输入数据要以 分割。NewDirectOutputCollector.write(K key, V value)其实是调用out.write(key, value)来完成写入HDFS文件的。

  二、NewOutputCollector是有reducer的作业的map的输出。这个类的主要包含的对象是MapOutputCollector<K,V> collector = new MapOutputBuffer<K,V>(umbilical, job, reporter),并且实例化了mapreduce.Partitioner<K,V> partitioner(默认是HashPartitioner.class)这个是用来对mapper的输出数据进行分区的就是要数据要汇总到那个reducer上,NewOutputCollector的write方法会调用collector.collect(key, value,partitioner.getPartition(key, value, partitions))。

  三、LineRecordReader类,是用来从指定的文件读取内容传递给Mapper的map方法做处理的。实际上读文件内容的是类中的LineReader对象in,该对象在initialize方法(上面的步骤(4))中进行了初始化,会根据输入文件的文件类型(压缩或不压缩)传入相应输入流对象。LineReader会总输入流对象中通过readLine(Text str, int maxLineLength,int maxBytesToConsume)(其实有3个readLine方法,但是最终都会调用这个方法)方法每次读取一行放入str中,并返回读取数据的长度。LineRecordReader.nextKeyValue()方法会设置两个对象key和value,key是一个偏移量指的是当前这行数据在输入文件中的偏移量(注意这个偏移量可不是对应单个分片内的偏移量,而是针对整个分布式文中的偏移量),value是通过LineReader的对象in读取的一行内容,如果没有数据可读了,这个方法会返回false,否则true。getCurrentKey()和getCurrentValue()是获取当前的key和value,调用这俩方法之前需要先调用nextKeyValue()为key和value赋新值,否则会重复,当然我们不用考虑这个因为在mapper.run方法中已经做了。

  四、mapper.run方法开始执行mapper,因为使用的新的API,我们查看类org.apache.hadoop.mapreduce.Mapper(我们开发的Mapper都是继承自这个类,我们只需要实现map方法即可),该类下的run方法代码如下:

1 public void run(Context context) throws IOException, InterruptedException {
2     setup(context);
3     while (context.nextKeyValue()) {
4       map(context.getCurrentKey(), context.getCurrentValue(), context);
5     }
6     cleanup(context);
7   }

  首先会执行setup方法,我们在开发自己的mapper时有时需要传一些自己的参数,可以写入context,自己重写setup方法,获取这个参数;然后循环调用nextKeyValue()方法获取key和value,执行map方法,这里有疑问了,上面讲的不是reader是用来读数据的么,这里怎么是context了?我们查看runNewMapper中的mapperContext,这是mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context对象,这个Context是Mapper的一个内部类,这句mapperContext = contextConstructor.newInstance(mapper, job, getTaskID(),input, output, committer,reporter, split)会实例化一个mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context对象,会将LineRecordReader的实例和NewOutputCollector的实例传进去,mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context extends MapContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT>,在MapContext类中LineRecordReader的实例会被赋给RecordReader<KEYIN,VALUEIN> reader,然后会有同样的nextKeyValue()、getCurrentValue()、getCurrentKey()会调用reader的相应方法,从而实现了Mapper.run方法中的nextKeyValue()不断获取key和value。回到run方法,循环中的map方法就是我们自己的map,当读完数据之后,会调用cleanup方法来做一些清理工作,这点我们同样可以利用,我们可以根据自己的需要重写cleanup方法。

  另外我们自己的map方法中最后都会有context.write(K,V)方法用来将计算数据输出,我们顺着上一段继续追查MapContext类中并无write方法,但是它继承自TaskInputOutputContext类,进去发现RecordWriter<KEYOUT,VALUEOUT> output这个对象是输出对象,被赋值NewOutputCollector,其write方法直接调用的是NewOutputCollector.write方法,我们上面的二已经说了,write方法会调用MapOutputBuffer.collect(key, value,partitioner.getPartition(key, value, partitions))方法,将数据先写入缓存中。好麻烦是吧,呵呵。

  五、接下来我们看看MapOutputBuffer implements MapOutputCollector这个类了。这个类比较复杂,有1000行代码。该类内部使用一个缓冲区暂时存储用户输出数据,当缓冲区使用率达到一定阈值后,再讲缓冲区中的数据写到磁盘上。Hadoop的这个缓冲区采用环形缓冲区:当缓冲区使用率达到一定的阈值后,便开始向磁盘上写入数据,同时生产者扔可以向不断增加的剩余空间中循环写入数据,进而达到读写并行(Map Task的collect阶段和spill阶段),性能也比较高。

  MapOutputBuffer采用两级索引结构,涉及三个环形缓冲区:int[] kvoffsets(偏移量索引数组,保存KV信息在位置索引kvindices中的偏移量)、int[] kvindices(位置索引数组,用于保存KV值在数据缓冲区kvbuffer中的起始位置)、byte[] kvbuffer(数据缓冲区,保存实际的KV值,默认情况下最多使用io.sort.mb的95%)。一对KV需占用数组kvoffsets的1个int大小,数组kvindices的3个int大小(分别保存所在partion号、key值开始位置、Value值开始位置),所以按比例1:3将大小为${io.sort.record.percent}*${io.sort.mb}的内存空间分配给数组kvoffsets和kvindices,默认是0.05*100MB。

  MapOutputBuffer类中有一个BlockingBuffer extends DataOutputStream内部类,该类中的OutputStream out对象也是MapOutputBuffer的一个内部类Buffer extends OutputStream,Buffer主要是对kvbuffer操纵,BlockingBuffer的实例化对象是bb,该值同时是keySerializer和valSerializer(默认都是org.apache.hadoop.io.serializer.WritableSerialization的内部类WritableSerializer)的输出流对象。

  MapOutputBuffer.collect方法每次都会先检查kvoffsets数组的有效容量是否超过io.sort.spill.percent,默认0.8,如果超过则唤醒spill线程写到临时文件中( startSpill()方法完成);然后通过keySerializer.serialize(key)将key写入上述说的bb输出流中,实际最终调用的是Buffer.write(byte b[], int off, int len),这个方法会将key写入环形缓冲区kvbuffer中,如果kvbuffer的有效内存容量超过io.sort.spill.percent则会唤醒spill线程写到临时文件中( startSpill()方法完成),如果发生key跨界情况(bufindex < keystart),要保证key不能跨界(因为是排序的关键字要求排序关键字连续存储),会调用bb.reset()来直接操纵kvbuffer处理两种情况(一种是头部可以放下key,另外一种则不可以);然后是keySerializer.serialize(key),写到kvbuffer中,可以参考序列化key时的过程,但value可以跨界。如果遇到一条记录的key或者value太大以至于真个缓冲区都放不下,则会抛出MapBufferTooSmallException,执行spillSingleRecord(key, value, partition)会将该记录单独输出到一个文件中。

  可以看出触发spill溢写操作的条件是:kvoffsets或者kvbuffer有效容量超过io.sort.spill.percent;出现一条缓冲区kvbuffer无法容纳的超大记录。

   SpillThread线程在构造方法中已经启动,线程的run方法就是一直等待被唤醒,一旦唤醒就调用sortAndSpill()方法排序并写文件,startSpill()会唤醒这个线程。

 sortAndSpill()方法代码如下:

View Code

   先计算写入文件的大小;然后获取写到本地(非HDFS)文件的文件名,会有一个编号,例如output/spill2.out;然后构造一个输出流;然后使用快排对缓冲区kvbuffe中区间[bufstart,bufend)内的数据进行排序,先按分区编号partition进行排序,然后按照key进行排序。这样经过排序后,数据以分区为单位聚集在一起,且同一分区内所有数据按照key有序。

  会构建一个IFile.Writer对象将输出流传进去,输出到指定的文件当中,这个对象支持行级的压缩。如果用户设置了Combiner(实际上是一个reducer),则写入文件之前会对每个分区中的数据进行一次聚集操作,通过combinerRunner.combine(kvIter, combineCollector)实现,因为使用了新版的API,所以combinerRunner会是NewCombinerRunner,它的combine方法会执行reducer.run方法,只不过输出和正常的reducer不一样而已,这里最终会调用IFile.Writer的append方法实现本地文件的写入。

  还有将分区数据的元信息写到内存索引数据结构SpillRecord中,其中每个分区的元信息包括在临时文件中的偏移量、压缩前数据大小和压缩后数据大小。如果内存中索引大于1MB,则写到文件output/spill2.out.index中。
  runNewMapper方法的最后会有输出流的关闭:output.close(mapperContext),其实就是NewOutputCollector.close(mapperContext)该方法会执行MapOutputBuffer.flush()操作会将剩余的数据也通过sortAndSpill()方法写入本地文件,并在最后调用mergeParts()方法合并所有spill文件。代码如下:  

  1 private void mergeParts() throws IOException, InterruptedException, 
  2                                      ClassNotFoundException {
  3       // get the approximate size of the final output/index files
  4       long finalOutFileSize = 0;
  5       long finalIndexFileSize = 0;
  6       final Path[] filename = new Path[numSpills];
  7       final TaskAttemptID mapId = getTaskID();
  8 
  9       for(int i = 0; i < numSpills; i++) {
 10         filename[i] = mapOutputFile.getSpillFile(i);    //通过spill文件的编号获取到指定的spill文件路径
 11         finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
 12       }
 13       //合并输出有俩文件一个是output/file.out,一个是output/file.out.index
 14       if (numSpills == 1) { //the spill is the final output
 15         rfs.rename(filename[0],
 16             new Path(filename[0].getParent(), "file.out"));
 17         if (indexCacheList.size() == 0) {
 18           rfs.rename(mapOutputFile.getSpillIndexFile(0),
 19               new Path(filename[0].getParent(),"file.out.index"));
 20         } else {    //写入文件
 21           indexCacheList.get(0).writeToFile(
 22                 new Path(filename[0].getParent(),"file.out.index"), job);
 23         }
 24         return;
 25       }
 26 
 27       // read in paged indices
 28       for (int i = indexCacheList.size(); i < numSpills; ++i) {
 29         Path indexFileName = mapOutputFile.getSpillIndexFile(i);
 30         indexCacheList.add(new SpillRecord(indexFileName, job, null));
 31       }
 32 
 33       //make correction in the length to include the sequence file header
 34       //lengths for each partition
 35       finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
 36       finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
 37       Path finalOutputFile =
 38           mapOutputFile.getOutputFileForWrite(finalOutFileSize);   //output/file.out
 39       Path finalIndexFile =
 40           mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);    //output/file.out.index
 41 
 42       //The output stream for the final single output file
 43       FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
 44 
 45       if (numSpills == 0) {
 46         //create dummy(假的,假设) files
 47         IndexRecord rec = new IndexRecord();
 48         SpillRecord sr = new SpillRecord(partitions);
 49         try {
 50           for (int i = 0; i < partitions; i++) {
 51             long segmentStart = finalOut.getPos();
 52             Writer<K, V> writer =
 53               new Writer<K, V>(job, finalOut, keyClass, valClass, codec, null);
 54             writer.close();
 55             rec.startOffset = segmentStart;
 56             rec.rawLength = writer.getRawLength();
 57             rec.partLength = writer.getCompressedLength();
 58             sr.putIndex(rec, i);
 59           }
 60           sr.writeToFile(finalIndexFile, job);
 61         } finally {
 62           finalOut.close();
 63         }
 64         return;
 65       }
 66       {
 67         IndexRecord rec = new IndexRecord();
 68         final SpillRecord spillRec = new SpillRecord(partitions);
 69         //finalOut最终输出文件。循环分区获得所有spill文件的该分区数据,合并写入finalOut
 70         for (int parts = 0; parts < partitions; parts++) {
 71           //create the segments to be merged
 72           List<Segment<K,V>> segmentList =
 73             new ArrayList<Segment<K, V>>(numSpills);
 74           for(int i = 0; i < numSpills; i++) {
 75             IndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
 76 
 77             Segment<K,V> s =
 78               new Segment<K,V>(job, rfs, filename[i], indexRecord.startOffset,
 79                                indexRecord.partLength, codec, true);
 80             segmentList.add(i, s);
 81 
 82             if (LOG.isDebugEnabled()) {
 83               LOG.debug("MapId=" + mapId + " Reducer=" + parts +
 84                   "Spill =" + i + "(" + indexRecord.startOffset + "," +
 85                   indexRecord.rawLength + ", " + indexRecord.partLength + ")");
 86             }
 87           }
 88 
 89           //merge
 90           @SuppressWarnings("unchecked")
 91           RawKeyValueIterator kvIter = Merger.merge(job, rfs,
 92                          keyClass, valClass, codec,
 93                          segmentList, job.getInt("io.sort.factor", 100),//做merge操作时同时操作的stream数上限
 94                          new Path(mapId.toString()),
 95                          job.getOutputKeyComparator(), reporter,
 96                          null, spilledRecordsCounter);
 97 
 98           //write merged output to disk
 99           long segmentStart = finalOut.getPos();
100           Writer<K, V> writer =
101               new Writer<K, V>(job, finalOut, keyClass, valClass, codec,
102                                spilledRecordsCounter);
103        // minSpillsForCombine 在MapOutputBuffer构造函数内被初始化,  
104        // numSpills 为mapTask已经溢写到磁盘spill文件数量  
105           if (combinerRunner == null || numSpills < minSpillsForCombine) {
106             Merger.writeFile(kvIter, writer, reporter, job);
107           } else {
108             combineCollector.setWriter(writer);
109           //其实写入数据的还是这里的writer类的append方法,这的输出是output/file.out文件,是合并后的文件
110             combinerRunner.combine(kvIter, combineCollector);
111           }
112 
113           //close
114           writer.close();
115 
116           // record offsets
117           rec.startOffset = segmentStart;
118           rec.rawLength = writer.getRawLength();
119           rec.partLength = writer.getCompressedLength();
120           spillRec.putIndex(rec, parts);
121         }
122         spillRec.writeToFile(finalIndexFile, job);    //写入索引文件
123         finalOut.close();        //合并后的输出文件
124         for(int i = 0; i < numSpills; i++) {
125           rfs.delete(filename[i],true);
126         }
127       }
128     }
View Code

   该方法会将所有临时文件合并成一个大文件保存到output/file.out中,同时生成相应的索引文件output/file.out.index。 在进行文件合并的过程中,Map Task以分区为单位进行合并。对于某个分区,它将采用多轮递归合并的方式:每轮合并io.sort.factor,默认是100,个文件,并将产生的文 件重新加入待合并列表中,对文件排序后,重复上述过程,直到只有一个文件。只生产一个文件可以避免同时打开大量的文件和同时读取大量的小文件产生的随机读 取带来的开销。最后会删除所有的spill文件。

  另外需要注意的是,mergeParts()中也有combiner的操作,但是需要满足一定的条件:1、用户设置了combiner;2、spill文件的数量超过了minSpillsForCombine的值,对应配置项"min.num.spills.for.combine",可自行设置,默认是3。这俩必须同时具备才会在此启动combiner的本地聚集操作。所以在Map阶段有可能combiner会执行两次,所以有可能你的combiner执行两次之后输出数据不符合预期了。

   

  这样Map阶段的任务就算完成了。主要是读取数据然后写入内存缓冲区,缓存区满足条件就会快排后并设置partition后,spill到本地文件和索引文件;如果有combiner,spill之前也会做一次聚集操作,待数据跑完会通过归并合并所有spill文件和索引文件,如果有combiner,合并之前在满足条件后会做一次综合的聚集操作。map阶段的结果都会存储在本地中(如果有reducer的话),非HDFS。

   参考:1、董西成,《hadoop技术内幕---深入理解MapReduce架构设计与实现原理》

原文地址:https://www.cnblogs.com/lxf20061900/p/3792190.html