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

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


The following commit(s) were added to refs/heads/master by this push:
     new d527c14d90f [refactor](fe) Add typed filesystem properties for S3 
(#63252)
d527c14d90f is described below

commit d527c14d90fa9cdec4e645ebd99745ebbac0a727
Author: Calvin Kirs <[email protected]>
AuthorDate: Wed May 20 11:14:57 2026 +0800

    [refactor](fe) Add typed filesystem properties for S3 (#63252)
    
    ### What problem does this PR solve?
    
    Issue Number: None
    
    Related PR: None
    
    Problem Summary: Add provider-owned typed filesystem properties and bind
    S3 runtime paths through S3FileSystemProperties, preparing for
    S3Properties removal from fe-core.
    
    ### Release note
    
    None
    
    ### Check List (For Author)
    
    - Test: Unit Test
    
    - mvn -Dmaven.build.cache.enabled=false -f fe/pom.xml -pl
    fe-filesystem/fe-filesystem-s3 -am -DfailIfNoTests=false test
    
    - mvn -Dmaven.build.cache.enabled=false -f fe/pom.xml -pl
    
fe-filesystem/fe-filesystem-oss,fe-filesystem/fe-filesystem-cos,fe-filesystem/fe-filesystem-obs
    -am -DfailIfNoTests=false test
    
    - mvn -Dmaven.build.cache.enabled=false -f fe/pom.xml -pl
    
fe-filesystem/fe-filesystem-s3,fe-filesystem/fe-filesystem-oss,fe-filesystem/fe-filesystem-cos,fe-filesystem/fe-filesystem-obs
    -am -DfailIfNoTests=false test
    
    - mvn -Dmaven.build.cache.enabled=false -f fe/pom.xml -pl
    fe-filesystem/fe-filesystem-s3 -am -DskipTests validate
    
    - env MAVEN_OPTS=-Xmx5g MVN_OPT=-Dmaven.build.cache.enabled=false
    FE_MAVEN_THREADS=1 FE_MAVEN_RETRY_THREADS=1
    CUSTOM_UI_DIST=/mnt/disk1/gq/idea/incubator-doris/ui/dist ./build.sh
    --fe
---
 .../filesystem/properties/BackendStorageKind.java  |  52 +++
 .../properties/BackendStorageProperties.java       |  43 +++
 .../properties/FileSystemProperties.java           |  92 +++++
 .../properties/HadoopStorageProperties.java        |  39 ++
 .../S3CompatibleFileSystemProperties.java          |  80 +++++
 .../doris/filesystem/properties/StorageKind.java   |  48 +++
 .../doris/filesystem/cos/CosObjStorageTest.java    |   9 +-
 .../doris/filesystem/obs/ObsObjStorageTest.java    |   2 +
 .../doris/filesystem/oss/OssObjStorageTest.java    |   1 +
 fe/fe-filesystem/fe-filesystem-s3/pom.xml          |  17 +
 .../s3/S3CredentialsProviderFactory.java           | 201 +++++++++++
 .../filesystem/s3/S3CredentialsProviderType.java   |  70 ++++
 .../apache/doris/filesystem/s3/S3FileSystem.java   |  17 +
 .../filesystem/s3/S3FileSystemProperties.java      | 398 +++++++++++++++++++++
 .../doris/filesystem/s3/S3FileSystemProvider.java  |  69 +++-
 .../apache/doris/filesystem/s3/S3ObjStorage.java   | 195 ++--------
 .../s3/S3CredentialsProviderFactoryTest.java       | 103 ++++++
 .../filesystem/s3/S3FileSystemPropertiesTest.java  | 245 +++++++++++++
 .../filesystem/s3/S3FileSystemProviderTest.java    |  73 ++++
 .../doris/filesystem/s3/S3ObjStorageMockTest.java  |  82 +++--
 .../doris/filesystem/s3/S3ObjStorageTest.java      | 156 +++-----
 .../doris/filesystem/s3/S3OutputStreamTest.java    |   4 +-
 .../doris/filesystem/spi/FileSystemProvider.java   |  23 ++
 23 files changed, 1698 insertions(+), 321 deletions(-)

diff --git 
a/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/BackendStorageKind.java
 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/BackendStorageKind.java
new file mode 100644
index 00000000000..8e6bf326580
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/BackendStorageKind.java
@@ -0,0 +1,52 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.properties;
+
+/**
+ * Backend-facing storage classification used to select FE-to-BE adapters.
+ *
+ * <p>This is intentionally more specific than {@link StorageKind}. For 
example,
+ * Azure Blob is object storage at the filesystem layer, but it may be sent to 
BE
+ * either through the S3-compatible adapter or through its native AZURE 
storage type.
+ */
+public enum BackendStorageKind {
+    /**
+     * S3-compatible object storage adapter, such as TS3StorageParam.
+     */
+    S3_COMPATIBLE,
+
+    /**
+     * Storage that should keep its own backend storage type, such as AZURE.
+     */
+    NATIVE,
+
+    /**
+     * HDFS or HDFS-compatible storage.
+     */
+    HDFS,
+
+    /**
+     * Broker-based storage.
+     */
+    BROKER,
+
+    /**
+     * Local file system.
+     */
+    LOCAL
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/BackendStorageProperties.java
 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/BackendStorageProperties.java
new file mode 100644
index 00000000000..cfc45aa86a1
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/BackendStorageProperties.java
@@ -0,0 +1,43 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.properties;
+
+import java.util.Map;
+
+/**
+ * Storage properties that can be passed to BE through an RPC adapter.
+ *
+ * <p>This interface deliberately exposes only a neutral key-value form. The 
API
+ * module should not depend on Thrift or any other RPC framework; fe-core 
adapters
+ * are responsible for converting the map to concrete RPC structures.</p>
+ */
+public interface BackendStorageProperties {
+
+    /**
+     * Returns the backend storage kind used to choose the corresponding BE 
RPC adapter.
+     */
+    BackendStorageKind backendKind();
+
+    /**
+     * Converts to a neutral key-value representation. Adapters in fe-core are 
responsible
+     * for creating RPC-specific structures such as TS3StorageParam.
+     *
+     * @return storage parameters using names understood by the corresponding 
adapter
+     */
+    Map<String, String> toMap();
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/FileSystemProperties.java
 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/FileSystemProperties.java
new file mode 100644
index 00000000000..930e5297a02
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/FileSystemProperties.java
@@ -0,0 +1,92 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.properties;
+
+import org.apache.doris.filesystem.FileSystemType;
+
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Parsed and validated FileSystem properties owned by a specific provider.
+ *
+ * <p>The API layer exposes this interface so framework code can pass typed
+ * configuration between filesystem modules without depending on provider
+ * implementations. Each provider is responsible for binding raw key-value
+ * properties, validating them, and converting them to the formats required by
+ * runtime clients, BE RPC adapters, or Hadoop clients.</p>
+ */
+public interface FileSystemProperties {
+
+    /**
+     * Returns the provider name, such as S3, OSS, COS, or OBS.
+     */
+    String providerName();
+
+    /**
+     * Returns the generic storage kind used for provider selection and 
routing.
+     */
+    StorageKind kind();
+
+    /**
+     * Returns the Doris filesystem type represented by this property model.
+     */
+    FileSystemType type();
+
+    /**
+     * Returns the original raw properties passed to FileSystemProvider.bind().
+     *
+     * <p>The returned map is intended for diagnostics and compatibility paths.
+     * Callers should prefer typed accessors or conversion methods for runtime
+     * behavior.</p>
+     */
+    Map<String, String> rawProperties();
+
+    /**
+     * Returns raw key-value pairs that matched provider-declared property 
aliases during binding.
+     *
+     * <p>If a provider accepts multiple aliases for the same setting, this map
+     * records which input keys were actually consumed. This is useful for 
auditing
+     * and for detecting unused or misspelled options without exposing provider
+     * implementation details.</p>
+     */
+    Map<String, String> matchedProperties();
+
+    /**
+     * Converts to the legacy key-value format used by existing FileSystem 
implementations.
+     *
+     * <p>This method exists as a migration bridge. New provider code should 
use
+     * typed accessors internally, while callers that still depend on the old 
map
+     * contract can continue to consume this representation.</p>
+     */
+    Map<String, String> toFileSystemKv();
+
+    /**
+     * Converts to backend storage properties if this provider supports BE 
access.
+     */
+    default Optional<BackendStorageProperties> toBackendProperties() {
+        return Optional.empty();
+    }
+
+    /**
+     * Converts to Hadoop configuration properties if this provider supports 
Hadoop access.
+     */
+    default Optional<HadoopStorageProperties> toHadoopProperties() {
+        return Optional.empty();
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/HadoopStorageProperties.java
 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/HadoopStorageProperties.java
new file mode 100644
index 00000000000..582838d2a0f
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/HadoopStorageProperties.java
@@ -0,0 +1,39 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.properties;
+
+import java.util.Map;
+
+/**
+ * Storage properties that can be exported as Hadoop-compatible configuration.
+ *
+ * <p>The API module returns a map instead of 
org.apache.hadoop.conf.Configuration
+ * so provider metadata can stay independent from Hadoop dependencies. Provider
+ * modules or callers that already depend on Hadoop can materialize the map 
into
+ * a Configuration instance.</p>
+ */
+public interface HadoopStorageProperties {
+
+    /**
+     * Converts to a Hadoop configuration map without exposing Hadoop 
dependencies in
+     * the API layer. Keys should use Hadoop configuration names such as 
fs.s3a.*.
+     *
+     * @return Hadoop configuration key-value pairs for this storage provider
+     */
+    Map<String, String> toHadoopConfigurationMap();
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/S3CompatibleFileSystemProperties.java
 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/S3CompatibleFileSystemProperties.java
new file mode 100644
index 00000000000..d1c1dd91bc4
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/S3CompatibleFileSystemProperties.java
@@ -0,0 +1,80 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.properties;
+
+/**
+ * Shared typed accessors for S3-compatible object storage properties.
+ *
+ * <p>Provider implementations may live in different plugin modules, but 
callers
+ * that only need common S3-compatible settings can depend on this API-level
+ * contract. The interface intentionally contains only JDK types.</p>
+ */
+public interface S3CompatibleFileSystemProperties extends FileSystemProperties 
{
+
+    /** Returns the service endpoint. */
+    String getEndpoint();
+
+    /** Returns the signing region. */
+    String getRegion();
+
+    /** Returns the static access key, or an empty value when static 
credentials are not used. */
+    String getAccessKey();
+
+    /** Returns the static secret key, or an empty value when static 
credentials are not used. */
+    String getSecretKey();
+
+    /** Returns the session token used with temporary static credentials. */
+    String getSessionToken();
+
+    /** Returns the IAM role ARN used for AssumeRole access. */
+    String getRoleArn();
+
+    /** Returns the external ID used for AssumeRole trust policy validation. */
+    String getExternalId();
+
+    /** Returns the default bucket configured for object-storage helper 
operations. */
+    String getBucket();
+
+    /** Returns the root path prefix inside the bucket. */
+    String getRootPath();
+
+    /** Returns the maximum connection count as a provider property value. */
+    String getMaxConnections();
+
+    /** Returns the request timeout in milliseconds as a provider property 
value. */
+    String getRequestTimeoutMs();
+
+    /** Returns the connection timeout in milliseconds as a provider property 
value. */
+    String getConnectionTimeoutMs();
+
+    /** Returns whether path-style bucket addressing is enabled. */
+    String getUsePathStyle();
+
+    /** Returns true when a static AK/SK credential pair is present. */
+    default boolean hasStaticCredentials() {
+        String ak = getAccessKey();
+        String sk = getSecretKey();
+        return ak != null && !ak.isBlank() && sk != null && !sk.isBlank();
+    }
+
+    /** Returns true when AssumeRole (IAM role ARN) access is configured. */
+    default boolean hasAssumeRole() {
+        String arn = getRoleArn();
+        return arn != null && !arn.isBlank();
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/StorageKind.java
 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/StorageKind.java
new file mode 100644
index 00000000000..238660386f2
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-api/src/main/java/org/apache/doris/filesystem/properties/StorageKind.java
@@ -0,0 +1,48 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.properties;
+
+/**
+ * High-level storage classification used by the filesystem framework.
+ */
+public enum StorageKind {
+    /**
+     * Object storage systems such as S3, OSS, COS, OBS, GCS, and Azure Blob.
+     */
+    OBJECT_STORAGE,
+
+    /**
+     * HDFS-compatible file systems such as HDFS and OSS-HDFS.
+     */
+    HDFS_COMPATIBLE,
+
+    /**
+     * Broker-based file systems.
+     */
+    BROKER,
+
+    /**
+     * Local file system.
+     */
+    LOCAL,
+
+    /**
+     * HTTP-based file systems.
+     */
+    HTTP
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-cos/src/test/java/org/apache/doris/filesystem/cos/CosObjStorageTest.java
 
b/fe/fe-filesystem/fe-filesystem-cos/src/test/java/org/apache/doris/filesystem/cos/CosObjStorageTest.java
index cee7a88cfb8..33d5f093960 100644
--- 
a/fe/fe-filesystem/fe-filesystem-cos/src/test/java/org/apache/doris/filesystem/cos/CosObjStorageTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-cos/src/test/java/org/apache/doris/filesystem/cos/CosObjStorageTest.java
@@ -140,7 +140,7 @@ class CosObjStorageTest {
     }
 
     @Test
-    void getPresignedUrl_missingRegionThrowsIOException() {
+    void constructor_missingRegionFailsTypedValidation() {
         COSClient mockCos = Mockito.mock(COSClient.class);
         Map<String, String> props = new HashMap<>();
         props.put("COS_ENDPOINT", "https://cos.myqcloud.com";);
@@ -149,10 +149,11 @@ class CosObjStorageTest {
         props.put("COS_BUCKET", "my-bucket-1234");
         // no region
 
-        CosObjStorage storage = new TestableCosObjStorage(props, mockCos);
+        IllegalArgumentException exception = Assertions.assertThrows(
+                IllegalArgumentException.class, () -> new 
TestableCosObjStorage(props, mockCos));
 
-        Assertions.assertThrows(IOException.class, () -> 
storage.getPresignedUrl("some/key"),
-                "Should throw when region is missing");
+        Assertions.assertTrue(exception.getMessage().contains("Invalid S3 
filesystem properties"));
+        Assertions.assertTrue(exception.getMessage().contains("Region is not 
set"));
     }
 
     @Test
diff --git 
a/fe/fe-filesystem/fe-filesystem-obs/src/test/java/org/apache/doris/filesystem/obs/ObsObjStorageTest.java
 
b/fe/fe-filesystem/fe-filesystem-obs/src/test/java/org/apache/doris/filesystem/obs/ObsObjStorageTest.java
index 22ac2867230..0cd47ce4c41 100644
--- 
a/fe/fe-filesystem/fe-filesystem-obs/src/test/java/org/apache/doris/filesystem/obs/ObsObjStorageTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-obs/src/test/java/org/apache/doris/filesystem/obs/ObsObjStorageTest.java
@@ -125,6 +125,7 @@ class ObsObjStorageTest {
         props.put("OBS_ENDPOINT", "https://obs.myhuaweicloud.com";);
         props.put("OBS_ACCESS_KEY", "ak");
         props.put("OBS_SECRET_KEY", "sk");
+        props.put("OBS_REGION", "cn-north-4");
         // no bucket
 
         ObsObjStorage storage = new TestableObsObjStorage(props, mockObs);
@@ -140,6 +141,7 @@ class ObsObjStorageTest {
         props.put("OBS_ACCESS_KEY", "ak");
         props.put("OBS_SECRET_KEY", "sk");
         props.put("OBS_BUCKET", "my-bucket");
+        props.put("OBS_REGION", "cn-north-4");
         // no endpoint
 
         ObsObjStorage storage = new TestableObsObjStorage(props, mockObs);
diff --git 
a/fe/fe-filesystem/fe-filesystem-oss/src/test/java/org/apache/doris/filesystem/oss/OssObjStorageTest.java
 
b/fe/fe-filesystem/fe-filesystem-oss/src/test/java/org/apache/doris/filesystem/oss/OssObjStorageTest.java
index 8d116d0ddb6..8e1fa753ee0 100644
--- 
a/fe/fe-filesystem/fe-filesystem-oss/src/test/java/org/apache/doris/filesystem/oss/OssObjStorageTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-oss/src/test/java/org/apache/doris/filesystem/oss/OssObjStorageTest.java
@@ -137,6 +137,7 @@ class OssObjStorageTest {
         props.put("AWS_ENDPOINT", "https://oss.aliyuncs.com";);
         props.put("OSS_ACCESS_KEY", "ak");
         props.put("OSS_SECRET_KEY", "sk");
+        props.put("OSS_REGION", "cn-hangzhou");
         // no bucket
 
         OssObjStorage storage = new TestableOssObjStorage(props, mockOss);
diff --git a/fe/fe-filesystem/fe-filesystem-s3/pom.xml 
b/fe/fe-filesystem/fe-filesystem-s3/pom.xml
index 239075c8e42..6c1ca5722ab 100644
--- a/fe/fe-filesystem/fe-filesystem-s3/pom.xml
+++ b/fe/fe-filesystem/fe-filesystem-s3/pom.xml
@@ -41,6 +41,23 @@ under the License.
             <version>${revision}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.doris</groupId>
+            <artifactId>fe-foundation</artifactId>
+            <version>${revision}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+
         <!-- AWS SDK v2 -->
         <dependency>
             <groupId>software.amazon.awssdk</groupId>
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3CredentialsProviderFactory.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3CredentialsProviderFactory.java
new file mode 100644
index 00000000000..6468c554ef6
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3CredentialsProviderFactory.java
@@ -0,0 +1,201 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.s3;
+
+import org.apache.commons.lang3.StringUtils;
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import 
software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
+import 
software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import 
software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider;
+import 
software.amazon.awssdk.auth.credentials.WebIdentityTokenFileCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.sts.StsClient;
+import 
software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.BiFunction;
+
+/**
+ * Credential provider factory for S3 filesystem AWS SDK v2 clients.
+ */
+public final class S3CredentialsProviderFactory {
+
+    private S3CredentialsProviderFactory() {
+    }
+
+    public static AwsCredentialsProvider 
createClientProvider(S3FileSystemProperties properties) {
+        return createClientProvider(properties, 
S3CredentialsProviderFactory::buildStsClient);
+    }
+
+    public static AwsCredentialsProvider 
createClientProvider(S3FileSystemProperties properties,
+            BiFunction<AwsCredentialsProvider, String, StsClient> 
stsClientFactory) {
+        if (properties.hasAssumeRole()) {
+            return createAssumeRoleProvider(properties, stsClientFactory);
+        }
+        return createBaseProvider(properties, true);
+    }
+
+    public static AwsCredentialsProvider 
createStsSourceProvider(S3FileSystemProperties properties) {
+        return createBaseProvider(properties, false);
+    }
+
+    public static AwsCredentialsProvider create(
+            S3CredentialsProviderType type, boolean includeAnonymousInDefault) 
{
+        switch (type) {
+            case ENV:
+                return EnvironmentVariableCredentialsProvider.create();
+            case SYSTEM_PROPERTIES:
+                return SystemPropertyCredentialsProvider.create();
+            case WEB_IDENTITY:
+                return WebIdentityTokenFileCredentialsProvider.create();
+            case CONTAINER:
+                return ContainerCredentialsProvider.create();
+            case INSTANCE_PROFILE:
+                return InstanceProfileCredentialsProvider.create();
+            case ANONYMOUS:
+                return AnonymousCredentialsProvider.create();
+            case DEFAULT:
+                return createDefault(includeAnonymousInDefault);
+            default:
+                throw new UnsupportedOperationException(
+                        "AWS SDK V2 does not support credentials provider 
mode: " + type);
+        }
+    }
+
+    public static String hadoopClassName(
+            S3CredentialsProviderType type, boolean includeAnonymousInDefault) 
{
+        switch (type) {
+            case ENV:
+                return EnvironmentVariableCredentialsProvider.class.getName();
+            case SYSTEM_PROPERTIES:
+                return SystemPropertyCredentialsProvider.class.getName();
+            case WEB_IDENTITY:
+                return WebIdentityTokenFileCredentialsProvider.class.getName();
+            case CONTAINER:
+                return ContainerCredentialsProvider.class.getName();
+            case INSTANCE_PROFILE:
+                return InstanceProfileCredentialsProvider.class.getName();
+            case ANONYMOUS:
+                return AnonymousCredentialsProvider.class.getName();
+            case DEFAULT:
+                List<String> providers = new ArrayList<>();
+                
providers.add(EnvironmentVariableCredentialsProvider.class.getName());
+                
providers.add(SystemPropertyCredentialsProvider.class.getName());
+                
providers.add(InstanceProfileCredentialsProvider.class.getName());
+                if (isWebIdentityConfigured()) {
+                    
providers.add(WebIdentityTokenFileCredentialsProvider.class.getName());
+                }
+                if (isContainerCredentialsConfigured()) {
+                    
providers.add(ContainerCredentialsProvider.class.getName());
+                }
+                providers.add(ProfileCredentialsProvider.class.getName());
+                if (includeAnonymousInDefault) {
+                    
providers.add(AnonymousCredentialsProvider.class.getName());
+                }
+                return String.join(",", providers);
+            default:
+                throw new UnsupportedOperationException(
+                        "AWS SDK V2 does not support credentials provider 
mode: " + type);
+        }
+    }
+
+    private static AwsCredentialsProvider 
createAssumeRoleProvider(S3FileSystemProperties properties,
+            BiFunction<AwsCredentialsProvider, String, StsClient> 
stsClientFactory) {
+        StsClient stsClient = 
stsClientFactory.apply(createStsSourceProvider(properties), 
properties.getRegion());
+        return StsAssumeRoleCredentialsProvider.builder()
+                .stsClient(stsClient)
+                .refreshRequest(builder -> {
+                    builder.roleArn(properties.getRoleArn())
+                            .roleSessionName("doris_" + 
UUID.randomUUID().toString().replace("-", ""));
+                    String externalId = properties.getExternalId();
+                    if (StringUtils.isNotBlank(externalId)) {
+                        builder.externalId(externalId);
+                    }
+                }).build();
+    }
+
+    private static StsClient buildStsClient(AwsCredentialsProvider 
credentialsProvider, String region) {
+        return StsClient.builder()
+                .credentialsProvider(credentialsProvider)
+                .region(Region.of(region))
+                .build();
+    }
+
+    private static AwsCredentialsProvider 
createBaseProvider(S3FileSystemProperties properties,
+            boolean includeAnonymousInDefault) {
+        AwsCredentialsProvider staticProvider = createStaticProvider(
+                properties.getAccessKey(),
+                properties.getSecretKey(),
+                properties.getSessionToken());
+        if (staticProvider != null) {
+            return staticProvider;
+        }
+        return create(properties.getCredentialsProviderType(), 
includeAnonymousInDefault);
+    }
+
+    private static AwsCredentialsProvider createStaticProvider(String 
accessKey, String secretKey,
+            String sessionToken) {
+        if (StringUtils.isBlank(accessKey) || StringUtils.isBlank(secretKey)) {
+            return null;
+        }
+        if (StringUtils.isNotBlank(sessionToken)) {
+            return StaticCredentialsProvider.create(
+                    AwsSessionCredentials.create(accessKey, secretKey, 
sessionToken));
+        }
+        return 
StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, 
secretKey));
+    }
+
+    private static AwsCredentialsProvider createDefault(boolean 
includeAnonymous) {
+        List<AwsCredentialsProvider> providers = new ArrayList<>();
+        providers.add(InstanceProfileCredentialsProvider.create());
+        if (isWebIdentityConfigured()) {
+            providers.add(WebIdentityTokenFileCredentialsProvider.create());
+        }
+        if (isContainerCredentialsConfigured()) {
+            providers.add(ContainerCredentialsProvider.create());
+        }
+        providers.add(EnvironmentVariableCredentialsProvider.create());
+        providers.add(SystemPropertyCredentialsProvider.create());
+        providers.add(ProfileCredentialsProvider.create());
+        if (includeAnonymous) {
+            providers.add(AnonymousCredentialsProvider.create());
+        }
+        return AwsCredentialsProviderChain.builder()
+                .credentialsProviders(providers)
+                .build();
+    }
+
+    private static boolean isWebIdentityConfigured() {
+        return StringUtils.isNotBlank(System.getenv("AWS_ROLE_ARN"))
+                && 
StringUtils.isNotBlank(System.getenv("AWS_WEB_IDENTITY_TOKEN_FILE"));
+    }
+
+    private static boolean isContainerCredentialsConfigured() {
+        return 
StringUtils.isNotBlank(System.getenv("AWS_CONTAINER_CREDENTIALS_FULL_URI"))
+                || 
StringUtils.isNotBlank(System.getenv("AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"));
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3CredentialsProviderType.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3CredentialsProviderType.java
new file mode 100644
index 00000000000..dcd43955f1b
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3CredentialsProviderType.java
@@ -0,0 +1,70 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.s3;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * AWS SDK v2 credentials provider mode for S3 filesystem access.
+ */
+public enum S3CredentialsProviderType {
+    DEFAULT("DEFAULT"),
+    ENV("ENV"),
+    SYSTEM_PROPERTIES("SYSTEM_PROPERTIES"),
+    WEB_IDENTITY("WEB_IDENTITY"),
+    CONTAINER("CONTAINER"),
+    INSTANCE_PROFILE("INSTANCE_PROFILE"),
+    ANONYMOUS("ANONYMOUS");
+
+    private final String mode;
+
+    S3CredentialsProviderType(String mode) {
+        this.mode = mode;
+    }
+
+    public String getMode() {
+        return mode;
+    }
+
+    public static S3CredentialsProviderType fromString(String value) {
+        if (StringUtils.isBlank(value)) {
+            return DEFAULT;
+        }
+        String normalized = value.trim().toUpperCase().replace('-', '_');
+        switch (normalized) {
+            case "ENV":
+            case "ENVIRONMENT":
+                return ENV;
+            case "SYSTEM_PROPERTIES":
+                return SYSTEM_PROPERTIES;
+            case "WEB_IDENTITY":
+            case "WEB_IDENTITY_TOKEN_FILE":
+                return WEB_IDENTITY;
+            case "CONTAINER":
+                return CONTAINER;
+            case "INSTANCE_PROFILE":
+                return INSTANCE_PROFILE;
+            case "ANONYMOUS":
+                return ANONYMOUS;
+            case "DEFAULT":
+                return DEFAULT;
+            default:
+                throw new IllegalArgumentException("Unsupported 
s3.credentials_provider_type: " + value);
+        }
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystem.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystem.java
index 6de2fbf853f..4b3584ef927 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystem.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystem.java
@@ -46,6 +46,7 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.regex.Pattern;
 
 /* S3-backed FileSystem implementation for the Doris FE filesystem SPI.
@@ -58,8 +59,24 @@ public class S3FileSystem extends ObjFileSystem {
     // S3 does not have real directories; use a zero-byte marker with trailing 
slash.
     private static final String DIR_MARKER_SUFFIX = "/";
 
+    private final S3FileSystemProperties properties;
+
+    public S3FileSystem(S3FileSystemProperties properties) {
+        this(properties, new S3ObjStorage(properties));
+    }
+
+    S3FileSystem(S3FileSystemProperties properties, S3ObjStorage objStorage) {
+        super("S3", objStorage);
+        this.properties = properties;
+    }
+
     public S3FileSystem(S3ObjStorage objStorage) {
         super("S3", objStorage);
+        this.properties = null;
+    }
+
+    public Optional<S3FileSystemProperties> properties() {
+        return Optional.ofNullable(properties);
     }
 
     @Override
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProperties.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProperties.java
new file mode 100644
index 00000000000..96a38e4db4b
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProperties.java
@@ -0,0 +1,398 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.s3;
+
+import org.apache.doris.filesystem.FileSystemType;
+import org.apache.doris.filesystem.properties.BackendStorageKind;
+import org.apache.doris.filesystem.properties.BackendStorageProperties;
+import org.apache.doris.filesystem.properties.FileSystemProperties;
+import org.apache.doris.filesystem.properties.HadoopStorageProperties;
+import org.apache.doris.filesystem.properties.S3CompatibleFileSystemProperties;
+import org.apache.doris.filesystem.properties.StorageKind;
+import org.apache.doris.foundation.property.ConnectorPropertiesUtils;
+import org.apache.doris.foundation.property.ConnectorProperty;
+import org.apache.doris.foundation.property.ParamRules;
+
+import lombok.Getter;
+import org.apache.commons.lang3.StringUtils;
+
+import java.lang.reflect.Field;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Provider-owned typed properties for AWS S3 and S3-compatible object storage.
+ * Binding uses {@link ConnectorProperty} aliases so legacy key names can 
continue
+ * to work while callers migrate to canonical s3.* names.
+ */
+public final class S3FileSystemProperties
+        implements FileSystemProperties, BackendStorageProperties, 
HadoopStorageProperties,
+                S3CompatibleFileSystemProperties {
+
+    public static final String ENDPOINT = "s3.endpoint";
+    public static final String REGION = "s3.region";
+    public static final String ACCESS_KEY = "s3.access_key";
+    public static final String SECRET_KEY = "s3.secret_key";
+    public static final String SESSION_TOKEN = "s3.session_token";
+    public static final String ROLE_ARN = "s3.role_arn";
+    public static final String EXTERNAL_ID = "s3.external_id";
+    public static final String BUCKET = "s3.bucket";
+    public static final String ROOT_PATH = "s3.root.path";
+    public static final String MAX_CONNECTIONS = "s3.connection.maximum";
+    public static final String REQUEST_TIMEOUT_MS = 
"s3.connection.request.timeout";
+    public static final String CONNECTION_TIMEOUT_MS = "s3.connection.timeout";
+    public static final String USE_PATH_STYLE = "use_path_style";
+    public static final String CREDENTIALS_PROVIDER_TYPE = 
"s3.credentials_provider_type";
+
+    public static final String DEFAULT_MAX_CONNECTIONS = "50";
+    public static final String DEFAULT_REQUEST_TIMEOUT_MS = "3000";
+    public static final String DEFAULT_CONNECTION_TIMEOUT_MS = "1000";
+    public static final String DEFAULT_CREDENTIALS_PROVIDER_TYPE = "DEFAULT";
+    public static final String DEFAULT_REGION = "us-east-1";
+
+    private static final Pattern[] ENDPOINT_PATTERNS = new Pattern[] {
+            Pattern.compile(
+                    "^(?:https?://)?(?:"
+                            + 
"s3(?:[-.]fips)?(?:[-.]dualstack)?[-.]([a-z0-9-]+)|"
+                            + "s3express-control\\.([a-z0-9-]+)|"
+                            + "s3express-[a-z0-9-]+\\.([a-z0-9-]+)"
+                            + ")\\.amazonaws\\.com(?:/.*)?$",
+                    Pattern.CASE_INSENSITIVE),
+            Pattern.compile(
+                    
"^(?:https?://)?glue(?:-fips)?\\.([a-z0-9-]+)\\.(amazonaws\\.com(?:\\.cn)?|api\\.aws)$",
+                    Pattern.CASE_INSENSITIVE)
+    };
+
+    @Getter
+    @ConnectorProperty(names = {ENDPOINT, "AWS_ENDPOINT", "endpoint", 
"ENDPOINT", "aws.endpoint",
+            "glue.endpoint", "aws.glue.endpoint"},
+            required = false,
+            description = "The endpoint of S3.")
+    private String endpoint = "";
+
+    @Getter
+    @ConnectorProperty(names = {REGION, "AWS_REGION", "region", "REGION", 
"aws.region", "glue.region",
+            "aws.glue.region", "iceberg.rest.signing-region", 
"rest.signing-region", "client.region"},
+            required = false,
+            isRegionField = true,
+            description = "The region of S3.")
+    private String region = "";
+
+    @Getter
+    @ConnectorProperty(names = {ACCESS_KEY, "AWS_ACCESS_KEY", "access_key", 
"ACCESS_KEY",
+            "glue.access_key", "aws.glue.access-key",
+            "client.credentials-provider.glue.access_key", 
"iceberg.rest.access-key-id",
+            "s3.access-key-id"},
+            required = false,
+            sensitive = true,
+            description = "The access key of S3.")
+    private String accessKey = "";
+
+    @Getter
+    @ConnectorProperty(names = {SECRET_KEY, "AWS_SECRET_KEY", "secret_key", 
"SECRET_KEY",
+            "glue.secret_key", "aws.glue.secret-key",
+            "client.credentials-provider.glue.secret_key", 
"iceberg.rest.secret-access-key",
+            "s3.secret-access-key"},
+            required = false,
+            sensitive = true,
+            description = "The secret key of S3.")
+    private String secretKey = "";
+
+    @Getter
+    @ConnectorProperty(names = {SESSION_TOKEN, "AWS_TOKEN", "session_token",
+            "s3.session-token", "iceberg.rest.session-token"},
+            required = false,
+            description = "The session token of S3.")
+    private String sessionToken = "";
+
+    @Getter
+    @ConnectorProperty(names = {ROLE_ARN, "AWS_ROLE_ARN", "glue.role_arn"},
+            required = false,
+            description = "The IAM role ARN for AssumeRole-based access.")
+    private String roleArn = "";
+
+    @Getter
+    @ConnectorProperty(names = {EXTERNAL_ID, "AWS_EXTERNAL_ID", 
"glue.external_id"},
+            required = false,
+            description = "The external ID for AssumeRole trust policy.")
+    private String externalId = "";
+
+    @Getter
+    @ConnectorProperty(names = {BUCKET, "AWS_BUCKET"},
+            required = false,
+            description = "The default bucket name.")
+    private String bucket = "";
+
+    @Getter
+    @ConnectorProperty(names = {ROOT_PATH, "AWS_ROOT_PATH"},
+            required = false,
+            description = "The root path prefix inside the bucket.")
+    private String rootPath = "";
+
+    @Getter
+    @ConnectorProperty(names = {MAX_CONNECTIONS, "AWS_MAX_CONNECTIONS"},
+            required = false,
+            description = "The maximum number of connections to S3.")
+    private String maxConnections = DEFAULT_MAX_CONNECTIONS;
+
+    @Getter
+    @ConnectorProperty(names = {REQUEST_TIMEOUT_MS, "AWS_REQUEST_TIMEOUT_MS"},
+            required = false,
+            description = "The request timeout of S3 in milliseconds.")
+    private String requestTimeoutMs = DEFAULT_REQUEST_TIMEOUT_MS;
+
+    @Getter
+    @ConnectorProperty(names = {CONNECTION_TIMEOUT_MS, 
"AWS_CONNECTION_TIMEOUT_MS"},
+            required = false,
+            description = "The connection timeout of S3 in milliseconds.")
+    private String connectionTimeoutMs = DEFAULT_CONNECTION_TIMEOUT_MS;
+
+    @Getter
+    @ConnectorProperty(names = {USE_PATH_STYLE, "s3.path-style-access"},
+            required = false,
+            description = "Whether to use path-style bucket addressing.")
+    private String usePathStyle = "false";
+
+    @ConnectorProperty(names = {CREDENTIALS_PROVIDER_TYPE, 
"AWS_CREDENTIALS_PROVIDER_TYPE",
+            "glue.credentials_provider_type", 
"iceberg.rest.credentials_provider_type"},
+            required = false,
+            description = "The credentials provider type.")
+    private String credentialsProviderType = DEFAULT_CREDENTIALS_PROVIDER_TYPE;
+
+    private final Map<String, String> rawProperties;
+    private final Map<String, String> matchedProperties;
+
+    private S3FileSystemProperties(Map<String, String> rawProperties) {
+        this.rawProperties = Collections.unmodifiableMap(new 
HashMap<>(rawProperties));
+        this.matchedProperties = 
Collections.unmodifiableMap(collectMatchedProperties(rawProperties));
+        ConnectorPropertiesUtils.bindConnectorProperties(this, rawProperties);
+        normalizeForLegacyS3Compatibility();
+    }
+
+    /** Binds and validates raw properties. */
+    public static S3FileSystemProperties of(Map<String, String> properties) {
+        S3FileSystemProperties props = new S3FileSystemProperties(properties);
+        props.validate();
+        return props;
+    }
+
+    private void validate() {
+        new ParamRules()
+                .requireTogether(new String[] {accessKey, secretKey},
+                        "s3.access_key and s3.secret_key must be set together")
+                .requireAllIfPresent(sessionToken, new String[] {accessKey, 
secretKey},
+                        "s3.session_token requires s3.access_key and 
s3.secret_key")
+                .requireAllIfPresent(externalId, new String[] {roleArn},
+                        "s3.external_id must be used together with 
s3.role_arn")
+                .check(this::hasUnsupportedCredentialsProviderType,
+                        "Unsupported s3.credentials_provider_type: " + 
credentialsProviderType)
+                .check(() -> StringUtils.isBlank(endpoint) && 
StringUtils.isBlank(region),
+                        "Either s3.endpoint or s3.region must be set")
+                .validate("Invalid S3 filesystem properties");
+    }
+
+    @Override
+    public String providerName() {
+        return "S3";
+    }
+
+    @Override
+    public StorageKind kind() {
+        return StorageKind.OBJECT_STORAGE;
+    }
+
+    @Override
+    public FileSystemType type() {
+        return FileSystemType.S3;
+    }
+
+    @Override
+    public Map<String, String> rawProperties() {
+        return rawProperties;
+    }
+
+    @Override
+    public Map<String, String> matchedProperties() {
+        return matchedProperties;
+    }
+
+    /**
+     * Returns canonical {@code AWS_*} keys consumed by {@link S3ObjStorage}.
+     * This preserves compatibility with the existing map-based path.
+     */
+    @Override
+    public Map<String, String> toFileSystemKv() {
+        Map<String, String> kv = new HashMap<>();
+        putIfNotBlank(kv, "AWS_ENDPOINT", endpoint);
+        putIfNotBlank(kv, "AWS_REGION", region);
+        putIfNotBlank(kv, "AWS_ACCESS_KEY", accessKey);
+        putIfNotBlank(kv, "AWS_SECRET_KEY", secretKey);
+        putIfNotBlank(kv, "AWS_TOKEN", sessionToken);
+        putIfNotBlank(kv, "AWS_ROLE_ARN", roleArn);
+        putIfNotBlank(kv, "AWS_EXTERNAL_ID", externalId);
+        putIfNotBlank(kv, "AWS_BUCKET", bucket);
+        putIfNotBlank(kv, "AWS_ROOT_PATH", rootPath);
+        kv.put("AWS_MAX_CONNECTIONS", maxConnections);
+        kv.put("AWS_REQUEST_TIMEOUT_MS", requestTimeoutMs);
+        kv.put("AWS_CONNECTION_TIMEOUT_MS", connectionTimeoutMs);
+        kv.put(USE_PATH_STYLE, usePathStyle);
+        kv.put("AWS_CREDENTIALS_PROVIDER_TYPE", 
getCredentialsProviderType().getMode());
+        return Collections.unmodifiableMap(kv);
+    }
+
+    @Override
+    public Optional<BackendStorageProperties> toBackendProperties() {
+        return Optional.of(this);
+    }
+
+    @Override
+    public Optional<HadoopStorageProperties> toHadoopProperties() {
+        return Optional.of(this);
+    }
+
+    @Override
+    public BackendStorageKind backendKind() {
+        return BackendStorageKind.S3_COMPATIBLE;
+    }
+
+    @Override
+    public Map<String, String> toMap() {
+        return toFileSystemKv();
+    }
+
+    @Override
+    public Map<String, String> toHadoopConfigurationMap() {
+        Map<String, String> cfg = new HashMap<>();
+        cfg.put("fs.s3.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem");
+        cfg.put("fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem");
+        cfg.put("fs.s3.impl.disable.cache", "true");
+        cfg.put("fs.s3a.impl.disable.cache", "true");
+        putIfNotBlank(cfg, "fs.s3a.endpoint", endpoint);
+        putIfNotBlank(cfg, "fs.s3a.endpoint.region", region);
+        if (StringUtils.isNotBlank(accessKey)) {
+            cfg.put("fs.s3a.aws.credentials.provider",
+                    "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider");
+            cfg.put("fs.s3a.access.key", accessKey);
+            cfg.put("fs.s3a.secret.key", secretKey);
+            putIfNotBlank(cfg, "fs.s3a.session.token", sessionToken);
+        }
+        if (StringUtils.isNotBlank(roleArn)) {
+            cfg.put("fs.s3a.assumed.role.arn", roleArn);
+            cfg.put("fs.s3a.aws.credentials.provider",
+                    
"org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider");
+            cfg.put("fs.s3a.assumed.role.credentials.provider",
+                    
S3CredentialsProviderFactory.hadoopClassName(getCredentialsProviderType(), 
false));
+            putIfNotBlank(cfg, "fs.s3a.assumed.role.external.id", externalId);
+        } else if (StringUtils.isBlank(accessKey)) {
+            cfg.put("fs.s3a.aws.credentials.provider",
+                    
S3CredentialsProviderFactory.hadoopClassName(getCredentialsProviderType(), 
true));
+        }
+        cfg.put("fs.s3a.connection.maximum", maxConnections);
+        cfg.put("fs.s3a.connection.request.timeout", requestTimeoutMs);
+        cfg.put("fs.s3a.connection.timeout", connectionTimeoutMs);
+        cfg.put("fs.s3a.path.style.access", usePathStyle);
+        return Collections.unmodifiableMap(cfg);
+    }
+
+    public S3CredentialsProviderType getCredentialsProviderType() {
+        return S3CredentialsProviderType.fromString(credentialsProviderType);
+    }
+
+    public boolean hasStaticCredentials() {
+        return StringUtils.isNotBlank(accessKey) && 
StringUtils.isNotBlank(secretKey);
+    }
+
+    public boolean hasAssumeRole() {
+        return StringUtils.isNotBlank(roleArn);
+    }
+
+    private static void putIfNotBlank(Map<String, String> map, String key, 
String value) {
+        if (StringUtils.isNotBlank(value)) {
+            map.put(key, value);
+        }
+    }
+
+    private boolean hasUnsupportedCredentialsProviderType() {
+        try {
+            getCredentialsProviderType();
+            return false;
+        } catch (IllegalArgumentException e) {
+            return true;
+        }
+    }
+
+    private static Map<String, String> collectMatchedProperties(Map<String, 
String> rawProperties) {
+        Map<String, String> matched = new HashMap<>();
+        for (Field field : 
ConnectorPropertiesUtils.getConnectorProperties(S3FileSystemProperties.class)) {
+            String matchedName = 
ConnectorPropertiesUtils.getMatchedPropertyName(field, rawProperties);
+            if (StringUtils.isNotBlank(matchedName)) {
+                matched.put(matchedName, rawProperties.get(matchedName));
+            }
+        }
+        return matched;
+    }
+
+    private void normalizeForLegacyS3Compatibility() {
+        if (StringUtils.isBlank(endpoint) && StringUtils.isNotBlank(region)) {
+            endpoint = buildS3Endpoint(region);
+        }
+        if (StringUtils.isBlank(region) && StringUtils.isNotBlank(endpoint)) {
+            region = extractRegion(endpoint).orElse(DEFAULT_REGION);
+        }
+        if (StringUtils.containsIgnoreCase(endpoint, "glue") && 
StringUtils.isNotBlank(region)) {
+            endpoint = buildS3Endpoint(region);
+        }
+    }
+
+    private static String buildS3Endpoint(String region) {
+        return "https://s3."; + region + ".amazonaws.com";
+    }
+
+    private static Optional<String> extractRegion(String endpoint) {
+        String lowerEndpoint = endpoint.toLowerCase();
+        for (Pattern pattern : ENDPOINT_PATTERNS) {
+            Matcher matcher = pattern.matcher(lowerEndpoint);
+            if (!matcher.matches()) {
+                continue;
+            }
+            for (int i = 1; i <= matcher.groupCount(); i++) {
+                String group = matcher.group(i);
+                if (StringUtils.isNotBlank(group)) {
+                    return Optional.of(group);
+                }
+            }
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public String toString() {
+        return "S3FileSystemProperties{"
+                + "endpoint='" + endpoint + '\''
+                + ", region='" + region + '\''
+                + ", accessKey=" + (StringUtils.isNotBlank(accessKey) ? "***" 
: "<empty>")
+                + ", roleArn='" + roleArn + '\''
+                + ", bucket='" + bucket + '\''
+                + ", usePathStyle=" + usePathStyle
+                + '}';
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProvider.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProvider.java
index 49e7e06f5d5..6a98c1c2e6e 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProvider.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3FileSystemProvider.java
@@ -18,8 +18,11 @@
 package org.apache.doris.filesystem.s3;
 
 import org.apache.doris.filesystem.FileSystem;
+import org.apache.doris.filesystem.properties.FileSystemProperties;
 import org.apache.doris.filesystem.spi.FileSystemProvider;
 
+import org.apache.commons.lang3.StringUtils;
+
 import java.io.IOException;
 import java.util.Map;
 
@@ -34,30 +37,72 @@ import java.util.Map;
  */
 public class S3FileSystemProvider implements FileSystemProvider {
 
+    private static final String STORAGE_TYPE_KEY = "_STORAGE_TYPE_";
+    private static final String STORAGE_TYPE_S3 = "S3";
+    private static final String PROVIDER_KEY = "provider";
+    private static final String FS_S3_SUPPORT = "fs.s3.support";
+    private static final String[] ACCESS_KEY_NAMES = {
+            S3FileSystemProperties.ACCESS_KEY, "AWS_ACCESS_KEY", "access_key", 
"ACCESS_KEY",
+            "glue.access_key", "aws.glue.access-key",
+            "client.credentials-provider.glue.access_key", 
"iceberg.rest.access-key-id",
+            "s3.access-key-id"};
+    private static final String[] ENDPOINT_NAMES = {
+            S3FileSystemProperties.ENDPOINT, "AWS_ENDPOINT", "endpoint", 
"ENDPOINT", "aws.endpoint",
+            "glue.endpoint", "aws.glue.endpoint"};
+    private static final String[] REGION_NAMES = {
+            S3FileSystemProperties.REGION, "AWS_REGION", "region", "REGION", 
"aws.region", "glue.region",
+            "aws.glue.region", "iceberg.rest.signing-region", 
"rest.signing-region", "client.region"};
+    private static final String[] ROLE_ARN_NAMES = {
+            S3FileSystemProperties.ROLE_ARN, "AWS_ROLE_ARN", "glue.role_arn"};
+    private static final String[] CREDENTIALS_PROVIDER_TYPE_NAMES = {
+            S3FileSystemProperties.CREDENTIALS_PROVIDER_TYPE, 
"AWS_CREDENTIALS_PROVIDER_TYPE",
+            "glue.credentials_provider_type", 
"iceberg.rest.credentials_provider_type"};
+
     @Override
     public boolean supports(Map<String, String> properties) {
-        Map<String, String> normalized = 
S3ObjStorage.normalizeProperties(properties);
-        String accessKey = normalized.get(S3ObjStorage.PROP_ACCESS_KEY);
-        String endpoint = normalized.get(S3ObjStorage.PROP_ENDPOINT);
-        String region = normalized.get(S3ObjStorage.PROP_REGION);
-        String roleArn = normalized.get(S3ObjStorage.PROP_ROLE_ARN);
-        boolean hasCredential = accessKey != null && !accessKey.isEmpty()
-                || roleArn != null && !roleArn.isEmpty();
-        boolean hasLocation = endpoint != null && !endpoint.isEmpty()
-                || region != null && !region.isEmpty();
+        boolean hasCredential = hasAny(properties, ACCESS_KEY_NAMES)
+                || hasAny(properties, ROLE_ARN_NAMES)
+                || hasAny(properties, CREDENTIALS_PROVIDER_TYPE_NAMES);
+        boolean hasLocation = hasAny(properties, ENDPOINT_NAMES) || 
hasAny(properties, REGION_NAMES);
+        if (isExplicitS3(properties)) {
+            return hasLocation;
+        }
         // Support both AK/SK and IAM role based access for cloud snapshot and 
stage flows.
         return hasCredential && hasLocation;
     }
 
+    private boolean isExplicitS3(Map<String, String> properties) {
+        return 
STORAGE_TYPE_S3.equalsIgnoreCase(properties.get(STORAGE_TYPE_KEY))
+                || 
STORAGE_TYPE_S3.equalsIgnoreCase(properties.get(PROVIDER_KEY))
+                || Boolean.parseBoolean(properties.getOrDefault(FS_S3_SUPPORT, 
"false"));
+    }
+
+    @Override
+    public FileSystemProperties bind(Map<String, String> properties) {
+        return S3FileSystemProperties.of(properties);
+    }
+
+    @Override
+    public FileSystem create(FileSystemProperties properties) throws 
IOException {
+        return new S3FileSystem((S3FileSystemProperties) properties);
+    }
+
     @Override
     public FileSystem create(Map<String, String> properties) throws 
IOException {
-        Map<String, String> normalized = 
S3ObjStorage.normalizeProperties(properties);
-        S3ObjStorage storage = new S3ObjStorage(normalized);
-        return new S3FileSystem(storage);
+        return create(bind(properties));
     }
 
     @Override
     public String name() {
         return "S3";
     }
+
+    private boolean hasAny(Map<String, String> properties, String[] names) {
+        for (String name : names) {
+            if (StringUtils.isNotBlank(properties.get(name))) {
+                return true;
+            }
+        }
+        return false;
+    }
 }
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3ObjStorage.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3ObjStorage.java
index f6e66de8cbb..e3bd7a6ef26 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3ObjStorage.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/main/java/org/apache/doris/filesystem/s3/S3ObjStorage.java
@@ -23,14 +23,11 @@ import org.apache.doris.filesystem.spi.RemoteObjects;
 import org.apache.doris.filesystem.spi.StsCredentials;
 import org.apache.doris.filesystem.spi.UploadPartResult;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
 import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
 import software.amazon.awssdk.core.exception.SdkException;
 import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient;
@@ -65,7 +62,6 @@ import 
software.amazon.awssdk.services.s3.presigner.S3Presigner;
 import 
software.amazon.awssdk.services.s3.presigner.model.PresignedPutObjectRequest;
 import 
software.amazon.awssdk.services.s3.presigner.model.PutObjectPresignRequest;
 import software.amazon.awssdk.services.sts.StsClient;
-import 
software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider;
 import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
 import software.amazon.awssdk.services.sts.model.AssumeRoleResponse;
 import software.amazon.awssdk.services.sts.model.Credentials;
@@ -78,7 +74,6 @@ import java.net.URI;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -87,81 +82,31 @@ import java.util.stream.Collectors;
 
 /**
  * Object storage implementation backed by AWS S3 SDK v2.
- * Accepts only Map<String, String> in constructor; no dependency on fe-core 
or fe-common.
- *
- * <p>Recognized property keys:
- * <ul>
- *   <li>AWS_ENDPOINT     - S3 endpoint URL (required)</li>
- *   <li>AWS_REGION       - AWS region identifier (required)</li>
- *   <li>AWS_ACCESS_KEY   - AWS access key ID</li>
- *   <li>AWS_SECRET_KEY   - AWS secret access key</li>
- *   <li>AWS_TOKEN        - AWS session token (optional)</li>
- *   <li>use_path_style   - "true" to enable path-style bucket access</li>
- * </ul>
+ * The Map constructor binds through {@link S3FileSystemProperties} so all 
runtime
+ * paths use the same typed S3 parameter model.
  */
 public class S3ObjStorage implements ObjStorage<S3Client> {
 
     private static final Logger LOG = LogManager.getLogger(S3ObjStorage.class);
 
-    static final String PROP_ENDPOINT = "AWS_ENDPOINT";
-    static final String PROP_REGION = "AWS_REGION";
-    static final String PROP_ACCESS_KEY = "AWS_ACCESS_KEY";
-    static final String PROP_SECRET_KEY = "AWS_SECRET_KEY";
-    static final String PROP_TOKEN = "AWS_TOKEN";
-    static final String PROP_PATH_STYLE = "use_path_style";
-    static final String PROP_BUCKET = "AWS_BUCKET";
-    static final String PROP_ROLE_ARN = "AWS_ROLE_ARN";
-    static final String PROP_EXTERNAL_ID = "AWS_EXTERNAL_ID";
-
     /** Validity period for pre-signed URLs and STS tokens (seconds). */
     private static final int SESSION_EXPIRE_SECONDS = 3600;
 
-    /**
-     * Normalizes property keys to canonical AWS_* form so that callers using
-     * alternate key formats (e.g. "s3.access_key", "access_key") are treated
-     * identically to callers that already use canonical keys like 
"AWS_ACCESS_KEY".
-     *
-     * <p>Only adds a canonical entry when the canonical key is absent; 
explicit
-     * canonical values are never overridden.
-     */
-    static Map<String, String> normalizeProperties(Map<String, String> props) {
-        Map<String, String> result = new HashMap<>(props);
-        addIfAbsent(result, PROP_ACCESS_KEY, "s3.access_key", "access_key", 
"ACCESS_KEY");
-        addIfAbsent(result, PROP_SECRET_KEY, "s3.secret_key", "secret_key", 
"SECRET_KEY");
-        addIfAbsent(result, PROP_ENDPOINT, "s3.endpoint", "endpoint", 
"ENDPOINT");
-        addIfAbsent(result, PROP_REGION, "s3.region", "region", "REGION");
-        addIfAbsent(result, PROP_TOKEN, "s3.session_token", "session_token");
-        return result;
-    }
-
-    /** Copies the first non-null alias value into {@code canonicalKey} if not 
already present. */
-    private static void addIfAbsent(Map<String, String> map, String 
canonicalKey, String... aliases) {
-        if (map.containsKey(canonicalKey)) {
-            return;
-        }
-        for (String alias : aliases) {
-            String value = map.get(alias);
-            if (value != null) {
-                map.put(canonicalKey, value);
-                return;
-            }
-        }
-    }
-
-    private final Map<String, String> properties;
+    private final S3FileSystemProperties s3Properties;
     private final boolean usePathStyle;
     /** Bucket name; may be null if not provided (listObjectsWithPrefix and 
related methods will fail). */
     private final String bucket;
     private final AtomicBoolean closed = new AtomicBoolean(false);
     private volatile S3Client client;
 
+    public S3ObjStorage(S3FileSystemProperties properties) {
+        this.s3Properties = properties;
+        this.usePathStyle = Boolean.parseBoolean(properties.getUsePathStyle());
+        this.bucket = properties.getBucket();
+    }
+
     public S3ObjStorage(Map<String, String> properties) {
-        // Always normalize so that subclasses (OssObjStorage, CosObjStorage, 
etc.)
-        // which pass s3.* property keys also get them mapped to canonical 
AWS_* form.
-        Map<String, String> normalized = normalizeProperties(properties);
-        this.properties = Collections.unmodifiableMap(normalized);
-        this.usePathStyle = 
Boolean.parseBoolean(normalized.getOrDefault(PROP_PATH_STYLE, "false"));
-        this.bucket = normalized.get(PROP_BUCKET);
+        this(S3FileSystemProperties.of(properties));
     }
 
     /**
@@ -189,25 +134,14 @@ public class S3ObjStorage implements ObjStorage<S3Client> 
{
     }
 
     protected S3Client buildClient() throws IOException {
-        String endpointStr = properties.get(PROP_ENDPOINT);
-        // #23: Region is required for SigV4 signing. Historically we silently 
fell back to
-        // "us-east-1" when none was configured, which can mis-route requests 
to the wrong AWS
-        // region for standard S3 (no endpoint override). Soft-deprecate by 
logging a WARN
-        // rather than throwing, to avoid breaking clusters that rely on the 
implicit default.
-        String region = properties.get(PROP_REGION);
-        if (region == null || region.isEmpty()) {
-            region = "us-east-1";
-            if (endpointStr == null || endpointStr.isEmpty()) {
-                LOG.warn("S3 region is not configured (set s3.region / region 
/ AWS_REGION); "
-                        + "falling back to '{}'. This is deprecated and may 
mis-route requests "
-                        + "for non-us-east-1 buckets — configure the region 
explicitly.", region);
-            } else {
-                LOG.warn("S3 region is not configured but endpoint '{}' is 
set; using '{}' as a "
-                        + "placeholder solely for SigV4 signing.", 
endpointStr, region);
-            }
-        }
-        AwsCredentialsProvider credentialsProvider = 
buildCredentialsProvider();
+        return buildClient(
+                s3Properties.getEndpoint(),
+                s3Properties.getRegion(),
+                buildCredentialsProvider());
+    }
 
+    private S3Client buildClient(String endpointStr, String region, 
AwsCredentialsProvider credentialsProvider)
+            throws IOException {
         S3ClientBuilder builder = S3Client.builder()
                 .httpClient(UrlConnectionHttpClient.builder()
                         .socketTimeout(Duration.ofSeconds(30))
@@ -222,7 +156,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
 
         // endpointOverride is only set for non-AWS endpoints (MinIO, COS, 
OSS, etc.).
         // Standard AWS S3 access uses region-only routing without an explicit 
endpoint.
-        if (endpointStr != null && !endpointStr.isEmpty()) {
+        if (StringUtils.isNotBlank(endpointStr)) {
             if (!endpointStr.contains("://")) {
                 endpointStr = "https://"; + endpointStr;
             }
@@ -244,47 +178,11 @@ public class S3ObjStorage implements ObjStorage<S3Client> 
{
     }
 
     protected AwsCredentialsProvider buildCredentialsProvider() {
-        String roleArn = properties.get(PROP_ROLE_ARN);
-        if (roleArn != null && !roleArn.isEmpty()) {
-            return buildAssumeRoleCredentialsProvider(roleArn, 
properties.get(PROP_EXTERNAL_ID));
-        }
-        return buildClientBaseCredentialsProvider();
-    }
-
-    private AwsCredentialsProvider buildClientBaseCredentialsProvider() {
-        String accessKey = properties.get(PROP_ACCESS_KEY);
-        String secretKey = properties.get(PROP_SECRET_KEY);
-        String token = properties.get(PROP_TOKEN);
-
-        if (accessKey != null && !accessKey.isEmpty() && secretKey != null && 
!secretKey.isEmpty()) {
-            if (token != null && !token.isEmpty()) {
-                return StaticCredentialsProvider.create(
-                        AwsSessionCredentials.create(accessKey, secretKey, 
token));
-            }
-            return 
StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, 
secretKey));
-        }
-        // Allow anonymous access: chain DefaultCredentialsProvider with 
AnonymousCredentialsProvider
-        // as fallback, so public buckets can be accessed without any 
credentials.
-        return AwsCredentialsProviderChain.builder()
-                .credentialsProviders(
-                        DefaultCredentialsProvider.create(),
-                        AnonymousCredentialsProvider.create())
-                .build();
+        return S3CredentialsProviderFactory.createClientProvider(s3Properties, 
this::buildStsClient);
     }
 
     private AwsCredentialsProvider buildStsSourceCredentialsProvider() {
-        String accessKey = properties.get(PROP_ACCESS_KEY);
-        String secretKey = properties.get(PROP_SECRET_KEY);
-        String token = properties.get(PROP_TOKEN);
-
-        if (accessKey != null && !accessKey.isEmpty() && secretKey != null && 
!secretKey.isEmpty()) {
-            if (token != null && !token.isEmpty()) {
-                return StaticCredentialsProvider.create(
-                        AwsSessionCredentials.create(accessKey, secretKey, 
token));
-            }
-            return 
StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, 
secretKey));
-        }
-        return DefaultCredentialsProvider.create();
+        return 
S3CredentialsProviderFactory.createStsSourceProvider(s3Properties);
     }
 
     protected StsClient buildStsClient(AwsCredentialsProvider 
credentialsProvider, String region) {
@@ -294,27 +192,13 @@ public class S3ObjStorage implements ObjStorage<S3Client> 
{
                 .build();
     }
 
-    private AwsCredentialsProvider buildAssumeRoleCredentialsProvider(String 
roleArn, String externalId) {
-        String region = properties.getOrDefault(PROP_REGION, "us-east-1");
-        StsClient stsClient = 
buildStsClient(buildStsSourceCredentialsProvider(), region);
-        return StsAssumeRoleCredentialsProvider.builder()
-                .stsClient(stsClient)
-                .refreshRequest(builder -> {
-                    builder.roleArn(roleArn)
-                            .roleSessionName("doris_" + 
UUID.randomUUID().toString().replace("-", ""));
-                    if (externalId != null && !externalId.isEmpty()) {
-                        builder.externalId(externalId);
-                    }
-                }).build();
-    }
-
     @Override
     public RemoteObjects listObjects(String remotePath, String 
continuationToken) throws IOException {
         S3Uri uri = S3Uri.parse(remotePath, usePathStyle);
         ListObjectsV2Request.Builder builder = ListObjectsV2Request.builder()
                 .bucket(uri.bucket())
                 .prefix(uri.key());
-        if (continuationToken != null && !continuationToken.isEmpty()) {
+        if (StringUtils.isNotBlank(continuationToken)) {
             builder.continuationToken(continuationToken);
         }
         try {
@@ -365,7 +249,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
         if (maxKeys > 0) {
             builder.maxKeys(maxKeys);
         }
-        if (continuationToken != null && !continuationToken.isEmpty()) {
+        if (StringUtils.isNotBlank(continuationToken)) {
             builder.continuationToken(continuationToken);
         }
         try {
@@ -399,7 +283,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
                 .bucket(uri.bucket())
                 .prefix(uri.key())
                 .delimiter("/");
-        if (continuationToken != null && !continuationToken.isEmpty()) {
+        if (StringUtils.isNotBlank(continuationToken)) {
             builder.continuationToken(continuationToken);
         }
         try {
@@ -612,27 +496,25 @@ public class S3ObjStorage implements ObjStorage<S3Client> 
{
 
     @Override
     public StsCredentials getStsToken() throws IOException {
-        String roleArn = properties.get(PROP_ROLE_ARN);
-        String externalId = properties.get(PROP_EXTERNAL_ID);
-        if (roleArn == null || roleArn.isEmpty()) {
+        if (StringUtils.isBlank(s3Properties.getRoleArn())) {
             throw new IOException("STS role ARN (AWS_ROLE_ARN) is not 
configured");
         }
-        String region = properties.getOrDefault(PROP_REGION, "us-east-1");
         try {
-            try (StsClient stsClient = 
buildStsClient(buildStsSourceCredentialsProvider(), region)) {
+            try (StsClient stsClient = buildStsClient(
+                    buildStsSourceCredentialsProvider(), 
s3Properties.getRegion())) {
                 AssumeRoleRequest.Builder reqBuilder = 
AssumeRoleRequest.builder()
-                        .roleArn(roleArn)
+                        .roleArn(s3Properties.getRoleArn())
                         .durationSeconds(SESSION_EXPIRE_SECONDS)
                         .roleSessionName("doris_" + 
UUID.randomUUID().toString().replace("-", ""));
-                if (externalId != null && !externalId.isEmpty()) {
-                    reqBuilder.externalId(externalId);
+                if (StringUtils.isNotBlank(s3Properties.getExternalId())) {
+                    reqBuilder.externalId(s3Properties.getExternalId());
                 }
                 AssumeRoleResponse resp = 
stsClient.assumeRole(reqBuilder.build());
                 Credentials cred = resp.credentials();
                 return new StsCredentials(cred.accessKeyId(), 
cred.secretAccessKey(), cred.sessionToken());
             }
         } catch (Exception e) {
-            LOG.warn("Failed to get STS token, roleArn={}", roleArn, e);
+            LOG.warn("Failed to get STS token, roleArn={}", 
s3Properties.getRoleArn(), e);
             throw new IOException("Failed to get STS token: " + 
e.getMessage(), e);
         }
     }
@@ -646,7 +528,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
             ListObjectsV2Request.Builder reqBuilder = 
ListObjectsV2Request.builder()
                     .bucket(bucket)
                     .prefix(fullPrefix);
-            if (continuationToken != null && !continuationToken.isEmpty()) {
+            if (StringUtils.isNotBlank(continuationToken)) {
                 reqBuilder.continuationToken(continuationToken);
             }
             ListObjectsV2Response resp = 
getClient().listObjectsV2(reqBuilder.build());
@@ -692,9 +574,6 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
     @Override
     public String getPresignedUrl(String objectKey) throws IOException {
         requireBucket("getPresignedUrl");
-        String accessKey = properties.get(PROP_ACCESS_KEY);
-        String secretKey = properties.get(PROP_SECRET_KEY);
-        String region = properties.getOrDefault(PROP_REGION, "us-east-1");
         try {
             PutObjectRequest putReq = PutObjectRequest.builder()
                     .bucket(bucket)
@@ -704,10 +583,10 @@ public class S3ObjStorage implements ObjStorage<S3Client> 
{
                     
.signatureDuration(Duration.ofSeconds(SESSION_EXPIRE_SECONDS))
                     .putObjectRequest(putReq)
                     .build();
-            AwsBasicCredentials cred = AwsBasicCredentials.create(accessKey, 
secretKey);
             try (S3Presigner presigner = S3Presigner.builder()
-                    .region(Region.of(region))
-                    
.credentialsProvider(StaticCredentialsProvider.create(cred))
+                    .region(Region.of(s3Properties.getRegion()))
+                    .credentialsProvider(StaticCredentialsProvider.create(
+                            
AwsBasicCredentials.create(s3Properties.getAccessKey(), 
s3Properties.getSecretKey())))
                     .build()) {
                 PresignedPutObjectRequest presigned = 
presigner.presignPutObject(presignReq);
                 LOG.info("Generated S3 presigned URL for key={}", objectKey);
@@ -767,7 +646,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
     // -----------------------------------------------------------------------
 
     private void requireBucket(String operation) throws IOException {
-        if (bucket == null || bucket.isEmpty()) {
+        if (StringUtils.isBlank(bucket)) {
             throw new IOException(operation + " requires AWS_BUCKET to be 
configured");
         }
     }
@@ -792,7 +671,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
 
     @Override
     public Map<String, String> getProperties() {
-        return properties;
+        return s3Properties.toFileSystemKv();
     }
 
     @Override
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3CredentialsProviderFactoryTest.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3CredentialsProviderFactoryTest.java
new file mode 100644
index 00000000000..5232740cbb9
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3CredentialsProviderFactoryTest.java
@@ -0,0 +1,103 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.s3;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain;
+import 
software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+
+import java.lang.reflect.Field;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class S3CredentialsProviderFactoryTest {
+
+    @Test
+    void createClientProvider_staticCredentialsOverrideProviderType() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+        raw.put("s3.credentials_provider_type", "ENV");
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        AwsCredentialsProvider provider = 
S3CredentialsProviderFactory.createClientProvider(properties);
+
+        Assertions.assertInstanceOf(StaticCredentialsProvider.class, provider);
+        Assertions.assertEquals("ak", 
provider.resolveCredentials().accessKeyId());
+    }
+
+    @Test
+    void 
createClientProvider_usesConfiguredProviderTypeWithoutStaticCredentials() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        raw.put("s3.credentials_provider_type", "ENV");
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        AwsCredentialsProvider provider = 
S3CredentialsProviderFactory.createClientProvider(properties);
+
+        
Assertions.assertInstanceOf(EnvironmentVariableCredentialsProvider.class, 
provider);
+    }
+
+    @Test
+    void 
createStsSourceProvider_usesConfiguredProviderTypeWithoutStaticCredentials() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        raw.put("s3.role_arn", "arn:aws:iam::123456789012:role/MyRole");
+        raw.put("s3.credentials_provider_type", "ENV");
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        AwsCredentialsProvider provider = 
S3CredentialsProviderFactory.createStsSourceProvider(properties);
+
+        
Assertions.assertInstanceOf(EnvironmentVariableCredentialsProvider.class, 
provider);
+    }
+
+    @Test
+    void createDefaultProviderChainIncludesProfileCredentialsProvider() {
+        AwsCredentialsProvider provider = S3CredentialsProviderFactory.create(
+                S3CredentialsProviderType.DEFAULT, true);
+
+        Assertions.assertInstanceOf(AwsCredentialsProviderChain.class, 
provider);
+        Assertions.assertTrue(providerClasses((AwsCredentialsProviderChain) 
provider)
+                .contains(ProfileCredentialsProvider.class));
+    }
+
+    @Test
+    void hadoopClassNameDefaultIncludesProfileCredentialsProvider() {
+        String className = S3CredentialsProviderFactory.hadoopClassName(
+                S3CredentialsProviderType.DEFAULT, true);
+
+        
Assertions.assertTrue(className.contains(ProfileCredentialsProvider.class.getName()));
+    }
+
+    private static List<Class<?>> providerClasses(AwsCredentialsProviderChain 
provider) {
+        try {
+            Field field = 
AwsCredentialsProviderChain.class.getDeclaredField("credentialsProviders");
+            field.setAccessible(true);
+            List<?> providers = (List<?>) field.get(provider);
+            return 
providers.stream().map(Object::getClass).collect(java.util.stream.Collectors.toList());
+        } catch (ReflectiveOperationException e) {
+            throw new AssertionError(e);
+        }
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemPropertiesTest.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemPropertiesTest.java
new file mode 100644
index 00000000000..63ebfff76fe
--- /dev/null
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemPropertiesTest.java
@@ -0,0 +1,245 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.filesystem.s3;
+
+import org.apache.doris.filesystem.properties.BackendStorageKind;
+import org.apache.doris.filesystem.properties.BackendStorageProperties;
+import org.apache.doris.filesystem.properties.HadoopStorageProperties;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import 
software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
+
+import java.util.HashMap;
+import java.util.Map;
+
+class S3FileSystemPropertiesTest {
+
+    @Test
+    void of_bindsAliasesAndExposesEffectiveViews() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://minio.local";);
+        raw.put("region", "us-west-2");
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+        raw.put("s3.session-token", "token");
+        raw.put("AWS_BUCKET", "bucket");
+        raw.put("s3.root.path", "root");
+        raw.put("s3.connection.maximum", "64");
+        raw.put("use_path_style", "true");
+
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        Assertions.assertEquals("https://minio.local";, 
properties.getEndpoint());
+        Assertions.assertEquals("us-west-2", properties.getRegion());
+        Assertions.assertEquals("ak", properties.getAccessKey());
+        Assertions.assertEquals("sk", properties.getSecretKey());
+        Assertions.assertEquals("token", properties.getSessionToken());
+        Assertions.assertEquals("bucket", properties.getBucket());
+        Assertions.assertEquals("root", properties.getRootPath());
+
+        Assertions.assertEquals("https://minio.local";, 
properties.matchedProperties().get("s3.endpoint"));
+        Assertions.assertEquals("us-west-2", 
properties.matchedProperties().get("region"));
+        Assertions.assertEquals("ak", 
properties.matchedProperties().get("s3.access_key"));
+
+        Map<String, String> fsKv = properties.toFileSystemKv();
+        Assertions.assertEquals("https://minio.local";, 
fsKv.get("AWS_ENDPOINT"));
+        Assertions.assertEquals("us-west-2", fsKv.get("AWS_REGION"));
+        Assertions.assertEquals("ak", fsKv.get("AWS_ACCESS_KEY"));
+        Assertions.assertEquals("sk", fsKv.get("AWS_SECRET_KEY"));
+        Assertions.assertEquals("token", fsKv.get("AWS_TOKEN"));
+        Assertions.assertEquals("bucket", fsKv.get("AWS_BUCKET"));
+        Assertions.assertEquals("root", fsKv.get("AWS_ROOT_PATH"));
+        Assertions.assertEquals("64", fsKv.get("AWS_MAX_CONNECTIONS"));
+        Assertions.assertEquals("true", fsKv.get("use_path_style"));
+    }
+
+    @Test
+    void of_bindsLegacyS3AliasesToCanonicalFileSystemKv() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("access_key", "ak-bare");
+        raw.put("secret_key", "sk-bare");
+        raw.put("ENDPOINT", "https://endpoint.bare";);
+        raw.put("REGION", "ap-southeast-1");
+        raw.put("session_token", "token");
+
+        Map<String, String> fsKv = 
S3FileSystemProperties.of(raw).toFileSystemKv();
+
+        Assertions.assertEquals("ak-bare", fsKv.get("AWS_ACCESS_KEY"));
+        Assertions.assertEquals("sk-bare", fsKv.get("AWS_SECRET_KEY"));
+        Assertions.assertEquals("https://endpoint.bare";, 
fsKv.get("AWS_ENDPOINT"));
+        Assertions.assertEquals("ap-southeast-1", fsKv.get("AWS_REGION"));
+        Assertions.assertEquals("token", fsKv.get("AWS_TOKEN"));
+    }
+
+    @Test
+    void of_rejectsPartialStaticCredentialsWithParamRulesMessage() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://minio.local";);
+        raw.put("s3.access_key", "ak");
+
+        IllegalArgumentException exception = Assertions.assertThrows(
+                IllegalArgumentException.class, () -> 
S3FileSystemProperties.of(raw));
+
+        Assertions.assertTrue(exception.getMessage().contains("Invalid S3 
filesystem properties"));
+        Assertions.assertTrue(exception.getMessage().contains("s3.access_key 
and s3.secret_key"));
+    }
+
+    @Test
+    void of_rejectsExternalIdWithoutRoleArnWithParamRulesMessage() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://minio.local";);
+        raw.put("s3.external_id", "external");
+
+        IllegalArgumentException exception = Assertions.assertThrows(
+                IllegalArgumentException.class, () -> 
S3FileSystemProperties.of(raw));
+
+        Assertions.assertTrue(exception.getMessage().contains("Invalid S3 
filesystem properties"));
+        Assertions.assertTrue(exception.getMessage().contains("s3.external_id 
must be used together with s3.role_arn"));
+    }
+
+    @Test
+    void of_rejectsMissingLocationWithParamRulesMessage() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+
+        IllegalArgumentException exception = Assertions.assertThrows(
+                IllegalArgumentException.class, () -> 
S3FileSystemProperties.of(raw));
+
+        Assertions.assertTrue(exception.getMessage().contains("Invalid S3 
filesystem properties"));
+        Assertions.assertTrue(exception.getMessage().contains("Either 
s3.endpoint or s3.region must be set"));
+    }
+
+    @Test
+    void of_acceptsEndpointOnlyS3CompatibleConfiguration() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://minio.local";);
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        Assertions.assertEquals("https://minio.local";, 
properties.getEndpoint());
+        Assertions.assertEquals("us-east-1", properties.getRegion());
+        Assertions.assertEquals("https://minio.local";, 
properties.toFileSystemKv().get("AWS_ENDPOINT"));
+        Assertions.assertEquals("us-east-1", 
properties.toFileSystemKv().get("AWS_REGION"));
+    }
+
+    @Test
+    void of_acceptsRegionOnlyS3Configuration() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.region", "us-west-2");
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        Assertions.assertEquals("us-west-2", properties.getRegion());
+        Assertions.assertEquals("https://s3.us-west-2.amazonaws.com";, 
properties.getEndpoint());
+    }
+
+    @Test
+    void of_derivesRegionFromAwsEndpoint() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        Assertions.assertEquals("us-west-2", properties.getRegion());
+        Assertions.assertEquals("us-west-2", 
properties.toFileSystemKv().get("AWS_REGION"));
+    }
+
+    @Test
+    void toBackendProperties_returnsLegacyAwsBackendMapForAdapters() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://minio.local";);
+        raw.put("s3.region", "us-west-2");
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+        raw.put("s3.bucket", "bucket");
+        raw.put("s3.root.path", "root");
+        raw.put("use_path_style", "true");
+
+        BackendStorageProperties backend = S3FileSystemProperties.of(raw)
+                .toBackendProperties()
+                .orElseThrow();
+
+        Assertions.assertEquals(BackendStorageKind.S3_COMPATIBLE, 
backend.backendKind());
+        Assertions.assertEquals("https://minio.local";, 
backend.toMap().get("AWS_ENDPOINT"));
+        Assertions.assertEquals("us-west-2", 
backend.toMap().get("AWS_REGION"));
+        Assertions.assertEquals("ak", backend.toMap().get("AWS_ACCESS_KEY"));
+        Assertions.assertEquals("sk", backend.toMap().get("AWS_SECRET_KEY"));
+        Assertions.assertEquals("bucket", backend.toMap().get("AWS_BUCKET"));
+        Assertions.assertEquals("root", backend.toMap().get("AWS_ROOT_PATH"));
+        Assertions.assertEquals("true", backend.toMap().get("use_path_style"));
+    }
+
+    @Test
+    void toHadoopProperties_returnsS3AConfigurationMap() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://minio.local";);
+        raw.put("s3.region", "us-west-2");
+        raw.put("s3.access_key", "ak");
+        raw.put("s3.secret_key", "sk");
+        raw.put("s3.session_token", "token");
+        raw.put("use_path_style", "true");
+
+        HadoopStorageProperties hadoop = S3FileSystemProperties.of(raw)
+                .toHadoopProperties()
+                .orElseThrow();
+
+        Map<String, String> hadoopMap = hadoop.toHadoopConfigurationMap();
+        Assertions.assertEquals("org.apache.hadoop.fs.s3a.S3AFileSystem", 
hadoopMap.get("fs.s3a.impl"));
+        Assertions.assertEquals("https://minio.local";, 
hadoopMap.get("fs.s3a.endpoint"));
+        Assertions.assertEquals("us-west-2", 
hadoopMap.get("fs.s3a.endpoint.region"));
+        Assertions.assertEquals("ak", hadoopMap.get("fs.s3a.access.key"));
+        Assertions.assertEquals("sk", hadoopMap.get("fs.s3a.secret.key"));
+        Assertions.assertEquals("token", 
hadoopMap.get("fs.s3a.session.token"));
+        Assertions.assertEquals("true", 
hadoopMap.get("fs.s3a.path.style.access"));
+    }
+
+    @Test
+    void of_bindsAndNormalizesCredentialsProviderType() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        raw.put("s3.credentials_provider_type", "environment");
+
+        S3FileSystemProperties properties = S3FileSystemProperties.of(raw);
+
+        Assertions.assertEquals(S3CredentialsProviderType.ENV, 
properties.getCredentialsProviderType());
+        Assertions.assertEquals("ENV", 
properties.toFileSystemKv().get("AWS_CREDENTIALS_PROVIDER_TYPE"));
+        
Assertions.assertEquals(EnvironmentVariableCredentialsProvider.class.getName(),
+                
properties.toHadoopConfigurationMap().get("fs.s3a.aws.credentials.provider"));
+    }
+
+    @Test
+    void of_rejectsUnsupportedCredentialsProviderType() {
+        Map<String, String> raw = new HashMap<>();
+        raw.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        raw.put("s3.credentials_provider_type", "bad-provider");
+
+        IllegalArgumentException exception = Assertions.assertThrows(
+                IllegalArgumentException.class, () -> 
S3FileSystemProperties.of(raw));
+
+        Assertions.assertTrue(exception.getMessage().contains("Invalid S3 
filesystem properties"));
+        Assertions.assertTrue(exception.getMessage().contains("Unsupported 
s3.credentials_provider_type"));
+    }
+}
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemProviderTest.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemProviderTest.java
index afee6ff09b2..fe5c21d9515 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemProviderTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3FileSystemProviderTest.java
@@ -17,6 +17,9 @@
 
 package org.apache.doris.filesystem.s3;
 
+import org.apache.doris.filesystem.FileSystem;
+import org.apache.doris.filesystem.properties.FileSystemProperties;
+
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
@@ -37,6 +40,16 @@ class S3FileSystemProviderTest {
         Assertions.assertTrue(provider.supports(props));
     }
 
+    @Test
+    void supports_acceptsConfiguredCredentialsProviderType() {
+        Map<String, String> props = new HashMap<>();
+        props.put("AWS_ENDPOINT", "https://s3.us-west-2.amazonaws.com";);
+        props.put("AWS_REGION", "us-west-2");
+        props.put("s3.credentials_provider_type", "ENV");
+
+        Assertions.assertTrue(provider.supports(props));
+    }
+
     @Test
     void supports_rejectsConfigurationWithoutCredentialsOrRole() {
         Map<String, String> props = new HashMap<>();
@@ -45,4 +58,64 @@ class S3FileSystemProviderTest {
 
         Assertions.assertFalse(provider.supports(props));
     }
+
+    @Test
+    void supports_acceptsLegacyConvertedMapWithoutExplicitCredentials() {
+        Map<String, String> props = new HashMap<>();
+        props.put("_STORAGE_TYPE_", "S3");
+        props.put("AWS_ENDPOINT", "https://s3.us-west-2.amazonaws.com";);
+        props.put("AWS_REGION", "us-west-2");
+
+        Assertions.assertTrue(provider.supports(props));
+    }
+
+    @Test
+    void supports_acceptsExplicitS3ProviderWithoutCredentials() {
+        Map<String, String> props = new HashMap<>();
+        props.put("provider", "S3");
+        props.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        props.put("s3.region", "us-west-2");
+
+        Assertions.assertTrue(provider.supports(props));
+    }
+
+    @Test
+    void supports_acceptsExplicitS3SupportWithoutCredentials() {
+        Map<String, String> props = new HashMap<>();
+        props.put("fs.s3.support", "true");
+        props.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        props.put("s3.region", "us-west-2");
+
+        Assertions.assertTrue(provider.supports(props));
+    }
+
+    @Test
+    void bind_returnsValidatedS3FileSystemProperties() {
+        Map<String, String> props = new HashMap<>();
+        props.put("s3.endpoint", "https://minio.local";);
+        props.put("s3.region", "us-west-2");
+        props.put("s3.access_key", "ak");
+        props.put("s3.secret_key", "sk");
+
+        FileSystemProperties bound = provider.bind(props);
+
+        Assertions.assertInstanceOf(S3FileSystemProperties.class, bound);
+        Assertions.assertEquals("https://minio.local";, 
((S3FileSystemProperties) bound).getEndpoint());
+    }
+
+    @Test
+    void create_usesTypedS3FileSystemProperties() throws Exception {
+        Map<String, String> props = new HashMap<>();
+        props.put("s3.endpoint", "https://minio.local";);
+        props.put("s3.region", "us-west-2");
+        props.put("s3.access_key", "ak");
+        props.put("s3.secret_key", "sk");
+
+        FileSystem fileSystem = provider.create(provider.bind(props));
+
+        Assertions.assertInstanceOf(S3FileSystem.class, fileSystem);
+        S3FileSystem s3 = (S3FileSystem) fileSystem;
+        Assertions.assertTrue(s3.properties().isPresent());
+        Assertions.assertEquals("https://minio.local";, 
s3.properties().orElseThrow().getEndpoint());
+    }
 }
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageMockTest.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageMockTest.java
index e102581a46b..1ef98d4bea8 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageMockTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageMockTest.java
@@ -28,7 +28,9 @@ import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatchers;
 import org.mockito.Mockito;
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import 
software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
 import software.amazon.awssdk.services.s3.S3Client;
 import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
 import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
@@ -391,6 +393,7 @@ class S3ObjStorageMockTest {
     void getPresignedUrl_throwsWhenBucketNotConfigured() {
         Map<String, String> noBucketProps = new HashMap<>();
         noBucketProps.put("AWS_ENDPOINT", "https://s3.amazonaws.com";);
+        noBucketProps.put("AWS_REGION", "us-east-1");
         noBucketProps.put("AWS_ACCESS_KEY", "ak");
         noBucketProps.put("AWS_SECRET_KEY", "sk");
         S3ObjStorage noBucket = new TestableS3ObjStorage(noBucketProps, 
mockS3);
@@ -424,6 +427,45 @@ class S3ObjStorageMockTest {
                 credentialsProvider.getClass().getSimpleName());
     }
 
+    @Test
+    void buildCredentialsProvider_usesTypedCredentialsProviderType() {
+        Map<String, String> props = new HashMap<>();
+        props.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        props.put("s3.credentials_provider_type", "ENV");
+        S3FileSystemProperties properties = S3FileSystemProperties.of(props);
+        InspectableS3ObjStorage typedStorage = new 
InspectableS3ObjStorage(properties, mockS3);
+
+        AwsCredentialsProvider credentialsProvider = 
typedStorage.inspectBuildCredentialsProvider();
+
+        
Assertions.assertInstanceOf(EnvironmentVariableCredentialsProvider.class, 
credentialsProvider);
+    }
+
+    @Test
+    void buildCredentialsProvider_usesAnonymousCredentialsProviderType() {
+        Map<String, String> props = new HashMap<>();
+        props.put("s3.endpoint", "https://s3.us-west-2.amazonaws.com";);
+        props.put("s3.credentials_provider_type", "ANONYMOUS");
+        S3FileSystemProperties properties = S3FileSystemProperties.of(props);
+        InspectableS3ObjStorage typedStorage = new 
InspectableS3ObjStorage(properties, mockS3);
+
+        AwsCredentialsProvider credentialsProvider = 
typedStorage.inspectBuildCredentialsProvider();
+
+        Assertions.assertInstanceOf(AnonymousCredentialsProvider.class, 
credentialsProvider);
+    }
+
+    @Test
+    void 
buildCredentialsProvider_usesBackendCredentialsProviderTypeFromLegacyMap() {
+        Map<String, String> props = new HashMap<>();
+        props.put("AWS_ENDPOINT", "https://s3.us-west-2.amazonaws.com";);
+        props.put("AWS_REGION", "us-west-2");
+        props.put("AWS_CREDENTIALS_PROVIDER_TYPE", "ENV");
+        InspectableS3ObjStorage mapStorage = new 
InspectableS3ObjStorage(props, mockS3);
+
+        AwsCredentialsProvider credentialsProvider = 
mapStorage.inspectBuildCredentialsProvider();
+
+        
Assertions.assertInstanceOf(EnvironmentVariableCredentialsProvider.class, 
credentialsProvider);
+    }
+
     // ------------------------------------------------------------------
     // close()
     // ------------------------------------------------------------------
@@ -436,37 +478,6 @@ class S3ObjStorageMockTest {
         Mockito.verify(mockS3).close();
     }
 
-    // ------------------------------------------------------------------
-    // buildClient() region fallback (#23)
-    // ------------------------------------------------------------------
-
-    /**
-     * #23: when no region is configured, {@code buildClient()} must NOT throw 
— it logs a
-     * deprecation WARN and falls back to {@code us-east-1} (used solely for 
SigV4 signing).
-     * This preserves backward compatibility for existing clusters that rely 
on the implicit
-     * default; the warning is the migration signal.
-     */
-    @Test
-    void buildClient_missingRegionLogsWarnAndFallsBack() throws IOException {
-        Map<String, String> props = new HashMap<>();
-        // Endpoint set so SDK does not need to resolve us-east-1 against the 
AWS DNS.
-        props.put("AWS_ENDPOINT", "https://s3.example.com";);
-        props.put("AWS_ACCESS_KEY", "ak");
-        props.put("AWS_SECRET_KEY", "sk");
-        props.put("AWS_BUCKET", "bucket");
-        // Intentionally no AWS_REGION / s3.region / region / REGION.
-
-        S3ObjStorage real = new S3ObjStorage(props);
-        // The real buildClient must succeed without throwing — that proves we 
took the WARN
-        // route rather than the throw route. (The WARN itself is asserted by 
inspection /
-        // operator log review; capturing log4j2 output here would couple the 
test to the
-        // logging backend without adding correctness signal.)
-        S3Client client = Assertions.assertDoesNotThrow(real::buildClient,
-                "buildClient() must not throw when region is missing");
-        Assertions.assertNotNull(client);
-        client.close();
-    }
-
     // ------------------------------------------------------------------
     // Test infrastructure
     // ------------------------------------------------------------------
@@ -479,6 +490,11 @@ class S3ObjStorageMockTest {
             this.mockClient = mockClient;
         }
 
+        TestableS3ObjStorage(S3FileSystemProperties properties, S3Client 
mockClient) {
+            super(properties);
+            this.mockClient = mockClient;
+        }
+
         @Override
         protected S3Client buildClient() {
             return mockClient;
@@ -490,6 +506,10 @@ class S3ObjStorageMockTest {
             super(properties, mockClient);
         }
 
+        InspectableS3ObjStorage(S3FileSystemProperties properties, S3Client 
mockClient) {
+            super(properties, mockClient);
+        }
+
         AwsCredentialsProvider inspectBuildCredentialsProvider() {
             return buildCredentialsProvider();
         }
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageTest.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageTest.java
index 5b80c1afa66..ecba33b4bf8 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3ObjStorageTest.java
@@ -19,136 +19,26 @@ package org.apache.doris.filesystem.s3;
 
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
 
 import java.util.HashMap;
 import java.util.Map;
 
-/**
- * Unit tests for {@link S3ObjStorage} focusing on property normalization
- * and constructor behavior. Does not require real AWS credentials — the
- * S3 client is never built.
- */
+/** Unit tests for {@link S3ObjStorage} constructor behavior. */
 class S3ObjStorageTest {
 
-    // ------------------------------------------------------------------
-    // normalizeProperties()
-    // ------------------------------------------------------------------
-
-    @Test
-    void normalizeProperties_canonicalKeysAlreadyPresent() {
-        Map<String, String> props = new HashMap<>();
-        props.put("AWS_ACCESS_KEY", "canonical-ak");
-        props.put("AWS_SECRET_KEY", "canonical-sk");
-        props.put("AWS_ENDPOINT", "https://s3.amazonaws.com";);
-        props.put("AWS_REGION", "us-east-1");
-        props.put("AWS_TOKEN", "tok");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertEquals("canonical-ak", result.get("AWS_ACCESS_KEY"));
-        Assertions.assertEquals("canonical-sk", result.get("AWS_SECRET_KEY"));
-        Assertions.assertEquals("https://s3.amazonaws.com";, 
result.get("AWS_ENDPOINT"));
-        Assertions.assertEquals("us-east-1", result.get("AWS_REGION"));
-        Assertions.assertEquals("tok", result.get("AWS_TOKEN"));
-    }
-
-    @Test
-    void normalizeProperties_s3DotPrefixAliasesNormalized() {
-        Map<String, String> props = new HashMap<>();
-        props.put("s3.access_key", "ak-from-s3-prefix");
-        props.put("s3.secret_key", "sk-from-s3-prefix");
-        props.put("s3.endpoint", "https://minio.local";);
-        props.put("s3.region", "us-west-2");
-        props.put("s3.session_token", "sess-tok");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertEquals("ak-from-s3-prefix", 
result.get("AWS_ACCESS_KEY"));
-        Assertions.assertEquals("sk-from-s3-prefix", 
result.get("AWS_SECRET_KEY"));
-        Assertions.assertEquals("https://minio.local";, 
result.get("AWS_ENDPOINT"));
-        Assertions.assertEquals("us-west-2", result.get("AWS_REGION"));
-        Assertions.assertEquals("sess-tok", result.get("AWS_TOKEN"));
-    }
-
-    @Test
-    void normalizeProperties_bareAliasesNormalized() {
-        Map<String, String> props = new HashMap<>();
-        props.put("access_key", "ak-bare");
-        props.put("secret_key", "sk-bare");
-        props.put("endpoint", "https://endpoint.bare";);
-        props.put("region", "ap-southeast-1");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertEquals("ak-bare", result.get("AWS_ACCESS_KEY"));
-        Assertions.assertEquals("sk-bare", result.get("AWS_SECRET_KEY"));
-        Assertions.assertEquals("https://endpoint.bare";, 
result.get("AWS_ENDPOINT"));
-        Assertions.assertEquals("ap-southeast-1", result.get("AWS_REGION"));
-    }
-
-    @Test
-    void normalizeProperties_uppercaseAliasesNormalized() {
-        Map<String, String> props = new HashMap<>();
-        props.put("ACCESS_KEY", "ak-upper");
-        props.put("SECRET_KEY", "sk-upper");
-        props.put("ENDPOINT", "https://upper.endpoint";);
-        props.put("REGION", "eu-west-1");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertEquals("ak-upper", result.get("AWS_ACCESS_KEY"));
-        Assertions.assertEquals("sk-upper", result.get("AWS_SECRET_KEY"));
-        Assertions.assertEquals("https://upper.endpoint";, 
result.get("AWS_ENDPOINT"));
-        Assertions.assertEquals("eu-west-1", result.get("AWS_REGION"));
-    }
-
-    @Test
-    void normalizeProperties_canonicalKeyTakesPrecedenceOverAlias() {
-        Map<String, String> props = new HashMap<>();
-        props.put("AWS_ACCESS_KEY", "canonical");
-        props.put("s3.access_key", "alias-should-be-ignored");
-        props.put("access_key", "bare-should-be-ignored");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertEquals("canonical", result.get("AWS_ACCESS_KEY"));
-    }
-
-    @Test
-    void normalizeProperties_firstMatchingAliasWins() {
-        // s3.access_key comes before access_key in alias order
-        Map<String, String> props = new HashMap<>();
-        props.put("s3.access_key", "s3-prefix-wins");
-        props.put("access_key", "bare-loses");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertEquals("s3-prefix-wins", 
result.get("AWS_ACCESS_KEY"));
-    }
-
-    @Test
-    void normalizeProperties_noMatchingAliasLeavesNull() {
-        Map<String, String> props = new HashMap<>();
-        props.put("unrelated_key", "value");
-
-        Map<String, String> result = S3ObjStorage.normalizeProperties(props);
-
-        Assertions.assertNull(result.get("AWS_ACCESS_KEY"));
-        Assertions.assertNull(result.get("AWS_SECRET_KEY"));
-        Assertions.assertNull(result.get("AWS_ENDPOINT"));
-        Assertions.assertNull(result.get("AWS_REGION"));
-    }
-
     // ------------------------------------------------------------------
     // Constructor & getProperties()
     // ------------------------------------------------------------------
 
     @Test
-    void constructor_propertiesAreNormalizedAndImmutable() {
+    void constructor_mapInputBindsThroughS3FileSystemProperties() {
         Map<String, String> props = new HashMap<>();
         props.put("s3.access_key", "ak");
         props.put("s3.secret_key", "sk");
         props.put("s3.endpoint", "https://ep";);
+        props.put("s3.region", "us-east-1");
         props.put("AWS_BUCKET", "my-bucket");
 
         S3ObjStorage storage = new S3ObjStorage(props);
@@ -157,16 +47,48 @@ class S3ObjStorageTest {
         Assertions.assertEquals("ak", stored.get("AWS_ACCESS_KEY"));
         Assertions.assertEquals("sk", stored.get("AWS_SECRET_KEY"));
         Assertions.assertEquals("https://ep";, stored.get("AWS_ENDPOINT"));
+        Assertions.assertEquals("us-east-1", stored.get("AWS_REGION"));
         Assertions.assertEquals("my-bucket", stored.get("AWS_BUCKET"));
 
         Assertions.assertThrows(UnsupportedOperationException.class, () -> 
stored.put("new", "val"),
                 "getProperties() should return unmodifiable map");
     }
 
+    @Test
+    void constructor_acceptsEndpointOnlyConfiguration() {
+        Map<String, String> props = new HashMap<>();
+        props.put("AWS_ENDPOINT", "https://minio.local";);
+        props.put("AWS_ACCESS_KEY", "ak");
+        props.put("AWS_SECRET_KEY", "sk");
+
+        S3ObjStorage storage = new S3ObjStorage(props);
+        Map<String, String> stored = storage.getProperties();
+
+        Assertions.assertEquals("https://minio.local";, 
stored.get("AWS_ENDPOINT"));
+        Assertions.assertEquals("us-east-1", stored.get("AWS_REGION"));
+    }
+
+    @Test
+    void getClient_endpointOnlyConfigurationUsesRegionBuiltByProperties() 
throws Exception {
+        Map<String, String> props = new HashMap<>();
+        props.put("AWS_ENDPOINT", "https://minio.local";);
+        props.put("AWS_ACCESS_KEY", "ak");
+        props.put("AWS_SECRET_KEY", "sk");
+
+        S3ObjStorage storage = new S3ObjStorage(props);
+        S3Client client = storage.getClient();
+        try {
+            Assertions.assertEquals(Region.US_EAST_1, 
client.serviceClientConfiguration().region());
+        } finally {
+            storage.close();
+        }
+    }
+
     @Test
     void constructor_usePathStyleDefaultsFalse() {
         Map<String, String> props = new HashMap<>();
         props.put("AWS_ENDPOINT", "https://s3.amazonaws.com";);
+        props.put("AWS_REGION", "us-east-1");
 
         S3ObjStorage storage = new S3ObjStorage(props);
         Map<String, String> stored = storage.getProperties();
@@ -178,6 +100,7 @@ class S3ObjStorageTest {
     void constructor_usePathStyleTrueWhenSet() {
         Map<String, String> props = new HashMap<>();
         props.put("AWS_ENDPOINT", "https://minio.local";);
+        props.put("AWS_REGION", "us-west-2");
         props.put("use_path_style", "true");
 
         S3ObjStorage storage = new S3ObjStorage(props);
@@ -190,7 +113,9 @@ class S3ObjStorageTest {
     void constructor_originalMapMutationDoesNotAffectStorage() {
         Map<String, String> props = new HashMap<>();
         props.put("AWS_ACCESS_KEY", "original");
+        props.put("AWS_SECRET_KEY", "secret");
         props.put("AWS_ENDPOINT", "https://ep";);
+        props.put("AWS_REGION", "us-east-1");
 
         S3ObjStorage storage = new S3ObjStorage(props);
         props.put("AWS_ACCESS_KEY", "mutated");
@@ -207,6 +132,7 @@ class S3ObjStorageTest {
     void close_doesNotThrowWhenClientNotBuilt() throws Exception {
         Map<String, String> props = new HashMap<>();
         props.put("AWS_ENDPOINT", "https://ep";);
+        props.put("AWS_REGION", "us-east-1");
 
         S3ObjStorage storage = new S3ObjStorage(props);
         storage.close();
diff --git 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3OutputStreamTest.java
 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3OutputStreamTest.java
index 9f85bd88421..d618dc9ed82 100644
--- 
a/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3OutputStreamTest.java
+++ 
b/fe/fe-filesystem/fe-filesystem-s3/src/test/java/org/apache/doris/filesystem/s3/S3OutputStreamTest.java
@@ -265,7 +265,9 @@ class S3OutputStreamTest {
     private static class CapturingStorage extends S3ObjStorage {
 
         CapturingStorage() {
-            super(Map.of()); // empty properties — no AWS client is built
+            super(Map.of(
+                    "s3.endpoint", "https://minio.local";,
+                    "s3.region", "us-east-1"));
         }
 
         @Override
diff --git 
a/fe/fe-filesystem/fe-filesystem-spi/src/main/java/org/apache/doris/filesystem/spi/FileSystemProvider.java
 
b/fe/fe-filesystem/fe-filesystem-spi/src/main/java/org/apache/doris/filesystem/spi/FileSystemProvider.java
index 41c57e479e1..a5cb00f13b7 100644
--- 
a/fe/fe-filesystem/fe-filesystem-spi/src/main/java/org/apache/doris/filesystem/spi/FileSystemProvider.java
+++ 
b/fe/fe-filesystem/fe-filesystem-spi/src/main/java/org/apache/doris/filesystem/spi/FileSystemProvider.java
@@ -20,6 +20,7 @@ package org.apache.doris.filesystem.spi;
 import org.apache.doris.extension.spi.Plugin;
 import org.apache.doris.extension.spi.PluginFactory;
 import org.apache.doris.filesystem.FileSystem;
+import org.apache.doris.filesystem.properties.FileSystemProperties;
 
 import java.io.IOException;
 import java.util.Map;
@@ -48,6 +49,28 @@ public interface FileSystemProvider extends PluginFactory {
      */
     boolean supports(Map<String, String> properties);
 
+    /**
+     * Binds raw key-value storage configuration into a provider-owned typed 
properties model.
+     *
+     * <p>Providers that have been migrated to typed properties should 
override this method and
+     * return a validated immutable properties object. Legacy providers can 
continue to implement
+     * {@link #create(Map)} directly during the migration period.
+     */
+    default FileSystemProperties bind(Map<String, String> properties) {
+        throw new UnsupportedOperationException(
+                name() + " does not support typed FileSystemProperties binding 
yet.");
+    }
+
+    /**
+     * Creates a FileSystem instance from validated typed properties.
+     *
+     * <p>The default implementation preserves compatibility for providers 
whose typed
+     * properties can still be represented as legacy FileSystem key-value 
pairs.
+     */
+    default FileSystem create(FileSystemProperties properties) throws 
IOException {
+        return create(properties.toFileSystemKv());
+    }
+
     /**
      * Creates a FileSystem instance from the given properties.
      * Called only after {@link #supports(Map)} returns true.


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to