beryllw commented on code in PR #1359:
URL: https://github.com/apache/fluss/pull/1359#discussion_r2317672098
##########
fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java:
##########
@@ -61,6 +61,12 @@ public class ConfigOptions {
"ch.qos.logback"
};
+ @Internal
+ public static final String[] PARENT_FIRST_HDFS_PATTERNS =
+ new String[] {
+ "hdfs-site", "core-site", "org.apache.hadoop.", "META-INF",
Review 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:
```yml
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
```
Key error logs:
```plaintext
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:
```yml
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-site
```
Key logs:
```plaintext
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:[email protected]
2025-09-03 11:20:51,177 DEBUG
org.apache.hadoop.security.UserGroupInformation [] - Using user:
"[email protected]" with name [email protected]
2025-09-03 11:20:51,178 DEBUG
org.apache.hadoop.security.UserGroupInformation [] - User entry:
"[email protected]"
2025-09-03 11:20:51,178 DEBUG
org.apache.hadoop.security.UserGroupInformation [] - UGI
loginUser:[email protected] (auth:KERBEROS)
```
The configuration below also fails with the same error as the first failure:
```yml
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: true
```
Key error logs:
```plaintext
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)
```
原因是 org.apache.hadoop.security.UserGroupInformation 类初始化时会使用
ensureInitialized 创建一个空的 Configuration,这个Configuration 会默认加载当前类加载器路径下的
core-site 配置文件。Flink 使用 Hadoop 相关的类加载器也是插件类加载器,但是 Flink 自己实现了一个
org.apache.flink.runtime.security 模块,这个模块里会加载Hadop 环境下的配置文件设置
UserGroupInformation 的 Conf ,可能是因为这个 Flink 可以不用include core-site 。
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`.
```java
//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);
}
}
}
}
```
```java
//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");
}
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]