概述
文章主要内容是提出并解决了一个 Flink 作业启动报错找不到类的问题 。并且借此机会从源码层面分析了 Flink on Yarn 的 Classpath 传递机制,该机制指明了 Flink 是怎样构建 ApplicationMaster 和 TaskManager 的 Classpath,同时说明了 Classpath 在客户端机器和Yarn Container上的差异。通过分析源码,发现 JM 和 TM 环境变量是由 YarnClusterDescriptor 构建的,并最终找到 Classpath 缺失的根本性解决方法。
注:JobManager 对于Yarn来说是 ApplicationMaster,ApplicationMaster 和 TaskManager 对于 Yarn来说都是 Container
关键词
Flink,Flink On Yarn,Yarn-Per-Job ,NoClassDefFoundError, org/apache/hadoop/mapred/MRVersion, Job Manager,TaskManager,Container, YarnClusterDescriptor
遇到的问题
用户 Flink 作业启动时发现缺少 MRVersion 类的定义。
如何发生的
- 提交 Flink 作业,该作业连接了 Hive。
- HiveConf 初始化过程中找不到类 org.apache.hadoop.mapred.MRVersion。
java.lang.NoClassDefFoundError: org/apache/hadoop/mapred/MRVersion
at org.apache.hadoop.hive.shims.Hadoop23Shims.isMR2(Hadoop23Shims.java:932) ~[hive-exec-1.1.0-cdh5.12.1-slankka.jar:1.1.0-cdh5.12.1]
at org.apache.hadoop.hive.shims.Hadoop23Shims.getHadoopConfNames(Hadoop23Shims.java:1003) ~[hive-exec-1.1.0-cdh5.12.1-slankka.jar:1.1.0-cdh5.12.1]
at org.apache.hadoop.hive.conf.HiveConf$ConfVars.<clinit>(HiveConf.java:370) ~[hive-exec-1.1.0-cdh5.12.1-slankka.jar:1.1.0-cdh5.12.1]
at org.apache.hadoop.hive.conf.HiveConf.<clinit>(HiveConf.java:108) ~[hive-exec-1.1.0-cdh5.12.1-slankka.jar:1.1.0-cdh5.12.1]
at org.apache.flink.connectors.hive.util.HiveConfUtils.create(HiveConfUtils.java:38) ~[flink-connector-hive_2.11-1.13.5.jar:1.13.5]
...
Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.mapred.MRVersion
at java.net.URLClassLoader.findClass(URLClassLoader.java:381) ~[?:1.8.0_121]
at java.lang.ClassLoader.loadClass(ClassLoader.java:424) ~[?:1.8.0_121]
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331) ~[?:1.8.0_121]
at java.lang.ClassLoader.loadClass(ClassLoader.java:357) ~[?:1.8.0_121]
... 24 more
解决
通过查找,下方两个jar均含有这个类
- hadoop-core-2.6.0-mr1-cdh5.12.1.jar
- hadoop-mapreduce-client-common-2.6.0-cdh5.12.1.jar
注:后者所在路径是 /opt/cloudera/parcels/CDH/lib/hadoop-mapreduce
解决办法
很简单,就是放进 Flink 的 lib 内,也符合 Flink 官方文档。
然而笔者并不满足于这个简单的解决方案,脑中出现了些许疑问。
除了按照官方文档放置缺少的 jar 以外,能否找到缺失 library 的根本原因,并最终得到根本解决方法?
现象
找问题原因的过程中,少不了一番排查。
- A. 观察客户端机器的 hadoop classpath。
- B. 观察启动过程中客户端打印的日志。
- C. 提交作业以后,在 AM 中观察启动过程中打印的 Classpath。
A. 客户端机器的 classpath
通过检查客户端的环境变量,确认客户端提供的环境变量 HADOOP_CLASSPATH 内容如下:
[@/opt/cloudera/parcels/GPLEXTRAS/lib]# hadoop classpath | tr ':' '\n'
/etc/hadoop/conf
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop/lib/*
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop/.//*
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop-hdfs/./
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop-hdfs/lib/*
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop-hdfs/.//*
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop-yarn/lib/*
/opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/libexec/../../hadoop-yarn/.//*
/opt/cloudera/parcels/CDH/lib/hadoop-mapreduce/lib/*
/opt/cloudera/parcels/CDH/lib/hadoop-mapreduce/.//*
/opt/cloudera/parcels/GPLEXTRAS-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/lib/*
确定没有缺少 /opt/cloudera/parcels/CDH/lib/hadoop-mapreduce/
B. 客户端打印的 classpath
org.apache.flink.client.cli.CliFrontend 打印的日志,没有缺少 hadoop-mapreduce部分的目录。
C. AM打印的 classpath
通过观察应用程序启动时,Application Master 端(注:JobManager 启动时打印的日志片段)
注:也适用于 Yarn-Application模式部署
- /opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/
- /opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop/lib/
- /opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop-hdfs/
- /opt/cloudera/parcels/CDH-5.12.1-1.cdh5.12.1.p0.3/lib/hadoop-yarn/
- /opt/cloudera/parcels/GPLEXTRAS/lib/hadoop/lib/
对比初始的 hadoop classpath,发现缺少以下的条目
- /opt/cloudera/parcels/CDH/lib/hadoop-mapreduce/
- /opt/cloudera/parcels/CDH/lib/hadoop-mapreduce/lib/
疑问
平台通过客户端设置 HADOOP_CLASSPATH 了:
- 在bin/config.sh设置了
INTERNAL_HADOOP_CLASSPATH=(`hadoop classpath`)
然而,Flink 提交到 Yarn 后仍然出现问题。具体现象是:
- ApplicationMaster 启动时打印的 Classpath 不包含平台的 /opt/cloudera/parcels/CDH/lib/hadoop-mapreduce。
分析
分析1:是否是客户端环境所致?
在Flink客户端的提交日志中,配置日志级别,org.apache.flink.client.cli.CliFrontend 打印出了Classpath,且非常完整,和 hadoop classpath --glob 打印的条目一致。
不成立。
分析2:是否是软链接目录所致?
Flink客户端打印的日志中,打印出的 Classpath ,包含了含有和不含有软链接的路径,且包含 hadoop-mapreduce 的 Jar。
AM启动日志内没有 hadoop-mapreduce 的 Jar,但有同时含有软链接形式的 jar。
不成立。
分析3:是否是因为YARN的节点上缺少CDH的 hadoop-mapreduce有关jar包?
每一个机器都安装有完整的cloudera的发行版,Classpath完整。
不成立。
分析4:是否是因为hadoop classpath 和 hadoop classpath --glob的差异?
客户端日志本身打印了具体的jar路径,且Classpath非常完整,且包含 hadoop-mapreduce 的Jar。
不成立。
分析5:是否因为 Yarn 的 Container 启动的时候采用了自身进程的Classpath,而忽略了客户端的Classpath?
阅读源码发现,客户端的Classpth是由 org.apache.flink.yarn.YarnClusterDescriptor 进行组装,排序,和上传的。
并且lib内的jar 一定会被上传到NodeManager上。这样看来,怀疑 Yarn 的 Container 是对 Yarn 有点低估了。
不成立。
分析6:是否因为 yarn-site.xml 覆盖了 AM 的 Classpath
观察发现,以下客户端所在机器的 yarn-site.xml 配置完全符合 AM 启动过程中 Classpath 的打印的内容,且该配置同样缺少 hadoop-mapreduce的jar包,现象和配置完全吻合。
<property>
<name>yarn.application.classpath</name>
<value>$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*,/opt/cloudera/parcels/GPLEXTRAS/lib/hadoop/lib/*</value>
</property>
结论成立。
论证依据
org.apache.flink.yarn.YarnClusterDescriptor.java based on Flink-1.13.5
org.apache.flink.yarn.Utils.java based on Flink-1.13.5
public static void setupYarnClassPath(Configuration conf, Map<String, String> appMasterEnv) {
addToEnvironment(
appMasterEnv, Environment.CLASSPATH.name(), appMasterEnv.get(ENV_FLINK_CLASSPATH));
String[] applicationClassPathEntries =
conf.getStrings(
YarnConfiguration.YARN_APPLICATION_CLASSPATH,
YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH);
for (String c : applicationClassPathEntries) {
addToEnvironment(appMasterEnv, Environment.CLASSPATH.name(), c.trim());
}
}
Flink 将自身的 lib 、plugin 、用户 jar 等依赖加入 ENV_FLINK_CLASSPATH ,作为 Container 的一部分,紧接着将 yarn.application.classpath 放入 Yarn 应用的 Classpath 。
反思
Classpath 是排序的吗?
是的,Flink 对用户 Classpath 和 System 的 Classpath 分别进行排序,默认按照 ORDER 策略,根据 jar 名称进行排序。
// normalize classpath by sorting
Collections.sort(systemClassPaths);
Collections.sort(userClassPaths);
// classpath assembler
StringBuilder classPathBuilder = new StringBuilder();
if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) {
for (String userClassPath : userClassPaths) {
classPathBuilder.append(userClassPath).append(File.pathSeparator);
}
}
for (String classPath : systemClassPaths) {
classPathBuilder.append(classPath).append(File.pathSeparator);
}
...
...
// set Flink app class path
appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString());
注: 参见YarnConfigOptions.UserJarInclusion的文档1这个选项在后续的版本中改名。
结论
根据 Flink 官方文档描述,向 Flink 提供 Hadoop classpath 应当使用 export HADOOP_CLASSPATH,并在每一个节点上配置,其次是在 lib 中提供。
进一步讲:
-
Flink 自身管理了 lib 和 user 的 jar,这无疑会影响 Container 的 classpath,但与此同时,默认读取
yarn-site.xml的yarn.application.classpath,并不会读取环境变量HADOOP_CLASSPATH 作为 AM 的 Classpath的一部分,因此出现上述不一致性。 -
lib 可以影响 Flink On Yarn 的 Classpath,但 bin/config.sh 内的shell变量无法影响。
-
此外,classpath 的 jar 顺序也很重要,如果在前的,对于同一个 JVM Classloader 的相同 FQCN 类,JVM 先加载排在前面的,则不会加载后面的。如果存在相同 FQCN 的两份字节码,有 bug 的在后面,则前面的掩盖后面的BUG。如果第一份是有 bug 的,则不幸中招。
补充
类加载的机制只是理论,有关类加载的工程方面,还涉及Maven打包插件的细节,这里分享一篇深度文章:《安全同学讲Maven重打包的故事》2 可通过过程细节加深对类加载顺序和类冲突的理解。
参考
Footnotes
-
yarn.per-job-cluster.include-user-jar nightlies.apache.org/flink/flink… ↩
-
安全同学讲Maven重打包的故事 zhuanlan.zhihu.com/p/610374858… ↩