Hadoop Distributed Cache Deploy

Hadoop: Distributed Cache Deploy

1. 简介
2. 如何配置?
3. 原理分析
4. 问题纪录

1. 简介

Hadoop支持通过分布式缓存的方式来部署不同版本的MapReduce框架。通过此方案,用户可以很方便的在Yarn上运行不同版本的MR任务。如果基于现有MR框架进行定制开发(添加新feature、修复bug等),新功能上线会比较麻烦。而分布式缓存则为该问题提供了一个很好的解决方案。

2. 如何配置?

实现该方案主要有三个步骤:

  1. 将新版的jar文件打包,上传到HDFS集群中。
  2. 配置mapreduce.application.framework.path,指向步骤1中的文件所在的路径,同时支持为路径指定一个别名。如:hdfs:///data/hadoop/mr/hadoop-mapreduce275.0.1.tar.gz#mr-opt。
  3. 配置mapreduce.application.classpath,根据2中的信息合理地设置classpath。
# mapred-site.xml中新增配置项
    <property>
        <name>mapreduce.application.framework.path</name>
        <value>hdfs:///data/hadoop/mr/hadoop-mapreduce275.0.1.tar.gz#mr-opt</value>
    </property>
    
    <property>
        <name>mapreduce.application.classpath</name>
        <value>$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*,$PWD/mr-opt/*,$PWD/mr-opt/lib/*</value>
    </property>

3. 原理分析

Container启动时,会根据mapreduce.application.classpath的内容来设定CLASSPATH,从而替换默认的CLASSPATH。Yarn中Container启动流程大致如下:NodeManager接收到Container启动请求后,会触发Container状态从NEW转变到LOCALIZING。这个过程就是资源本地化(比如上面提到的mapreduce.application.framework.path指定的tar包资源),主要工作由ResourceLocalizationService负责。NodeManager会根据资源的"可见性"分别将其下载到不同的目录中,供用户提交job使用。资源会被按照"可见性"分为三类,分别是:

可见性 说明 本地目录
PUBLIC 该NodeManager上所有用户提交的APP都能访问。 ${yarn.nodemanager.local-dirs}/filecache
PRIVATE 同一个用户提交的APP都可以访问。 {yarn.nodemanager.local-dirs}/usercache/\{user}/filecache
APPLICATION 同一个APP下的Container都可以访问。 ${yarn.nodemanager.local-dirs}/usercache/${user}/appcache/${appId}/filecache

NodeManager中是由LocalResourcesTracker类负责维护Resource的生命周期,(download、remove、recover等)。下面是ResourceLocalizationService中相关code:

# 负责PUBLIC级别的Resource
private LocalResourcesTracker publicRsrc;
# 负责PRIVATE级别的Resource
private final ConcurrentMap<String/*username*/,LocalResourcesTracker> privateRsrc =
    new ConcurrentHashMap<String,LocalResourcesTracker>();
# 负责APPLICATION级别的Resource
private final ConcurrentMap<String/*appid*/,LocalResourcesTracker> appRsrc =
    new ConcurrentHashMap<String,LocalResourcesTracker>();

对于PRIVATE或APPLICATION级别的Resource,会为每个username或appid维护一个LocalResourcesTracker。这主要考虑到不同可见性的Resource对于并发性要求不同。由于mapreduce.application.framework.path指定的资源可见性是PUBLIC,即:所有用户提交到该节点的任务都可以访问。所以最终会被下载到${yarn.nodemanager.local-dirs}/filecache文件夹中。

对于PUBLIC的资源,可见性范围越大,意味着潜在的访问者越多。其副本数应尽量设置大一些,避免任务在LOCALIZING时下载资源带来额外竞争,造成JOB启动性能损耗。可以参考:mapreduce.client.submit.file.replication的设置。

4. 问题纪录

利用hadoop自带的wordcount样例进行测试。遇到以下几个问题。

问题1

