记一次神奇的 MapReduce OOM

背景

使用 HiBench 对 CDH 集群中各个组件进行基准测试(HiBench的使用这里不过多赘述)。由于最初 conf/hibench.conf 文件中配置的 hibench.scale.profile (影响数据规模的参数) 为 huge, hibench.default.map.parallelism & hibench.default.shuffle.parallelism (影响并行度的参数) 为 30。最终导致在 sql.aggregation 测试项中,总共只有 1.7 G 的数据被分割为了 30 个文件。

hdfs@t1:~/HiBench-master/report/aggregation/hadoop$ hdfs dfs -du -h /HiBench/Aggregation/Input/uservisits
0       0        /HiBench/Aggregation/Input/uservisits/_SUCCESS
59.4 M  178.1 M  /HiBench/Aggregation/Input/uservisits/part-00000
59.4 M  178.3 M  /HiBench/Aggregation/Input/uservisits/part-00001
...
59.4 M  178.3 M  /HiBench/Aggregation/Input/uservisits/part-00028
59.6 M  178.9 M  /HiBench/Aggregation/Input/uservisits/part-00029

这时,测试 SQL 的执行效率并不高,但这不是重点。重点是脑洞大开的我想着这些小文件连 128 M 都不到,如果两两合并可以让文件数减少为原来的一半,以此降低 NameNode 所需存储的元数据信息。

而且,Hive 处理数据时,默认情况下如果单个文件小于 128 M,即使只有10 M,也会分配一个 map 任务单独处理该文件。这就导致 map 任务太多,每个 map 处理的数据量太小。(map 数量过多过少都不好,平衡这个数量真的很玄学,实践或许才是检验真理的唯一标准)

$ cat uservisits_aggre.hive 
USE DEFAULT;
set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
set mapreduce.job.maps=15; # 软性指定 15 后, map 数量依旧为 30
set mapreduce.job.reduces=3;
set hive.stats.autogather=false;

DROP TABLE IF EXISTS uservisits;
CREATE EXTERNAL TABLE uservisits (sourceIP STRING,destURL STRING,visitDate STRING,adRevenue DOUBLE,userAgent STRING,countryCode STRING,languageCode STRING,searchWord STRING,duration INT ) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' STORED AS  SEQUENCEFILE LOCATION 'hdfs://t1.dev.dxy.cn:8020/HiBench/Aggregation/Input/uservisits';
DROP TABLE IF EXISTS uservisits_aggre;
CREATE EXTERNAL TABLE uservisits_aggre ( sourceIP STRING, sumAdRevenue DOUBLE) STORED AS  SEQUENCEFILE LOCATION 'hdfs://t1.dev.dxy.cn:8020/HiBench/Aggregation/Output/uservisits_aggre';
INSERT OVERWRITE TABLE uservisits_aggre SELECT sourceIP, SUM(adRevenue) FROM uservisits GROUP BY sourceIP;

$ hive -f uservisits_aggre.hive 
...
Hadoop job information for Stage-1: number of mappers: 30; number of reducers: 3
...

因此,尝试将这些小文件合并一下,那么问题来了!这种 SEQUENCEFILE 能不能像文本文件那样直接 cat 出来然后追加到另一个文件后面来合并呢?而且这里我想当然的认为,最坏的情况下也只不过应该是处理 Sequence 文件时由于格式错误报一个 IOException 的异常出来吧?

然而,事与愿违。在以 cat 追加的方式合并完后,再次执行上述测试 sql,没有发现任何文件格式相关的 IO 错误,而是直接 Java heap space 堆空间溢出了……

合并方式如下:

hdfs dfs -cat /HiBench/Aggregation/Input/uservisits/part-00001 | hdfs dfs -appendToFile - /HiBench/Aggregation/Input/uservisits/part-00000
hdfs dfs -rm /HiBench/Aggregation/Input/uservisits/part-00001
...
hdfs dfs -cat /HiBench/Aggregation/Input/uservisits/part-00029 | hdfs dfs -appendToFile - /HiBench/Aggregation/Input/uservisits/part-00028
hdfs dfs -rm /HiBench/Aggregation/Input/uservisits/part-00029

hdfs@t1:~/HiBench-master/report/aggregation/hadoop$ hdfs dfs -du -h /HiBench/Aggregation/Input/uservisits
0        0        /HiBench/Aggregation/Input/uservisits/_SUCCESS
118.8 M  356.4 M  /HiBench/Aggregation/Input/uservisits/part-00000
119.1 M  357.3 M  /HiBench/Aggregation/Input/uservisits/part-00002
...
119.0 M  357.0 M  /HiBench/Aggregation/Input/uservisits/part-00026
119.1 M  357.2 M  /HiBench/Aggregation/Input/uservisits/part-00028

再次执行后,错误信息如下:

Task with the most failures(4): 
-----
Task ID:
  task_1526882261055_0006_m_000002

URL:
  http://t1.dev.dxy.cn:8088/taskdetails.jsp?jobid=job_1526882261055_0006&tipid=task_1526882261055_0006_m_000002
-----
Diagnostic Messages for this Task:
Error: Java heap space

故障分析

