diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt
index cc9932a89b13..46a66768c54c 100644
--- a/contrib/aws-cmake/CMakeLists.txt
+++ b/contrib/aws-cmake/CMakeLists.txt
@@ -57,6 +57,10 @@ SET(AWS_SDK_S3_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-s3")
 SET(AWS_SDK_KMS_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-kms")
 SET(AWS_SDK_GLUE_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-glue")
 
+SET(AWS_SDK_IDENTITY_MANAGEMENT_DIR "${AWS_SDK_DIR}/src/aws-cpp-sdk-identity-management")
+SET(AWS_SDK_STS_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-sts")
+SET(AWS_SDK_COGNITO_IDENTITY_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-cognito-identity")
+
 SET(AWS_AUTH_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-auth")
 SET(AWS_CAL_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-cal")
 SET(AWS_CHECKSUMS_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-checksums")
@@ -145,6 +149,34 @@ list(APPEND AWS_SOURCES ${AWS_SDK_S3_SRC})
 list(APPEND AWS_PUBLIC_INCLUDES "${AWS_SDK_S3_DIR}/include/")
 
 
+# aws-cpp-sdk-identity-management
+file(GLOB AWS_SDK_IDENTITY_MANAGEMENT_SRC
+    "${AWS_SDK_IDENTITY_MANAGEMENT_DIR}/source/auth/*.cpp"
+)
+
+list(APPEND AWS_SOURCES ${AWS_SDK_IDENTITY_MANAGEMENT_SRC})
+list(APPEND AWS_PUBLIC_INCLUDES "${AWS_SDK_IDENTITY_MANAGEMENT_DIR}/include/")
+
+
+# aws-cpp-sdk-sts
+file(GLOB AWS_SDK_STS_SRC
+        "${AWS_SDK_STS_DIR}/source/*.cpp"
+        "${AWS_SDK_STS_DIR}/source/model/*.cpp"
+)
+
+list(APPEND AWS_SOURCES ${AWS_SDK_STS_SRC})
+list(APPEND AWS_PUBLIC_INCLUDES "${AWS_SDK_STS_DIR}/include/")
+
+# aws-cpp-sdk-cognito-identity
+file(GLOB AWS_SDK_COGNITO_IDENTITY_SRC
+        "${AWS_SDK_COGNITO_IDENTITY_DIR}/source/*.cpp"
+        "${AWS_SDK_COGNITO_IDENTITY_DIR}/source/model/*.cpp"
+)
+
+list(APPEND AWS_SOURCES ${AWS_SDK_COGNITO_IDENTITY_SRC})
+list(APPEND AWS_PRIVATE_INCLUDES "${AWS_SDK_COGNITO_IDENTITY_DIR}/include/")
+
+
 if(CLICKHOUSE_CLOUD)
     # aws-cpp-sdk-kms
     file(GLOB AWS_SDK_KMS_SRC
diff --git a/contrib/sparse-checkout/update-aws.sh b/contrib/sparse-checkout/update-aws.sh
index 735204e12eeb..ee8a27264dbb 100755
--- a/contrib/sparse-checkout/update-aws.sh
+++ b/contrib/sparse-checkout/update-aws.sh
@@ -6,10 +6,13 @@ FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout
 echo '/*' > $FILES_TO_CHECKOUT
 echo '!/*/*' >> $FILES_TO_CHECKOUT
 echo '/src/aws-cpp-sdk-core/*' >> $FILES_TO_CHECKOUT
+echo '/src/aws-cpp-sdk-identity-management/*' >> $FILES_TO_CHECKOUT
 echo '/generated/src/aws-cpp-sdk-s3/*' >> $FILES_TO_CHECKOUT
 echo '/generated/src/aws-cpp-sdk-aws/*' >> $FILES_TO_CHECKOUT
 echo '/generated/src/aws-cpp-sdk-glue/*' >> $FILES_TO_CHECKOUT
 echo '/generated/src/aws-cpp-sdk-kms/*' >> $FILES_TO_CHECKOUT
+echo '/generated/src/aws-cpp-sdk-sts/*' >> $FILES_TO_CHECKOUT
+echo '/generated/src/aws-cpp-sdk-cognito-identity/*' >> $FILES_TO_CHECKOUT
 
 git config core.sparsecheckout true
 git checkout $1
diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md
index aaa237057c62..2590bb27d569 100644
--- a/docs/en/sql-reference/table-functions/s3.md
+++ b/docs/en/sql-reference/table-functions/s3.md
@@ -22,8 +22,8 @@ When using the `s3 table function` with [`INSERT INTO...SELECT`](../../sql-refer
 
 ## Syntax {#syntax}
 
-```sql
-s3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,structure] [,compression_method],[,headers])
+``` sql
+s3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,structure] [,compression_method] [,headers] [,extra_credentials])
 s3(named_collection[, option=value [,..]])
 ```
 
@@ -47,6 +47,7 @@ For GCS, substitute your HMAC key and HMAC secret where you see `access_key_id`
 | `structure`                             | Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`.                                                                                                          |
 | `compression_method`                    | Parameter is optional. Supported values: `none`, `gzip` or `gz`, `brotli` or `br`, `xz` or `LZMA`, `zstd` or `zst`. By default, it will autodetect compression method by file extension.                 |
 | `headers`                               | Parameter is optional. Allows headers to be passed in the S3 request. Pass in the format `headers(key=value)` e.g. `headers('x-amz-request-payer' = 'requester')`.                                    |
+| `extra_credentials`                     | Parameter is optional. Allows to specify role ARN and role session name for AssumeRole (see below). Pass in the format `extra_credentials(key=value)`.                                    |
 
 :::note GCS
 The GCS url is in this format as the endpoint for the Google XML API is different than the JSON API:
@@ -280,7 +281,51 @@ Once configured, a `roleARN` can be passed to the s3 function via an `extra_cred
 SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv','CSVWithNames',extra_credentials(role_arn = 'arn:aws:iam::111111111111:role/ClickHouseAccessRole-001'))
 ```
 
-Further examples can be found [here](/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role)
+## Role Assumption
+
+ClickHouse supports assuming an AWS IAM role using a set of AWS credentials (`access_key_id`, `secret_access_key`, `session_token`) or EC2 metadata (only when running on EC2 instance).
+This allows ClickHouse to obtain temporary credentials for accessing an S3 bucket, even if the original credentials or instance do not have direct access.
+
+For example, if the provided credentials have permission to assume a role but lack direct access to the S3 bucket, ClickHouse will first request temporary credentials from AWS STS and then use those credentials to access S3.
+
+For more details on role assumption, read [AWS AssumeRole documentation](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html).
+
+To use this mechanism, pass parameters via the `extra_credentials` argument to the `s3` function. The following keys are supported:
+
+* `role_arn` (required) — ARN of the IAM role to assume. **If this key is not provided, ClickHouse will not attempt to assume a role and will try to access the bucket as-is.**
+* `role_session_name` (optional) — Custom session name to include in the AssumeRole request. If not specified, a random UUID will be assigned.
+* `sts_endpoint_override` (optional) — Overrides the default AWS STS endpoint (https://sts.amazonaws.com). Useful for testing with a mock or when using another STS-compatible service.
+
+If explicit `access_key_id` and `secret_access_key` are provided as parameters to `s3(...)` function, then they will be used for retrieving temporary credentials from STS:
+
+```sql
+SELECT count() FROM s3(
+    '<s3_bucket_uri>/*.csv',
+    access_key_id,
+    secret_access_key,
+    'CSVWithNames',
+    extra_credentials(
+        role_arn = 'arn:aws:iam::111111111111:role/BucketAccessRole-001',
+        role_session_name = 'ClickHouseSession',
+        sts_endpoint_override = 'http://mock-sts:8080'
+    )
+)
+```
+
+Otherwise, ClickHouse will attempt to extract credentials from EC2 metadata:
+
+```sql
+SELECT count() FROM s3(
+    '<s3_bucket_uri>/*.csv',
+    'CSVWithNames',
+    extra_credentials(
+        role_arn = 'arn:aws:iam::111111111111:role/BucketAccessRole-001',
+        role_session_name = 'ClickHouseSession'
+    )
+)
+```
+
+Further examples can be found [here](/docs/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role)
 
 ## Working with archives {#working-with-archives}
 
diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp
index 6117827868ee..34ee3ce61ddf 100644
--- a/src/Disks/ObjectStorages/S3/diskSettings.cpp
+++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp
@@ -50,6 +50,9 @@ namespace S3AuthSetting
     extern const S3AuthSettingsString secret_access_key;
     extern const S3AuthSettingsString server_side_encryption_customer_key_base64;
     extern const S3AuthSettingsString session_token;
+    extern const S3AuthSettingsString role_arn;
+    extern const S3AuthSettingsString role_session_name;
+    extern const S3AuthSettingsString sts_endpoint_override;
     extern const S3AuthSettingsBool use_adaptive_timeouts;
     extern const S3AuthSettingsBool use_environment_credentials;
     extern const S3AuthSettingsBool use_insecure_imds_request;
@@ -175,6 +178,9 @@ std::unique_ptr<S3::Client> getClient(
         auth_settings[S3AuthSetting::use_insecure_imds_request],
         auth_settings[S3AuthSetting::expiration_window_seconds],
         auth_settings[S3AuthSetting::no_sign_request],
+        auth_settings[S3AuthSetting::role_arn],
+        auth_settings[S3AuthSetting::role_session_name],
+        auth_settings[S3AuthSetting::sts_endpoint_override]
     };
 
     return S3::ClientFactory::instance().create(
diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp
index 37cec351aba4..bf34c4323c84 100644
--- a/src/IO/S3/Credentials.cpp
+++ b/src/IO/S3/Credentials.cpp
@@ -39,6 +39,9 @@ namespace S3
 #    include <aws/core/utils/UUID.h>
 #    include <aws/core/http/HttpClientFactory.h>
 
+#    include <aws/sts/STSClient.h>
+#    include <aws/identity-management/auth/STSAssumeRoleCredentialsProvider.h>
+
 #    include <aws/core/utils/HashingUtils.h>
 #    include <aws/core/platform/FileSystem.h>
 
@@ -560,6 +563,86 @@ void AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::refreshIfExpired()
     Reload();
 }
 
+AWSInstanceMetadataAssumeRoleCredentialsProvider::AWSInstanceMetadataAssumeRoleCredentialsProvider(
+    const Aws::String & role_arn_,
+    const Aws::String & session_name_,
+    DB::S3::PocoHTTPClientConfiguration aws_client_configuration,
+    uint64_t expiration_window_seconds_)
+    : role_arn(role_arn_)
+    , session_name(session_name_.empty() ? Aws::String(Aws::Utils::UUID::RandomUUID()) : session_name_)
+    , logger(getLogger("AWSInstanceMetadataAssumeRoleCredentialsProvider"))
+    , expiration_window_seconds(expiration_window_seconds_)
+{
+    // Create metadata credentials provider
+    auto ec2_metadata_client = createEC2MetadataClient(aws_client_configuration);
+    auto config_loader = std::make_shared<AWSEC2InstanceProfileConfigLoader>(ec2_metadata_client, true);
+    metadata_provider = std::make_shared<AWSInstanceProfileCredentialsProvider>(config_loader);
+
+    aws_client_configuration.scheme = Aws::Http::Scheme::HTTPS;
+
+    std::vector<Aws::String> retryable_errors;
+    retryable_errors.push_back("IDPCommunicationError");
+    retryable_errors.push_back("InvalidIdentityToken");
+    aws_client_configuration.retryStrategy = std::make_shared<Aws::Client::SpecifiedRetryableErrorsRetryStrategy>(
+        retryable_errors, /*maxRetries=*/ 3);
+
+    sts_client = std::make_unique<Aws::STS::STSClient>(metadata_provider, aws_client_configuration);
+
+    LOG_INFO(logger, "Created STS AssumeRole provider using EC2 instance metadata");
+}
+
+Aws::Auth::AWSCredentials AWSInstanceMetadataAssumeRoleCredentialsProvider::GetAWSCredentials()
+{
+    refreshIfExpired();
+    Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock);
+    return credentials;
+}
+
+void AWSInstanceMetadataAssumeRoleCredentialsProvider::Reload()
+{
+    // Get fresh metadata credentials
+    auto metadata_creds = metadata_provider->GetAWSCredentials();
+    if (metadata_creds.IsEmpty())
+    {
+        LOG_ERROR(logger, "Failed to obtain instance metadata credentials");
+        return;
+    }
+
+    // Perform AssumeRole
+    Aws::STS::Model::AssumeRoleRequest request;
+    request.SetRoleArn(role_arn);
+    request.SetRoleSessionName(session_name);
+
+    auto outcome = sts_client->AssumeRole(request);
+    if (!outcome.IsSuccess())
+    {
+        LOG_ERROR(logger, "Failed to assume role: {}", outcome.GetError().GetMessage());
+        return;
+    }
+
+    const auto & result = outcome.GetResult().GetCredentials();
+    credentials = Aws::Auth::AWSCredentials(
+        result.GetAccessKeyId(),
+        result.GetSecretAccessKey(),
+        result.GetSessionToken(),
+        result.GetExpiration().SecondsWithMSPrecision());
+
+    LOG_TRACE(logger, "Successfully assumed role using metadata credentials");
+}
+
+void AWSInstanceMetadataAssumeRoleCredentialsProvider::refreshIfExpired()
+{
+    Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock);
+    if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds))
+        return;
+
+    guard.UpgradeToWriterLock();
+    if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds))
+        return;
+
+    Reload();
+}
+
 
 SSOCredentialsProvider::SSOCredentialsProvider(DB::S3::PocoHTTPClientConfiguration aws_client_configuration_, uint64_t expiration_window_seconds_)
     : profile_to_use(Aws::Auth::GetConfigProfileName())
@@ -704,11 +787,64 @@ S3CredentialsProviderChain::S3CredentialsProviderChain(
     if (credentials_configuration.no_sign_request)
         return;
 
-    /// add explicit credentials to the front of the chain
-    /// because it's manually defined by the user
-    if (!credentials.IsEmpty())
+    if (credentials_configuration.role_arn.empty())
     {
-        AddProvider(std::make_shared<Aws::Auth::SimpleAWSCredentialsProvider>(credentials));
+        if (!credentials.IsEmpty())
+        {
+            AddProvider(std::make_shared<Aws::Auth::SimpleAWSCredentialsProvider>(credentials));
+            return;
+        }
+    }
+    else
+    {
+        if (!credentials.IsEmpty())
+        {
+            auto sts_client_config = Aws::STS::STSClientConfiguration();
+
+            if (!credentials_configuration.sts_endpoint_override.empty())
+            {
+                auto endpoint_uri = Poco::URI(credentials_configuration.sts_endpoint_override);
+
+                String url_without_scheme = endpoint_uri.getHost();
+                if (endpoint_uri.getPort() != 0)
+                    url_without_scheme += ":" + std::to_string(endpoint_uri.getPort());
+
+                sts_client_config.endpointOverride = url_without_scheme;
+                sts_client_config.scheme = endpoint_uri.getScheme() == "https" ? Aws::Http::Scheme::HTTPS : Aws::Http::Scheme::HTTP;
+            }
+
+            AddProvider(std::make_shared<Aws::Auth::STSAssumeRoleCredentialsProvider>(
+                credentials_configuration.role_arn,
+                /* sessionName */ credentials_configuration.role_session_name,
+                /* externalId */ Aws::String(),
+                /* loadFrequency */ Aws::Auth::DEFAULT_CREDS_LOAD_FREQ_SECONDS,
+                std::make_shared<Aws::STS::STSClient>(credentials,
+                                                      /* endpointProvider */ Aws::MakeShared<Aws::STS::STSEndpointProvider>(Aws::STS::STSClient::ALLOCATION_TAG),
+                                                      /* clientConfiguration */ sts_client_config)
+                )
+            );
+        }
+        else
+        {
+            DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(
+                configuration.region,
+                configuration.remote_host_filter,
+                configuration.s3_max_redirects,
+                configuration.s3_retry_attempts,
+                configuration.s3_slow_all_threads_after_network_error,
+                configuration.enable_s3_requests_logging,
+                configuration.for_disk_s3,
+                configuration.get_request_throttler,
+                configuration.put_request_throttler);
+
+            // Use metadata credentials for AssumeRole
+            AddProvider(std::make_shared<AWSInstanceMetadataAssumeRoleCredentialsProvider>(
+                Aws::String(credentials_configuration.role_arn),
+                Aws::String(credentials_configuration.role_session_name),
+                std::move(aws_client_configuration),
+                credentials_configuration.expiration_window_seconds)
+                );
+        }
         return;
     }
 
diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h
index 69349bf5d510..fbe204042683 100644
--- a/src/IO/S3/Credentials.h
+++ b/src/IO/S3/Credentials.h
@@ -11,6 +11,8 @@
 #    include <aws/core/config/AWSProfileConfigLoader.h>
 #    include <aws/core/auth/AWSCredentialsProviderChain.h>
 #    include <aws/core/auth/bearer-token-provider/SSOBearerTokenProvider.h>
+#    include <aws/sts/STSClient.h>
+#    include <aws/sts/model/AssumeRoleRequest.h>
 
 #    include <IO/S3/PocoHTTPClient.h>
 #    include <IO/S3Defines.h>
@@ -108,6 +110,32 @@ class AWSInstanceProfileCredentialsProvider : public Aws::Auth::AWSCredentialsPr
     LoggerPtr logger;
 };
 
+class AWSInstanceMetadataAssumeRoleCredentialsProvider : public Aws::Auth::AWSCredentialsProvider
+{
+public:
+    explicit AWSInstanceMetadataAssumeRoleCredentialsProvider(
+        const Aws::String & role_arn_,
+        const Aws::String & session_name_,
+        DB::S3::PocoHTTPClientConfiguration aws_client_configuration,
+        uint64_t expiration_window_seconds_);
+
+    Aws::Auth::AWSCredentials GetAWSCredentials() override;
+
+protected:
+    void Reload() override;
+
+private:
+    void refreshIfExpired();
+
+    std::shared_ptr<AWSInstanceProfileCredentialsProvider> metadata_provider;
+    std::unique_ptr<Aws::STS::STSClient> sts_client;
+    Aws::Auth::AWSCredentials credentials;
+    Aws::String role_arn;
+    Aws::String session_name;
+    LoggerPtr logger;
+    uint64_t expiration_window_seconds;
+};
+
 class AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider : public Aws::Auth::AWSCredentialsProvider
 {
     /// See STSAssumeRoleWebIdentityCredentialsProvider.
@@ -174,6 +202,9 @@ struct CredentialsConfiguration
     bool use_insecure_imds_request = false;
     uint64_t expiration_window_seconds = DEFAULT_EXPIRATION_WINDOW_SECONDS;
     bool no_sign_request = false;
+    String role_arn = ""; // NOLINT(*-redundant-string-init)
+    String role_session_name = ""; // NOLINT(*-redundant-string-init)
+    String sts_endpoint_override = ""; // NOLINT(*-redundant-string-init)
 };
 
 class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain
diff --git a/src/IO/S3AuthSettings.cpp b/src/IO/S3AuthSettings.cpp
index b87280ef3209..1f28e756eb1f 100644
--- a/src/IO/S3AuthSettings.cpp
+++ b/src/IO/S3AuthSettings.cpp
@@ -36,7 +36,10 @@ namespace ErrorCodes
     DECLARE(String, secret_access_key, "", "", 0) \
     DECLARE(String, session_token, "", "", 0) \
     DECLARE(String, region, "", "", 0) \
-    DECLARE(String, server_side_encryption_customer_key_base64, "", "", 0)
+    DECLARE(String, server_side_encryption_customer_key_base64, "", "", 0) \
+    DECLARE(String, role_arn, "", "", 0) \
+    DECLARE(String, role_session_name, "", "", 0) \
+    DECLARE(String, sts_endpoint_override, "", "", 0)
 
 #define CLIENT_SETTINGS_LIST(M, ALIAS) \
     CLIENT_SETTINGS(M, ALIAS) \
diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp
index 60050eb249f4..80e657398f85 100644
--- a/src/Storages/ObjectStorage/S3/Configuration.cpp
+++ b/src/Storages/ObjectStorage/S3/Configuration.cpp
@@ -14,6 +14,8 @@
 #include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
 #include <Disks/ObjectStorages/S3/diskSettings.h>
 
+#include <Interpreters/evaluateConstantExpression.h>
+
 #include <Parsers/ASTFunction.h>
 #include <Parsers/ASTIdentifier.h>
 #include <Parsers/ASTLiteral.h>
@@ -46,6 +48,9 @@ namespace S3AuthSetting
     extern const S3AuthSettingsString secret_access_key;
     extern const S3AuthSettingsString session_token;
     extern const S3AuthSettingsBool use_environment_credentials;
+    extern const S3AuthSettingsString role_arn;
+    extern const S3AuthSettingsString role_session_name;
+    extern const S3AuthSettingsString sts_endpoint_override;
 }
 
 namespace ErrorCodes
@@ -182,8 +187,65 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect
     keys = {url.key};
 }
 
+void StorageS3Configuration::extractExtraCreds(ASTs & args, ContextPtr context)
+{
+    ASTs::iterator extra_creds_it = args.end();
+
+    for (auto * arg_it = args.begin(); arg_it != args.end(); ++arg_it)
+    {
+        const auto * extra_creds_ast_function = (*arg_it)->as<ASTFunction>();
+        if (extra_creds_ast_function && extra_creds_ast_function->name == "extra_credentials")
+        {
+            if (extra_creds_it != args.end())
+                throw Exception(
+                    ErrorCodes::BAD_ARGUMENTS,
+                    "S3 table function can have only one extra_credentials argument");
+
+            const auto * extra_creds_function_args_expr = assert_cast<const ASTExpressionList *>(extra_creds_ast_function->arguments.get());
+            auto extra_creds_function_args = extra_creds_function_args_expr->children;
+
+            for (auto & extra_cred_arg : extra_creds_function_args)
+            {
+                const auto * extra_cred_ast = extra_cred_arg->as<ASTFunction>();
+                if (!extra_cred_ast || extra_cred_ast->name != "equals")
+                    throw Exception(ErrorCodes::BAD_ARGUMENTS, "extra_credentials argument is incorrect: shall be key=value");
+
+                const auto * extra_cred_args_expr = assert_cast<const ASTExpressionList *>(extra_cred_ast->arguments.get());
+                auto extra_cred_args = extra_cred_args_expr->children;
+                if (extra_cred_args.size() != 2)
+                    throw Exception(
+                        ErrorCodes::BAD_ARGUMENTS,
+                        "extra_credentials argument is incorrect: expected 2 arguments, got {}",
+                        extra_cred_args.size());
+
+                auto ast_literal = evaluateConstantExpressionOrIdentifierAsLiteral(extra_cred_args[0], context);
+                auto arg_name_value = ast_literal->as<ASTLiteral>()->value;
+                if (arg_name_value.getType() != Field::Types::Which::String)
+                    throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected string as extra_credentials name");
+                auto arg_name = arg_name_value.safeGet<String>();
+
+                ast_literal = evaluateConstantExpressionOrIdentifierAsLiteral(extra_cred_args[1], context);
+                auto arg_value = ast_literal->as<ASTLiteral>()->value;
+                if (arg_value.getType() != Field::Types::Which::String)
+                    throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected string as extra_credentials value");
+
+                extra_credentials_from_ast.emplace_back(arg_name, arg_value.safeGet<String>());
+            }
+
+            extra_creds_it = arg_it;
+        }
+    }
+
+    /// To avoid making unnecessary changes and avoid potential conflicts in future,
+    /// simply remove the "extra" argument after processing if it exists.
+    if (extra_creds_it != args.end())
+        args.erase(extra_creds_it);
+}
+
 void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure)
 {
+    extractExtraCreds(args, context);
+
     size_t count = StorageURL::evalArgsAndCollectHeaders(args, headers_from_ast, context);
 
     if (count == 0 || count > getMaxNumberOfArguments(with_structure))
@@ -381,6 +443,23 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
     if (no_sign_request)
         auth_settings[S3AuthSetting::no_sign_request] = no_sign_request;
 
+    if (!extra_credentials_from_ast.empty())
+    {
+        auto extract_extra_cred_value = [&extra_creds = this->extra_credentials_from_ast](const String & cred_name) -> String
+        {
+            auto role_arn_it = std::find_if(extra_creds.begin(), extra_creds.end(),
+                                            [&cred_name](const HTTPHeaderEntry & entry) { return entry.name == cred_name; });
+            if (role_arn_it != extra_creds.end())
+                return role_arn_it->value;
+
+            return {};
+        };
+
+        auth_settings[S3AuthSetting::role_arn] = extract_extra_cred_value("role_arn");
+        auth_settings[S3AuthSetting::role_session_name] = extract_extra_cred_value("role_session_name");
+        auth_settings[S3AuthSetting::sts_endpoint_override] = extract_extra_cred_value("sts_endpoint_override");
+    }
+
     static_configuration = !auth_settings[S3AuthSetting::access_key_id].value.empty() || auth_settings[S3AuthSetting::no_sign_request].changed;
     auth_settings[S3AuthSetting::no_sign_request] = no_sign_request;
 
diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h
index 52fcc45390c0..c0ca9fc85a87 100644
--- a/src/Storages/ObjectStorage/S3/Configuration.h
+++ b/src/Storages/ObjectStorage/S3/Configuration.h
@@ -37,7 +37,7 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration
         " - url, access_key_id, secret_access_key, session_token, format, structure\n"
         " - url, access_key_id, secret_access_key, format, structure, compression_method\n"
         " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n"
-        "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)";
+        "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`) and extra credentials for role assumption (`extra_credentials(role_arn=value, role_session_name=value)`)";
 
     /// All possible signatures for S3 storage without structure argument (for example for S3 table engine).
     static constexpr auto max_number_of_arguments_without_structure = 6;
@@ -54,7 +54,7 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration
         " - url, access_key_id, secret_access_key, session_token, format\n"
         " - url, access_key_id, secret_access_key, format, compression_method\n"
         " - url, access_key_id, secret_access_key, session_token, format, compression_method\n"
-        "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)";
+        "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`) and extra credentials for role assumption (`extra_credentials(role_arn=value, role_session_name=value)`)";
 
     StorageS3Configuration() = default;
 
@@ -96,6 +96,7 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration
         bool with_structure) override;
 
 private:
+    void extractExtraCreds(ASTs & args, ContextPtr context);
     void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override;
     void fromAST(ASTs & args, ContextPtr context, bool with_structure) override;
 
@@ -105,6 +106,8 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration
     S3::S3AuthSettings auth_settings;
     S3::S3RequestSettings request_settings;
     HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration.
+    HTTPHeaderEntries extra_credentials_from_ast; /// Avoid duplicated entities: HTTPHeaderEntry structure matches our needs here, use it.
+
     /// If s3 configuration was passed from ast, then it is static.
     /// If from config - it can be changed with config reload.
     bool static_configuration = true;