【Flink系列十七】Flink 最新yarn-application和yarn-per-job部署模式的Classpath傳遞分析
問題
flink-1.13.5 用戶提交Flink作業,連接Hive時發現缺少MRVersion類的定義。
背景說明
- bin/config.sh 內已經確認配置了完整的HADOOP_CLASSPATH變量。
- Flink作業中使用了HiveCatalog。
- Flink部署采用yarn-application方式,因此 main函數的執行是在Yarn的節點上。
NoClassDefFoundError: 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]
at org.apache.flink.connectors.hive.HiveTableMetaStoreFactory$HiveTableMetaStore.<init>(HiveTableMetaStoreFactory.java:72) ~[flink-connector-hive_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.connectors.hive.HiveTableMetaStoreFactory$HiveTableMetaStore.<init>(HiveTableMetaStoreFactory.java:64) ~[flink-connector-hive_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.connectors.hive.HiveTableMetaStoreFactory.createTableMetaStore(HiveTableMetaStoreFactory.java:61) ~[flink-connector-hive_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.connectors.hive.HiveTableMetaStoreFactory.createTableMetaStore(HiveTableMetaStoreFactory.java:43) ~[flink-connector-hive_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.table.filesystem.stream.PartitionCommitter.commitPartitions(PartitionCommitter.java:157) ~[flink-table-blink_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.table.filesystem.stream.PartitionCommitter.processElement(PartitionCommitter.java:143) ~[flink-table-blink_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:205) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:134) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:105) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:66) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:423) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:684) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:639) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:650) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:623) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) ~[flink-dist_2.11-1.13.5.jar:1.13.5]
at java.lang.Thread.run(Thread.java:745) ~[?:1.8.0_121]
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
現象
org.apache.flink.client.cli.CliFrontend 打印的客戶端日志中,缺少 hadoop-mapreduce部分的目錄。
差異:
客戶端提供
[@/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/*
AM端(Application Master,Aka. Job Master)
/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/
唯獨缺少
/opt/cloudera/parcels/CDH/lib/hadoop-mapreduce/
分析
分析下方兩個jar均含有這個類
hadoop-core-2.6.0-mr1-cdh5.12.1.jar
hadoop-mapreduce-client-common-2.6.0-cdh5.12.1.jar
解決辦法很簡單,就是放進lib內,也符合Flink官方文檔。然而筆者并不滿足于這個簡單的解決方案,腦中出現了些許疑問。
疑問
平臺通過客戶端設置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,且非常完整。
不成立。
分析2:是否是軟鏈接目錄所致?
在Flink客戶端的提交日志中,打印出了Classpath,包含了含有和不含有軟鏈接的路徑。而同時AM啟動日志內既沒有hadoop-mapreduce的Jar,也有其他的含有軟鏈接的jar。
不成立。
分析3:是否是因為YARN的節點上缺少CDH的 hadoop-mapreduce有關jar包?
每一個機器都安裝有完整的cloudera的發行版,Classpath完整。
不成立。
分析4:是否是因為hadoop classpath 和 hadoop classpath --glob的差異?
客戶端日志打印了具體的jar路徑,且Classpath非常完整。
不成立。
分析5:是否因為Yarn NodeManager 啟動的時候采用了自身進程的Classpath,而忽略了客戶端的Classpath?
閱讀源碼發現,客戶端的Classpth是由 org.apache.flink.yarn.YarnClusterDescriptor 進行組裝,排序,和上傳的。
并且lib內的jar 一定會被上傳到NodeManager上。
不成立。
分析6:受否因為yarn-site.xml覆蓋了AM的Classpath
以下基本符合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());
結論
根據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的Classpath,但bin/config.sh內的shell變量無法影響,尤其是當使用 yarn-application模式的時候。
此外,classpath的jar順序也很重要,如果在前的,對于同一個JVM Classloader的同一個FQCN類,JVM先加載排在前面的,則不會加載后面的。如果存在相同FQCN的兩個類,有bug的在后面,則前面的掩蓋后面的BUG。如果第一個是有bug的,則不幸中招。

浙公網安備 33010602011771號