提交任务
# 通过-D指定mapreduce.application.classpath
hadoop jar /usr/local/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.5.jar wordcount  -Dmapreduce.application.classpath=$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*,$PWD/hadoop-mapreduce275.0.1.tar.gz/*,
$PWD/hadoop-mapreduce275.0.1.tar.gz/lib/* /tmp/input /tmp/output
异常纪录
18/11/06 20:04:43 INFO mapreduce.JobSubmitter: number of splits:4
18/11/06 20:04:43 INFO mapreduce.JobSubmitter: Submitting tokens for job: job_1540972651914_0011
18/11/06 20:04:44 INFO mapreduce.JobSubmitter: Cleaning up the staging area /staging/hadoop/.staging/job_1540972651914_0011
java.lang.IllegalArgumentException: Could not locate MapReduce framework name 'mr-opt' in mapreduce.application.classpath
  at org.apache.hadoop.mapreduce.v2.util.MRApps.setMRFrameworkClasspath(MRApps.java:231)
  at org.apache.hadoop.mapreduce.v2.util.MRApps.setClasspath(MRApps.java:258)
  at org.apache.hadoop.mapred.YARNRunner.createApplicationSubmissionContext(YARNRunner.java:468)
  at org.apache.hadoop.mapred.YARNRunner.submitJob(YARNRunner.java:296)
  at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:244)
  at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1290)
  at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1287)
  • 异常分析:如果mapreduce.application.framework.path中指定了path的别名,那么mapreduce.application.classpath中必须通过别名来引用jar包。
  • 解决方案:将$PWD/hadoop-mapreduce275.0.1.tar.gz中的hadoop-mapreduce275.0.1.tar.gz改为其别名即可。即:$PWD/mr-opt/

问题2

提交任务
hadoop jar /usr/local/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.5.jar wordcount
-Dmapreduce.application.classpath=$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*,$PWD/mr-opt/*,$PWD/mr-opt/lib/* /tmp/input /tmp/output
异常纪录
18/11/07 15:28:50 INFO mapreduce.Job: Job job_1540972651914_0042 failed with state FAILED due to: Application application_1540972651914_0042 failed 4 times due to AM Container for appattempt_1540972651914_0042_000004 exited with  exitCode: -1000
For more detailed output, check application tracking page:http://yq01-sw-backup-hds01.yq01.baidu.com:8088/cluster/app/application_1540972651914_0042Then, click on links to logs of each attempt.
Diagnostics: Permission denied: user=work, access=READ, inode="/data/hadoop/mr/hadoop-mapreduce275.0.1.tar.gz":hadoop:hadoop:-rwx------
    at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:308)
    at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:220)
    at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:190)
    at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPermission(FSDirectory.java:1808)
    at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPermission(FSDirectory.java:1792)
    at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPathAccess(FSDirectory.java:1765)
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1844)
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1814)
    at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1729)
    at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:588)
  • 异常分析:这很明显是权限问题,上传的tar包必须能够让所有用户都能访问。
  • 解决方案:将hadoop-mapreduce275.0.1.tar.gz的acl修改为755即可。

问题3

提交任务
hadoop jar /usr/local/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.5.jar wordcount
-Dmapreduce.application.classpath=$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*,$PWD/mr-opt/*,$PWD/mr-opt/lib/* /tmp/input /tmp/output
异常纪录
18/11/06 20:06:09 INFO mapreduce.Job: Running job: job_1540972651914_0012
18/11/06 20:06:14 INFO mapreduce.Job: Job job_1540972651914_0012 running in uber mode : false
18/11/06 20:06:14 INFO mapreduce.Job:  map 0% reduce 0%
18/11/06 20:06:14 INFO mapreduce.Job: Job job_1540972651914_0012 failed with state FAILED due to: Application application_1540972651914_0012 failed 4 times due to AM Container for appattempt_1540972651914_0012_000004 exited with  exitCode: -1000
For more detailed output, check application tracking page:http://yq01-sw-backup-hds01.yq01.baidu.com:8088/cluster/app/application_1540972651914_0012Then, click on links to logs of each attempt.
Diagnostics: ExitCodeException exitCode=2:
gzip: /home/disk0/yarn/local/filecache/10_tmp/tmp_hadoop-mapreduce275.0.1.tar.gz: not in gzip format
tar: This does not look like a tar archive
tar: Exiting with failure status due to previous errors
Failing this attempt. Failing the application.
18/11/06 20:06:14 INFO mapreduce.Job: Counters: 0
  • 异常分析:打包过程语法由于,使用了tar -cvf打成了tar包。
  • 解决方案:通过tar -zcvf打包。

问题4

提交任务
hadoop jar /usr/local/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.5.jar wordcount
-Dmapreduce.application.classpath=$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*,$PWD/mr-opt/*,$PWD/mr-opt/lib/* /tmp/input /tmp/output
异常纪录
Log Type: stderr
Log Upload Time: Tue Nov 06 20:09:02 +0800 2018
Log Length: 88
Error: Could not find or load main class org.apache.hadoop.mapreduce.v2.app.MRAppMaster
  • 异常分析:AM重试2次都失败,日志显示没有加载到MRAppMaster导致。

debug过程

  1. 开启NodeManager的debug:yarn.nodemanager.delete.debug-delay-sec设置长点时间,如:3600。
  2. 查看AM的启动脚本,位于AM所在NodeManager服务器的${yarn.nodemanager.local-dirs}/nmPrivate/${applicationId}/${containerID}/launch_container.sh,其中CLASSPATH信息如下:
# launch_container.sh部分内容
export PWD="/home/disk0/yarn/local/usercache/hadoop/appcache/application_1540972651914_0043/container_e27_1540972651914_0043_03_000001"
。。。
export CLASSPATH="$PWD:$HADOOP_CONF_DIR:$HADOOP_COMMON_HOME/share/hadoop/common/*:
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*:$HADOOP_HDFS_HOME/share/hadoop/hdfs/*:
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*:$HADOOP_YARN_HOME/share/hadoop/yarn/*:
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*:/home/hadoop/mr-opt/*:
/home/hadoop/mr-opt/lib/*:job.jar/job.jar:job.jar/classes/:job.jar/lib/*:$PWD/*"
。。。
ln -sf "/home/disk0/yarn/local/filecache/11/hadoop-mapreduce275.0.1.tar.gz" "mr-opt"

可以看到MR相关的classpath为:/home/hadoop/mr-opt/:/home/hadoop/mr-opt/lib/。这显然是不对的。理论上应该是:$PWD/mr-opt/:$PWD/mr-opt/lib/。最终查明原因是提交任务的是$PWD被转义成当前目录(提交任务时的目录),而我提交任务的目录就是:/home/hadoop。

  • 解决方案:不通过-D来指定mapreduce.application.classpath,直接将该项配置在mapred-site.xml中即可。
# 下面是正确的CLASSPATH
export PWD="/home/disk0/yarn/local/usercache/hadoop/appcache/application_1540972651914_0043/container_e27_1540972651914_0043_03_000001"

export CLASSPATH="$PWD:$HADOOP_CONF_DIR:$HADOOP_COMMON_HOME/share/hadoop/common/*:
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*:$HADOOP_HDFS_HOME/share/hadoop/hdfs/*:
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*:$HADOOP_YARN_HOME/share/hadoop/yarn/*:
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*:$PWD/mr-opt/*:
$PWD/mr-opt/lib/*:job.jar/job.jar:job.jar/classes/:job.jar/lib/*:$PWD/*"

ln -sf "/home/disk0/yarn/local/filecache/11/hadoop-mapreduce275.0.1.tar.gz" "mr-opt"

参考

發表評論
所有評論
還沒有人評論,想成為第一個評論的人麼? 請在上方評論欄輸入並且點擊發布.
相關文章