从源码看Spark读取Hive表数据小文件和分块的问题
前言
有同事问到,Spark读取一张Hive表的数据Task有一万多个,看了Hive表分区下都是3MB~4MB的小文件,每个Task只处理这么小的文件,实在浪费资源浪费时间。而我们都知道Spark的Task数由partitions决定,所以他想通过repartition(num)
的方式来改变分区数,结果发现读取文件的时候Task数并没有改变。遂问我有什么参数可以设置,从而改变读取Hive表时的Task数,将小文件合并大文件读上来
本文涉及源码基于Spark2.0.0和Hadoop2.6.0,不同版本代码可能不一致,需自己对应。此外针对TextInputFormat格式的Hive表,其他格式的比如Parquet有Spark自己的高效实现,不在讨论范围之内
分析
Spark读取Hive表是通过HadoopRDD
扫描上来的,具体可见 org.apache.spark.sql.hive.TableReader
类,构建HadoopRDD的代码如下
val rdd = new HadoopRDD( sparkSession.sparkContext, _broadcastedHadoopConf.asInstanceOf[Broadcast[SerializableConfiguration]], Some(initializeJobConfFunc), inputFormatClass, classOf[Writable], classOf[Writable], _minSplitsPerRDD)
这里inputFormatClass
是Hive创建时指定的,默认不指定为 org.apache.hadoop.mapred.TextInputFormat
,由它就涉及到了HDFS文件的FileSplit数,从而决定了上层Spark的partition数。在进入HadoopRDD类查看之前,还有一个参数需要我们注意,就是 _minSplitsPerRDD
,它在后面SplitSize的计算中是起了作用的。
我们看一下它的定义
private val _minSplitsPerRDD = if (sparkSession.sparkContext.isLocal) { 0 // will splitted based on block by default. } else { math.max(hadoopConf.getInt("mapred.map.tasks", 1), sparkSession.sparkContext.defaultMinPartitions) }
在我们指定以--master local
模式跑的时候,它为0,而在其他模式下,则是求的一个最大值。这里重点看 defaultMinPartitions
,如下
def defaultMinPartitions: Int = math.min(defaultParallelism, 2) // defaultParallelism 在yarn和standalone模式下的计算 def defaultParallelism(): Int = { conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2)) }
从这里可以看到,defaultMinPartitions的值一般为2,而 mapred.map.tasks
或者 mapreduce.job.maps
( 新版参数)是Hadoop的内建参数,其默认值也为2,一般很少去改变它。所以这里_minSplitsPerRDD的值基本就是2了。
下面我们跟到HadoopRDD类里,去看看它的partitions是如何来的
def getPartitions: Array[Partition] = { val jobConf = getJobConf() // add the credentials here as this can be called before SparkContext initialized SparkHadoopUtil.get.addCredentials(jobConf) // inputFormat就是上面参数inputFormatClass所配置的类的实例 val inputFormat = getInputFormat(jobConf) // 此处获取FileSplit数,minPartitions就是上面的_minSplitsPerRDD val inputSplits = inputFormat.getSplits(jobConf, minPartitions) val array = new Array[Partition](inputSplits.size) // 从这里可以看出FileSplit数决定了Spark扫描Hive表的partition数 for (i <- 0 until inputSplits.size) { array(i) = new HadoopPartition(id, i, inputSplits(i)) } array }
在 getPartitions
方法里我们可以看到 FileSplit数最后决定了Spark读取Hive表的Task数,下面我们再来看看 mapred.TextInputFormat
类里 getSplits
的实现
分两步来看,首先是扫描文件,计算文件大小的部分
FileStatus[] files = listStatus(job); ..... long totalSize = 0; // compute total size for (FileStatus file: files) { // check we have valid files if (file.isDirectory()) { throw new IOException("Not a file: "+ file.getPath()); } totalSize += file.getLen(); } // numSplits就是上面传入的minPartitions,为2 long goalSize = totalSize / (numSplits == 0 ? 1 : numSplits); long minSize = Math.max(job.getLong("mapreduce.input.fileinputformat.split.minsize", 1), minSplitSize); // minSplitSize 默认为1,唯一可通过 setMinSplitSize 方法设置 private long minSplitSize = 1;
针对Hive表的分区,Spark对每个分区都构建了一个HadoopRDD,每个分区目录下就是实际的数据文件,例如我们集群的某一张表按天分区,每天下面有200个数据文件,每个文件大概3MB~4MB之间,这些实际上是reduce设置不合理导致的小文件产生,如下图
此处 listStatus
方法就是扫描的分区目录,它返回的就是图中显示的具体 part-*****
文件的FileStatus对象,一共200个。从 totalSize
的计算可以看出,它是这200个文件的总大小,为838MB,因此 goalSize
就为419MB。
参数 mapreduce.input.fileinputformat.split.minsize
在Spark程序没有配的情况下,获取的值为0,而 minSplitSize
在Spark获取FileSplits的时候并没有被设置,所以为默认值1,那么 minSize
就为1
其次,我们再来看从文件划分Split,部分代码如下(部分解释见注释)
ArrayList<FileSplit> splits = new ArrayList<FileSplit>(numSplits); NetworkTopology clusterMap = new NetworkTopology(); // files是上面扫描的分区目录下的part-*****文件 for (FileStatus file: files) { Path path = file.getPath(); long length = file.getLen(); if (length != 0) { FileSystem fs = path.getFileSystem(job); BlockLocation[] blkLocations; if (file instanceof LocatedFileStatus) { blkLocations = ((LocatedFileStatus) file).getBlockLocations(); } else { blkLocations = fs.getFileBlockLocations(file, 0, length); } // 判断文件是否可切割 if (isSplitable(fs, path)) { // 这里获取的不是文件本身的大小,它的大小从上面的length就可以知道,这里获取的是HDFS文件块(跟文件本身没有关系)的大小 // HDFS文件块的大小由两个参数决定,分别是 dfs.block.size 和 fs.local.block.size // 在HDFS集群模式下,由 dfs.block.size 决定,对于Hadoop2.0来说,默认值是128MB // 在HDFS的local模式下,由 fs.local.block.size 决定,默认值是32MB long blockSize = file.getBlockSize(); // 128MB // 这里计算splitSize,根据前面计算的goalSize=419MB,minSize为1 long splitSize = computeSplitSize(goalSize, minSize, blockSize); long bytesRemaining = length; // 如果文件大小大于splitSize,就按照splitSize对它进行分块 // 由此可以看出,这里是为了并行化更好,所以按照splitSize会对文件分的更细,因而split会更多 while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) { String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, length-bytesRemaining, splitSize, clusterMap); splits.add(makeSplit(path, length-bytesRemaining, splitSize, splitHosts[0], splitHosts[1])); bytesRemaining -= splitSize; } if (bytesRemaining != 0) { String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, length - bytesRemaining, bytesRemaining, clusterMap); splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining, splitHosts[0], splitHosts[1])); } } else { String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations,0,length,clusterMap); splits.add(makeSplit(path, 0, length, splitHosts[0], splitHosts[1])); } } else { //Create empty hosts array for zero length files splits.add(makeSplit(path, 0, length, new String[0])); } }
从上面可以看到,splitSize是从 computeSplitSize(goalSize, minSize, blockSize)
计算来的,这三个参数我们都知道大小,那么计算规则是怎么样的呢
规则:Math.max(minSize, Math.min(goalSize, blockSize))
,从而我们可以知道 splitSize = 128MB,对于3MB~4MB的小文件来说,就 决定了一个小文件就是一个split了,从而对应了一个Spark的partition,所以我们一个分区下就有200个partition,当取两个月的数据时,就是 200 * 30 * 2 = 12000,从而是12000个Task,跟同事所说的吻合!
而从TextInputFormat里分Split的逻辑来看,它只会把一个文件分得越来越小,而不会对小文件采取合并,所以无论调整哪个参数,都没法改变这种情况!而通过repartition强行分区,也是在拿到HDFS文件之后对这12000个partition进行重分区,改变不了小文件的问题,也无法改变读取Hive表Task数多的情况
总结
-
Block是物理概念,而Split是逻辑概念,最后数据的分片是根据Split来的。一个文件可能大于BlockSize也可能小于BlockSize,大于它就会被分成多个Block存储到不同的机器上,SplitSize可能大于BlockSize也可能小于BlockSize,SplitSize如果大于BlockSize,那么一个Split就可能要跨多个Block。对于数据分隔符而言,不用担心一个完整的句子分在两个Block里,因为在Split构建RecordReader时,它会被补充完整
-
对于采用
org.apache.hadoop.mapred.TextInputFormat
作为InputFormat的Hive表,如果存在小文件,Spark在读取的时候单凭调参数和repartition是改变不了分区数的!对于小文件的合并,目前除了Hadoop提供的Archive方式之外,也只能通过写MR来手动合了,最好的方式还是写数据的时候自己控制reduce的个数,把握文件数 -
对于Spark直接通过SparkContext的
textFile(inputPath, numPartitions)
方法读取HDFS文件的,它底层也是通过HadoopRDD构建的,它的参数numPartitions就是上面计算goalSize的numSplits参数,这篇 文章 对原理描述的非常详细,非常值得一读 -
对于小文件合并的InputFormat有
org.apache.hadoop.mapred.lib.CombineFileInputFormat
,跟它相关的参数是mapreduce.input.fileinputformat.split.maxsize
,它用于设置一个Split的最大值 -
跟 mapred.TextInputFormat 里的Split划分相关的参数
-
mapreduce.input.fileinputformat.split.minsize
: 决定了计算Split划分时的minSize -
mapreduce.job.maps
或mapred.map.tasks
: 决定了getSplits(JobConf job, int numSplits)方法里的numSplits,从而可以影响goalSize的大小 -
dfs.block.size
或fs.local.block.size
: 决定了HDFS的BlockSize
-
-
MapReduce新版API里的
org.apache.hadoop.mapreduce.lib.input.TextInputFormat
,它的SplitSize与上面说到的计算方式不一样,getSplits方法的签名为getSplits(JobContext job)
,不再有numSplilts这个参数,splitSize的计算规则改为Math.max(minSize, Math.min(maxSize, blockSize))
,minSize和blockSize跟之前一样,新的maxSize为conf.getLong("mapreduce.input.fileinputformat.split.maxsize", Long.MAX_VALUE)
-
在Spark2.0.0里,设置Hadoop相关的参数(比如mapreduce开头的)要通过
spark.sparkContext.hadoopConfiguration
来设置