这个报错信息非常具有误导性,任何脑回路正常的人或许都会觉得这个问题是由于合并了文件后,每个 map 处理的数据量过大导致。从而在错误的思路上越走越远……

通过对内存参数 N 多次调整之后,都无法得到一个能够自圆其说的结论,都是在求证过程中就自相矛盾了。

最终不得不下载对应 CDH 版本的 Hadoop 源码对 map 任务进行远程 debug。

因此我们需要先从这失败的 15 个 map 任务的执行日志中来寻找蛛丝马迹。

2018-05-21 15:34:11,773 FATAL [main] org.apache.hadoop.mapred.YarnChild: Error running child : java.lang.OutOfMemoryError: Java heap space
    at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:66)
    at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:120)
    at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2446)
    at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2578)
    at org.apache.hadoop.mapred.SequenceFileRecordReader.next(SequenceFileRecordReader.java:82)
    at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:360)
    at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:79)
    at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:33)
    at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:116)
    at org.apache.hadoop.mapred.MapTask$TrackedRecordReader.moveToNext(MapTask.java:199)
    at org.apache.hadoop.mapred.MapTask$TrackedRecordReader.next(MapTask.java:185)
    at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:52)
    at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:459)
    at org.apache.hadoop.mapred.MapTask.run(MapTask.java:343)
    at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:415)
    at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1920)
    at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)

观察 DataOutputBuffer.java:66 的源码,如下所示:

这行代码如果想把内存搞到溢出,很大概率是开了个极大的 byte 数组。那么我们在这样先下一个断点,观察下运行过程中,这个数组一般情况下都开的多大吧。

通过多次循环操作,发现这个值基本在几十到几百之间,上图中的值为 220。完全不足以导致内存溢出。因此我们需要定位内存溢出时,这个值到底多大,而且要定位出这个值来自哪里。

DataOutputBuffer.java:120 处下一个条件断点,当开辟的数组大小大于 102400 时触发。

最终断下来的时候,发现尝试分配的 byte 数组大小为 1397051654。

值找到了,然后需要再进一步向前追溯。

最终找到该值为 readRecordLength 方法返回。

readRecordLength 内部实现代码如下:

readInt 的返回值为:\
the next four bytes of this input stream, interpreted as an int.

因此没猜错的话,这个 Sequence 文件中一定包含可以解析出 1397051654 这个十进制数值的二进制数据。

由于工具读取二进制文件时都有 16 进制对应的数值,所以找到该 10 进制数据的 16 进制表现形式 5345 5106 。

最终以二进制格式打开这个文件看到文件头时,一切都真相大白了。

一开始我们将一个 sequence 文件 cat 出来追加到另一个 sequence 文件后面,那么我们最终得到的这个文件在两文件衔接点的地方一定还有一段标记文件头的二进制数据。不妨搜索一下。

果然,我们在文件中部又找到了这个标记

由于该段数据被误判为记录的长度,map 任务尝试创建一个 1397051654 个元素的 byte 数组(1.3 G 左右),最终导致 map 任务发生了内存溢出。

解决方案

如果真的要合并 sequence 文件要怎么做呢?

样例代码如下:

package com.temple.main;

import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.*;
import org.apache.hadoop.io.SequenceFile.Writer;
import org.apache.hadoop.util.ReflectionUtils;

public class AppendSequenceFile
{
    public static void main(String[] args) throws IOException {

        if (args.length != 2) {
            System.out.println("usages: appendSequenceFile.jar + inputFilePath + targetFilePath");
            return;
        }

        Configuration conf = new Configuration();

        String targetFile = args[1];
        Path targetPath = new Path(targetFile);

        String inputFile = args[0];
        Path inputPath = new Path(inputFile);

        SequenceFile.Reader reader = new SequenceFile.Reader(conf, SequenceFile.Reader.file(inputPath), SequenceFile.Reader.bufferSize(4096), SequenceFile.Reader.start(0));
        Writable inputKey = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
        Writable inputValue = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);

        SequenceFile.Writer writer = null;
        try
        {
            writer = SequenceFile.createWriter(conf, Writer.file(targetPath),
                    Writer.keyClass(inputKey.getClass()),
                    Writer.valueClass(inputValue.getClass()),
                    Writer.compression(reader.getCompressionType(), reader.getCompressionCodec()),
                    SequenceFile.Writer.appendIfExists(true) ); // 控制如果已经存在该文件,则采用追加的方式写入
            while (reader.next(inputKey, inputValue)) {
                //System.out.printf("%s\t%s\n", inputKey, inputValue);
                writer.append(inputKey, inputValue);
            }
        } finally {
            IOUtils.closeStream(writer);
        }
    }
}
hadoop jar appendSequenceFile.jar  /HiBench/Aggregation/Input/uservisits/part-00001 /HiBench/Aggregation/Input/uservisits/part-00000
hdfs dfs -rm /HiBench/Aggregation/Input/uservisits/part-00001
hadoop jar appendSequenceFile.jar  /HiBench/Aggregation/Input/uservisits/part-00003 /HiBench/Aggregation/Input/uservisits/part-00002
hdfs dfs -rm /HiBench/Aggregation/Input/uservisits/part-00003
...

你可能感兴趣的:(记一次神奇的 MapReduce OOM)