[Filesystem] Support use Hadoop dependencies from environment variables HADOOP_CLASSPATH#1359
[Filesystem] Support use Hadoop dependencies from environment variables HADOOP_CLASSPATH#1359wuchong merged 13 commits intoapache:mainfrom
Conversation
| @Internal | ||
| public static final String[] PARENT_FIRST_HDFS_PATTERNS = | ||
| new String[] { | ||
| "hdfs-site", "core-site", "org.apache.hadoop.", "META-INF", |
There was a problem hiding this comment.
As discussed offline, seems hdfs-site, core-site, META-INF is a must to configure? But I'm still wondering why it don't work if not configured thems. At begining, I think it may cause it load the hdfs-site.xml in plugin/hdfs/fluss-fs-hadoop-0.8-SNAPSHOT.jar, but I unzip plugin/hdfs/fluss-fs-hadoop-0.8-SNAPSHOT.jar, can't find any hdfs-site.xml file, a little of confused..
There was a problem hiding this comment.
As discussed offline, seems
hdfs-site,core-site,META-INFis a must to configure?
hdfs-site and core-site are used to load machine hdfs environment conf. When users work with an existing Hadoop environment on their machines, they typically want to use their own HDFS configurations.
Alternatively, configurations can be set via the fluss.hadoop.* prefix, as detailed in fluss doc.
META-INF is used to load related classes via SPI.
There was a problem hiding this comment.
Thanks for the explanation. But if user already set HADOOP_HOME, won't hdfs-site and core-site in HADOOP_HOME be loaded? See HadoopUtils#getHadoopConfiguration, or some wrong in here?
META-INF is for loading some classes that hadoop required?
There was a problem hiding this comment.
But if user already set HADOOP_HOME, won't hdfs-site and core-site in HADOOP_HOME be loaded?
Yes, but for plugin ClassLoader, without classloader.parent-first-patterns.default it cannot retrieve the hdfs-site resource.
The same problem exists for META-INF.
org.apache.hadoop. only for hadoop class resource, make plugin ClassLoader can retrieve hadoop class but not hdfs-site and META-INF resource.
There was a problem hiding this comment.
I'm also curious about this. Why Flink doesn't need to include hdfs-site, core-site, META-INF into parent first pattern? Is there any specifc plugin that requires to read the hdfs-site and META-INF from the classloader by the plugin itself?
There was a problem hiding this comment.
After re-testing, I found that these three files are not required when Kerberos is not enabled, but the core-site file is needed when Kerberos authentication is enabled.
Failed test case:
remote.data.dir: hdfs://xxxxxx/rbf/data/fluss
datalake.format: paimon
datalake.paimon.metastore: filesystem
datalake.paimon.warehouse: hdfs://xxxxxx/rbf/warehouse
plugin.classloader.parent-first-patterns.default: java.,com.alibaba.fluss.,javax.annotation.,org.slf4j,org.apache.log4j,org.apache.logging,org.apache.commons.logging,ch.qos.logback,org.apache.hadoopKey error logs:
2025-09-03 10:24:26,304 DEBUG org.apache.hadoop.security.UserGroupInformation [] - hadoop login
2025-09-03 10:24:40,781 DEBUG org.apache.hadoop.security.UserGroupInformation [] - hadoop login commit
2025-09-03 10:24:40,782 DEBUG org.apache.hadoop.security.UserGroupInformation [] - Using user: "xxxxx" with name xxxxx
2025-09-03 10:24:40,782 DEBUG org.apache.hadoop.security.UserGroupInformation [] - User entry: "xxxxx"
2025-09-03 10:24:40,782 DEBUG org.apache.hadoop.security.UserGroupInformation [] - UGI loginUser:xxxxx (auth:SIMPLE)
Successful test case:
remote.data.dir: hdfs://xxxxxx/rbf/data/fluss
datalake.format: paimon
datalake.paimon.metastore: filesystem
datalake.paimon.warehouse: hdfs://xxxxxx/rbf/warehouse
plugin.classloader.parent-first-patterns.default: java.,com.alibaba.fluss.,javax.annotation.,org.slf4j,org.apache.log4j,org.apache.logging,org.apache.commons.logging,ch.qos.logback,org.apache.hadoop,core-siteKey logs:
2025-09-03 11:20:51,175 DEBUG org.apache.hadoop.security.UserGroupInformation [] - hadoop login
2025-09-03 11:20:51,177 DEBUG org.apache.hadoop.security.UserGroupInformation [] - hadoop login commit
2025-09-03 11:20:51,177 DEBUG org.apache.hadoop.security.UserGroupInformation [] - using kerberos user:xxxxx@HADOOP.XXXX.COM
2025-09-03 11:20:51,177 DEBUG org.apache.hadoop.security.UserGroupInformation [] - Using user: "xxxxx@HADOOP.XXXX.COM" with name xxxxx@HADOOP.XXXX.COM
2025-09-03 11:20:51,178 DEBUG org.apache.hadoop.security.UserGroupInformation [] - User entry: "xxxxx@HADOOP.XXXX.COM"
2025-09-03 11:20:51,178 DEBUG org.apache.hadoop.security.UserGroupInformation [] - UGI loginUser:xxxxx@HADOOP.XXXX.COM (auth:KERBEROS)
The configuration below also fails with the same error as the first failure:
remote.data.dir: hdfs://xxxxxx/rbf/data/fluss
datalake.format: paimon
datalake.paimon.metastore: filesystem
datalake.paimon.warehouse: hdfs://xxxxxx/rbf/warehouse
plugin.classloader.parent-first-patterns.default: java.,com.alibaba.fluss.,javax.annotation.,org.slf4j,org.apache.log4j,org.apache.logging,org.apache.commons.logging,ch.qos.logback,org.apache.hadoop
fluss.hadoop.hadoop.security.authentication: kerberos
fluss.hadoop.hadoop.security.authorization: trueKey error logs:
2025-09-03 11:24:15,457 DEBUG org.apache.hadoop.security.UserGroupInformation [] - hadoop login
2025-09-03 11:24:15,457 DEBUG org.apache.hadoop.security.UserGroupInformation [] - hadoop login commit
2025-09-03 11:24:15,458 DEBUG org.apache.hadoop.security.UserGroupInformation [] - Using user: "xxxxx" with name xxxxx
2025-09-03 11:24:15,458 DEBUG org.apache.hadoop.security.UserGroupInformation [] - User entry: "xxxxx"
2025-09-03 11:24:15,458 DEBUG org.apache.hadoop.security.UserGroupInformation [] - UGI loginUser:xxxxx (auth:SIMPLE)
The reason is that when initializing the org.apache.hadoop.security.UserGroupInformation class, it uses ensureInitialized to create an empty Configuration, which by default loads the core-site configuration file in the current class loader path.
Flink uses a plugin class loader for Hadoop-related classes, but Flink has its own org.apache.flink.runtime.security module that loads configuration files from the Hadoop environment to set up UserGroupInformation's Conf. This might be why Flink can work without including core-site.
//org.apache.hadoop.security.UserGroupInformation#ensureInitialized
private static void ensureInitialized() {
if (!isInitialized()) {
synchronized(UserGroupInformation.class) {
if (!isInitialized()) { // someone might have beat us
initialize(new Configuration(), false);
}
}
}
}//org.apache.hadoop.conf.Configuration#static
static {
// Add default resources
addDefaultResource("core-default.xml");
addDefaultResource("core-site.xml");
// print deprecation warning if hadoop-site.xml is found in classpath
ClassLoader cL = Thread.currentThread().getContextClassLoader();
if (cL == null) {
cL = Configuration.class.getClassLoader();
}
if (cL.getResource("hadoop-site.xml") != null) {
LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " +
"Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, "
+ "mapred-site.xml and hdfs-site.xml to override properties of " +
"core-default.xml, mapred-default.xml and hdfs-default.xml " +
"respectively");
addDefaultResource("hadoop-site.xml");
}
}There was a problem hiding this comment.
@beryllw Thanks for the great work!! Let's me have a summary, please correct me if I'm wrong.
So, It turns out to be:
option1:
Only add org.apache.hadoop. to parent first pattern in Fluss code, works for when Kerberos is not enabled. When Kerberos is enabled, users must add core-site.xml to parent first pattern manually.
option2: Add org.apache.hadoop., core-site.xml to parent first pattern in Fluss code, works whatever Kerberos is enabled or not. But after we introduce security module, we can also remove core-site.xml to parent first pattern.
I perfer option2 since it require users do nothing in Kerberos enabled env. And I do meet some users are in Kerberos enabled env. @wuchong What do you think of it?
There was a problem hiding this comment.
Option 3: Adding the configuration fluss.hadoop.hadoop.security.authentication does not work. The root cause is that when initializing the UserGroupInformation class, it uses an empty Configuration instance instead of our Fluss-specific Hadoop configuration.
I noticed that Flink includes a security module which is responsible for setting the configuration of UserGroupInformation — this might be relevant to our issue. For reference, see the following link:
https://github.com/apache/flink/blob/78136133fbec4ca145dec66d4bc0c324c8e16d82/flink-runtime/src/main/java/org/apache/flink/runtime/security/modules/HadoopModule.java#L68-L70
It seems the failure is related to the shadedPattern. Could we retrigger the test? I can't figure out the issue. |
|
Why we have to shade the |
Same problem in |
|
Shade the The conflict arises because both versions are present in the classpath: Another dependency conflict: |
d88c76c to
434a4f4
Compare
|
@beryllw Test fail due to the shade of |
|
…es HADOOP_CLASSPATH
|
LGTM! Thanks for helping improve the doc! |
…es HADOOP_CLASSPATH (apache#1359)
Purpose
Linked issue: close #1358
Brief change log
Tests
API and Format
Documentation
Enable HDFS and Paimon plugins to utilize the host machine's Hadoop environment, and document the process for configuring Paimon with a Hive catalog.