This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 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 27e9224a3c [oss] add fs.oss.sld.enabled to support oss private link
(#6413)
27e9224a3c is described below
commit 27e9224a3c8959088aa7117ad39896eb0f272517
Author: shyjsarah <[email protected]>
AuthorDate: Thu Oct 16 19:36:17 2025 +0800
[oss] add fs.oss.sld.enabled to support oss private link (#6413)
---
.../main/java/org/apache/paimon/oss/OSSFileIO.java | 23 ++++++++++++++++++++++
1 file changed, 23 insertions(+)
diff --git
a/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java
b/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java
index 7e9d8fc68a..3737a07f17 100644
---
a/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java
+++
b/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java
@@ -24,10 +24,14 @@ import org.apache.paimon.fs.Path;
import org.apache.paimon.fs.TwoPhaseOutputStream;
import org.apache.paimon.options.Options;
import org.apache.paimon.utils.IOUtils;
+import org.apache.paimon.utils.ReflectionUtils;
+import com.aliyun.oss.OSSClient;
+import com.aliyun.oss.common.comm.ServiceClient;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem;
+import org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -58,6 +62,7 @@ public class OSSFileIO extends HadoopCompliantFileIO {
private static final String OSS_ACCESS_KEY_ID = "fs.oss.accessKeyId";
private static final String OSS_ACCESS_KEY_SECRET =
"fs.oss.accessKeySecret";
private static final String OSS_SECURITY_TOKEN = "fs.oss.securityToken";
+ private static final String OSS_SECOND_LEVEL_DOMAIN_ENABLED =
"fs.oss.sld.enabled";
private static final Map<String, String> CASE_SENSITIVE_KEYS =
new HashMap<String, String>() {
@@ -152,6 +157,11 @@ public class OSSFileIO extends HadoopCompliantFileIO {
} catch (IOException e) {
throw new UncheckedIOException(e);
}
+
+ if
(hadoopOptions.getBoolean(OSS_SECOND_LEVEL_DOMAIN_ENABLED, false)) {
+ enableSecondLevelDomain(fs);
+ }
+
return fs;
};
@@ -171,6 +181,19 @@ public class OSSFileIO extends HadoopCompliantFileIO {
}
}
+ public void enableSecondLevelDomain(AliyunOSSFileSystem fs) {
+ AliyunOSSFileSystemStore store = fs.getStore();
+ try {
+ OSSClient ossClient = ReflectionUtils.getPrivateFieldValue(store,
"ossClient");
+ ServiceClient serviceClient =
+ ReflectionUtils.getPrivateFieldValue(ossClient,
"serviceClient");
+ serviceClient.getClientConfiguration().setSLDEnabled(true);
+ } catch (Exception e) {
+ LOG.error("Failed to enable second level domain.", e);
+ throw new RuntimeException("Failed to enable second level
domain.", e);
+ }
+ }
+
private static class CacheKey {
private final Options options;