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

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


The following commit(s) were added to refs/heads/master by this push:
     new 8457600daf3 [FLINK-39552][s3] Add native-s3-fs in 
DIRECTLY_SUPPORTED_FILESYSTEM
8457600daf3 is described below

commit 8457600daf3f3a4cd096d35cbaa2d55a864bb12b
Author: Samrat <[email protected]>
AuthorDate: Mon Apr 27 18:49:20 2026 +0530

    [FLINK-39552][s3] Add native-s3-fs in DIRECTLY_SUPPORTED_FILESYSTEM
---
 .../src/main/java/org/apache/flink/core/fs/FileSystem.java    |  2 ++
 .../test/java/org/apache/flink/core/fs/FileSystemTest.java    | 11 +++++++----
 2 files changed, 9 insertions(+), 4 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java 
b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
index e91c233ff82..3699b2a2962 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
@@ -254,7 +254,9 @@ public abstract class FileSystem implements IFileSystem {
                     .put("oss", "flink-oss-fs-hadoop")
                     .put("s3", "flink-s3-fs-hadoop")
                     .put("s3", "flink-s3-fs-presto")
+                    .put("s3", "flink-s3-fs-native")
                     .put("s3a", "flink-s3-fs-hadoop")
+                    .put("s3a", "flink-s3-fs-native")
                     .put("s3p", "flink-s3-fs-presto")
                     .put("gs", "flink-gs-fs-hadoop")
                     .build();
diff --git 
a/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java 
b/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java
index d252739d86d..e358d39394b 100644
--- a/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java
+++ b/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java
@@ -88,14 +88,15 @@ class FileSystemTest {
         exception should be:
         org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could 
not find a file
         system implementation for scheme 's3'. The scheme is directly 
supported by Flink through the following
-        plugins: flink-s3-fs-hadoop, flink-s3-fs-presto. Please ensure that 
each plugin resides within its own
-        subfolder within the plugins directory.
+        plugins: flink-s3-fs-hadoop, flink-s3-fs-presto, flink-s3-fs-native. 
Please ensure that each plugin
+        resides within its own subfolder within the plugins directory.
         See 
https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/plugins/
 for more information. */
         assertThatThrownBy(() -> 
getFileSystemWithoutSafetyNet("s3://authority/"))
                 .isInstanceOf(UnsupportedFileSystemSchemeException.class)
                 .hasMessageContaining("is directly supported")
                 .hasMessageContaining("flink-s3-fs-hadoop")
                 .hasMessageContaining("flink-s3-fs-presto")
+                .hasMessageContaining("flink-s3-fs-native")
                 .hasMessageNotContaining("no Hadoop file system to support 
this scheme");
     }
 
@@ -110,14 +111,16 @@ class FileSystemTest {
             exception should be:
             org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: 
Could not find a file
             system implementation for scheme 's3'. File system schemes are 
supported by Flink through the following
-            plugin(s): flink-s3-fs-hadoop, flink-s3-fs-presto. No file system 
to support this scheme could be loaded.
-            Please ensure that each plugin is configured properly and resides 
within its own subfolder in the plugins directory.
+            plugin(s): flink-s3-fs-hadoop, flink-s3-fs-presto, 
flink-s3-fs-native. No file system to support this
+            scheme could be loaded. Please ensure that each plugin is 
configured properly and resides within its own
+            subfolder in the plugins directory.
             See 
https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/plugins/
 for more information. */
             assertThatThrownBy(() -> 
getFileSystemWithoutSafetyNet("s3://authority/"))
                     .isInstanceOf(UnsupportedFileSystemSchemeException.class)
                     .hasMessageContaining("File system schemes are supported")
                     .hasMessageContaining("flink-s3-fs-hadoop")
                     .hasMessageContaining("flink-s3-fs-presto")
+                    .hasMessageContaining("flink-s3-fs-native")
                     .hasMessageContaining("Please ensure that each plugin is 
configured properly");
         } finally {
             FileSystem.initialize(new Configuration(), null);

Reply via email to