This is an automated email from the ASF dual-hosted git repository.

JingsongLi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new 1d0f005f78 [core] Preserve external UGI when paimon has no kerberos 
credentials (#7862)
1d0f005f78 is described below

commit 1d0f005f784efee10ba4e21b964b402dda39b270
Author: Kerwin Zhang <[email protected]>
AuthorDate: Tue May 19 17:37:46 2026 +0800

    [core] Preserve external UGI when paimon has no kerberos credentials (#7862)
---
 .../paimon/fs/hadoop/HadoopSecuredFileSystem.java  |  9 ++++++++
 .../fs/hadoop/HadoopSecuredFileSystemTest.java     | 24 ++++++++++++++++++++++
 2 files changed, 33 insertions(+)

diff --git 
a/paimon-common/src/main/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystem.java
 
b/paimon-common/src/main/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystem.java
index d229579ce5..cbfca1b6d9 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystem.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystem.java
@@ -21,6 +21,7 @@ package org.apache.paimon.fs.hadoop;
 import org.apache.paimon.options.Options;
 import org.apache.paimon.security.HadoopModule;
 import org.apache.paimon.security.SecurityConfiguration;
+import org.apache.paimon.utils.StringUtils;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -200,6 +201,14 @@ public class HadoopSecuredFileSystem extends FileSystem {
             throws IOException {
         SecurityConfiguration config = new SecurityConfiguration(options);
         if (config.isLegal()) {
+            if (StringUtils.isNullOrWhitespaceOnly(config.getKeytab())
+                    && 
StringUtils.isNullOrWhitespaceOnly(config.getPrincipal())) {
+                LOG.info(
+                        "No paimon Kerberos credentials configured "
+                                + "(security.kerberos.login.keytab/principal); 
"
+                                + "skip HadoopModule.install() to preserve 
externally-established UGI.");
+                return fileSystem;
+            }
             LOG.info("Hadoop security configuration is legal, use the secured 
FileSystem.");
             HadoopModule module = new HadoopModule(config, configuration);
             module.install();
diff --git 
a/paimon-common/src/test/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystemTest.java
 
b/paimon-common/src/test/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystemTest.java
index 10d60f2bc4..8a14c7d264 100644
--- 
a/paimon-common/src/test/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystemTest.java
+++ 
b/paimon-common/src/test/java/org/apache/paimon/fs/hadoop/HadoopSecuredFileSystemTest.java
@@ -47,4 +47,28 @@ public class HadoopSecuredFileSystemTest {
         assertThat(fileIO.getFileSystem(new 
org.apache.hadoop.fs.Path("file:///tmp/test")))
                 .isInstanceOf(HadoopSecuredFileSystem.class);
     }
+
+    @Test
+    public void testPreserveExternalUgiWhenNoKerberosCredentials() throws 
Exception {
+        Options options = new Options();
+
+        HadoopFileIO fileIO = new HadoopFileIO(new Path("file:///tmp/test"));
+        fileIO.configure(CatalogContext.create(options));
+        assertThat(fileIO.getFileSystem(new 
org.apache.hadoop.fs.Path("file:///tmp/test")))
+                .isNotInstanceOf(HadoopSecuredFileSystem.class);
+    }
+
+    @Test
+    public void testReturnOriginalFileSystemWhenSecurityConfigIsIllegal() 
throws Exception {
+        File keytabFile = new File(tmp.toFile(), "test-keytab.keytab");
+        assertThat(keytabFile.createNewFile()).isTrue();
+
+        Options options = new Options();
+        options.set("security.kerberos.login.keytab", 
keytabFile.getAbsolutePath());
+
+        HadoopFileIO fileIO = new HadoopFileIO(new Path("file:///tmp/test"));
+        fileIO.configure(CatalogContext.create(options));
+        assertThat(fileIO.getFileSystem(new 
org.apache.hadoop.fs.Path("file:///tmp/test")))
+                .isNotInstanceOf(HadoopSecuredFileSystem.class);
+    }
 }

Reply via email to