select asset_inout_ex['asset_inout_1c_sum_1'],dt
from ASSET_INOUT_AMT a
where dt<20181119 and cust_code=0000000
order by dt
limit 10000;
因为原始表中有456 个DT分区,所以DAG中是一个包含456个 HadoopRDD 的 UnionRDD 。 程序启动的时候在Main Thread中会先去分析HadoopRDD 的partitions(split)信息.
这 456 HadoopRDD 在一个PoolSize为8的ForkJoinPool线程池中依次取每个输入文件的getSplits 信息,执行耗时63秒,所以想看看是否有优化空间~~
耗时在3秒内
456 HadoopRDD
引擎 | 生成Task个数 | 执行耗时 | 分析入口 |
---|---|---|---|
Hive | 6426 | 3s | CombineHiveInputFormat.java:getSplits |
Spark | 14269 | 63s | HadoopRDD:getPartitions() |
Hive 使用 CombineHiveInputFormat.java:getSplits
来解析MR Job Input 的Split信息,整个过长只发生一次RPC请求。这次请求最终调用到了FileSystem:listLocatedStatus
,没有发生其他IO请求,耗时较少。
RPC 请求过长
[1277.083729ms] org.apache.hadoop.hive.ql.io.CombineHiveInputFormat:getCombineSplits()
[862.087558ms] org.apache.hadoop.hive.shims.HadoopShims$CombineFileInputFormatShim:getSplits()
[994.181593ms] org.apache.hadoop.mapred.lib.CombineFileInputFormat:getSplits()
+---[1048.585168ms] org.apache.hadoop.mapred.lib.CombineFileInputFormat:getSplits()
| `---[1046.739154ms] org.apache.hadoop.mapred.lib.CombineFileInputFormat:getSplits()
| +---[0.314481ms] org.apache.hadoop.mapreduce.Job:()
| +---[991.715064ms] org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat:getSplits()
| | `---[991.535227ms] org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat:getSplits()
| | +---[615.097799ms] org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat:listStatus()
[680.564369ms] org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileInputFormatShim:listStatus()
`---[680.889335ms] org.apache.hadoop.mapreduce.lib.input.FileInputFormat:singleThreadedListStatus()
+---[min=3.7E-4ms,max=0.002759ms,total=0.003129ms,count=2] java.util.ArrayList:()
+---[min=3.76E-4ms,max=8.3E-4ms,total=0.218522ms,count=456] org.apache.hadoop.mapreduce.JobContext:getConfiguration()
+---[min=0.012602ms,max=0.049432ms,total=8.139451ms,count=456] org.apache.hadoop.fs.Path:getFileSystem()
+---[min=0.149669ms,max=0.70001ms,total=117.35056ms,count=456] org.apache.hadoop.fs.FileSystem:globStatus()
+---[min=3.74E-4ms,max=0.007866ms,total=0.245166ms,count=456] org.apache.hadoop.fs.FileStatus:isDirectory()
+---[min=2.49E-4ms,max=0.002124ms,total=0.181058ms,count=456] org.apache.hadoop.fs.FileStatus:getPath()
+---[min=0.318306ms,max=9.274348ms,total=435.386765ms,count=456] org.apache.hadoop.fs.FileSystem:listLocatedStatus()
+---[min=2.76E-4ms,max=6.723323ms,total=50.527758ms,count=11127] org.apache.hadoop.fs.RemoteIterator:hasNext()
+---[min=2.39E-4ms,max=0.01739ms,total=3.652574ms,count=10671] org.apache.hadoop.fs.RemoteIterator:next()
+---[min=2.24E-4ms,max=0.015722ms,total=3.361117ms,count=10671] org.apache.hadoop.fs.LocatedFileStatus:getPath()
+---[min=2.86E-4ms,max=0.0205ms,total=4.236477ms,count=10671] org.apache.hadoop.fs.PathFilter:accept()
+---[min=2.33E-4ms,max=0.017565ms,total=3.55066ms,count=10671] java.util.List:add()
`---[0.012998ms] java.util.List:isEmpty()
因为Spark采用了UnionRDD来封装下一层的456个HadoopRDD,所以会发生456次OrcInputFormat.getSplits() ,所以Total TimeCost = TimeCost(单次Split RPC) * 456 / 8 ,直接将解析时间放大了456 / 8= 57倍(线性增长,貌似和实际结果一直) 。每次HadoopRDD OrcInputFormat.getSplits() 的耗时分布也非常不均匀,从几ms 到1200+ ms。OrcInputFormat内部启动新线程池解析所有待处理文件,对每个Orc文件都使用orc.ReaderImpl 来读取和解析,Socket 连接的建立和读取数据是非常耗时的,所以如果其中表的某个分区下的文件个数特别多,又是一个线性增长的耗时。
在分析过程中发现一个非常奇怪的问题,CombineHiveInputFormat的OrcGetSplits方法执行耗时非常快,而HadoopRDD:getPartitions 方法执行过慢,差了一个数量级。
19/03/13 15:57:41 ForkJoinPool-2-worker-1 INFO PerfLogger:
19/03/13 15:57:41 ForkJoinPool-2-worker-1 INFO HadoopRDD: HadoopRDD getPartitions start=1552463861363 end=1552463861701 duration=338
在 OrcInputFormat 代码中使用了一个静态变量 perfLogger 来跟踪方法耗时,那么多线程在更新 perfLogger 中的数据的时候,统计的数据就乱套了~~
\\ OrcInputFormat.java
private static final PerfLogger perfLogger = PerfLogger.getPerfLogger();
@Override
public InputSplit[] getSplits(JobConf job,
int numSplits) throws IOException {
perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ORC_GET_SPLITS);
List result = generateSplitsInfo(job);
perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ORC_GET_SPLITS);
return result.toArray(new InputSplit[result.size()]);
}
Main Thread
scala.concurrent.forkjoin.ForkJoinTask.externalAwaitDone() ForkJoinTask.java:295
scala.concurrent.forkjoin.ForkJoinTask.doJoin() ForkJoinTask.java:341
scala.concurrent.forkjoin.ForkJoinTask.join() ForkJoinTask.java:673
...
scala.collection.parallel.immutable.ParVector.map(Function1, CanBuildFrom) ParVector.scala:38
org.apache.spark.rdd.UnionRDD.getPartitions() UnionRDD.scala:84
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply() RDD.scala:253
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply() RDD.scala:251
scala.Option.getOrElse(Function0) Option.scala:121
org.apache.spark.rdd.RDD.partitions() RDD.scala:251
org.apache.spark.rdd.MapPartitionsRDD.getPartitions() MapPartitionsRDD.scala:46
ForkJoinPool-2-worker-7 Waiting CPU usage on sample: 2ms
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(Configuration) OrcInputFormat.java:998
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(JobConf, int) OrcInputFormat.java:1048
org.apache.spark.rdd.HadoopRDD.getPartitions() HadoopRDD.scala:200
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply() RDD.scala:253
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply() RDD.scala:251
...
org.apache.spark.rdd.RDD.partitions() RDD.scala:251
org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(RDD) UnionRDD.scala:84
ORC_GET_SPLITS #4 Waiting in native CPU usage on sample: 63ms
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.call() OrcInputFormat.java:836
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$SplitGenerator.call() OrcInputFormat.java:702
性能分析利器 Arthas
trace org.apache.spark.rdd.HadoopRDD getPartitions
`---ts=2019-03-13 16:51:33;thread_name=ForkJoinPool-2-worker-15;id=93;is_daemon=true;priority=5;TCCL=org.apache.spark.util.MutableURLClassLoader@14bee915
`---[259.42514ms] org.apache.spark.rdd.HadoopRDD:getPartitions()
+---[0.683444ms] org.apache.spark.rdd.HadoopRDD:getJobConf()
+---[0.001664ms] org.apache.spark.deploy.SparkHadoopUtil$:get()
+---[0.028293ms] org.apache.spark.deploy.SparkHadoopUtil:addCredentials()
+---[0.004647ms] org.apache.spark.rdd.HadoopRDD:getInputFormat()
+---[min=9.69E-4ms,max=0.001166ms,total=0.002135ms,count=2] java.lang.System:currentTimeMillis()
+---[258.514402ms] org.apache.hadoop.mapred.InputFormat:getSplits()
+---[0.001906ms] org.apache.spark.rdd.HadoopRDD:ignoreEmptySplits()
+---[min=5.57E-4ms,max=9.38E-4ms,total=0.001495ms,count=2] scala.Predef$:refArrayOps()
+---[min=5.65E-4ms,max=0.00162ms,total=0.002185ms,count=2] scala.collection.mutable.ArrayOps:size()
+---[0.001059ms] scala.Predef$:intWrapper()
+---[0.00496ms] scala.runtime.RichInt$:until$extension0()
+---[0.003234ms] org.apache.spark.rdd.HadoopRDD$$anonfun$getPartitions$1:()
+---[0.005576ms] scala.collection.immutable.Range:foreach$mVc$sp()
+---[0.001439ms] org.apache.spark.rdd.HadoopRDD$$anonfun$getPartitions$2:()
`---[0.046003ms] org.apache.spark.rdd.HadoopRDD:logInfo()
trace org.apache.hadoop.mapred.InputFormat getSplits
`---ts=2019-03-13 16:55:55;thread_name=ForkJoinPool-2-worker-13;id=84;is_daemon=true;priority=5;TCCL=org.apache.spark.util.MutableURLClassLoader@14bee915
`---[184.818819ms] org.apache.hadoop.hive.ql.io.orc.OrcInputFormat:getSplits()
+---[0.020363ms] org.apache.hadoop.hive.ql.log.PerfLogger:PerfLogBegin()
+---[184.665928ms] org.apache.hadoop.hive.ql.io.orc.OrcInputFormat:generateSplitsInfo()
+---[0.025264ms] org.apache.hadoop.hive.ql.log.PerfLogger:PerfLogEnd()
+---[0.001177ms] java.util.List:size()
`---[0.001928ms] java.util.List:toArray()
备注: