diff --git a/docs/src/main/sphinx/admin/fault-tolerant-execution.rst b/docs/src/main/sphinx/admin/fault-tolerant-execution.rst
index ef655a58a889..8ebbbe957c67 100644
--- a/docs/src/main/sphinx/admin/fault-tolerant-execution.rst
+++ b/docs/src/main/sphinx/admin/fault-tolerant-execution.rst
@@ -373,6 +373,14 @@ the property may be configured for:
be ignored for other S3 storage systems.
-
- AWS S3, GCS
+ * - ``exchange.s3.iam-role``
+ - IAM role to assume.
+ -
+ - AWS S3, GCS
+ * - ``exchange.s3.external-id``
+ - External ID for the IAM role trust policy.
+ -
+ - AWS S3, GCS
* - ``exchange.s3.region``
- Region of the S3 bucket.
-
diff --git a/plugin/trino-exchange-filesystem/pom.xml b/plugin/trino-exchange-filesystem/pom.xml
index 21ffc1f88313..43dcac3a53c8 100644
--- a/plugin/trino-exchange-filesystem/pom.xml
+++ b/plugin/trino-exchange-filesystem/pom.xml
@@ -297,14 +297,12 @@
software.amazon.awssdk
- utils
+ sts
-
software.amazon.awssdk
- sts
- runtime
+ utils
diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java
index b680c425732f..44c698eaaa9d 100644
--- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java
+++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java
@@ -37,9 +37,10 @@ public class ExchangeS3Config
{
private String s3AwsAccessKey;
private String s3AwsSecretKey;
+ private Optional s3IamRole = Optional.empty();
+ private Optional s3ExternalId = Optional.empty();
private Optional s3Region = Optional.empty();
private Optional s3Endpoint = Optional.empty();
- private boolean s3UseWebIdentityTokenCredentials;
private int s3MaxErrorRetries = 10;
// Default to S3 multi-part upload minimum size to avoid excessive memory consumption from buffering
private DataSize s3UploadPartSize = DataSize.of(5, MEGABYTE);
@@ -75,6 +76,32 @@ public ExchangeS3Config setS3AwsSecretKey(String s3AwsSecretKey)
return this;
}
+ public Optional getS3IamRole()
+ {
+ return s3IamRole;
+ }
+
+ @Config("exchange.s3.iam-role")
+ @ConfigDescription("ARN of an IAM role to assume when connecting to S3")
+ public ExchangeS3Config setS3IamRole(String s3IamRole)
+ {
+ this.s3IamRole = Optional.ofNullable(s3IamRole);
+ return this;
+ }
+
+ public Optional getS3ExternalId()
+ {
+ return s3ExternalId;
+ }
+
+ @Config("exchange.s3.external-id")
+ @ConfigDescription("External ID for the IAM role trust policy when connecting to S3")
+ public ExchangeS3Config setS3ExternalId(String s3ExternalId)
+ {
+ this.s3ExternalId = Optional.ofNullable(s3ExternalId);
+ return this;
+ }
+
public Optional getS3Region()
{
return s3Region;
@@ -102,18 +129,6 @@ public ExchangeS3Config setS3Endpoint(String s3Endpoint)
return this;
}
- public boolean isS3UseWebIdentityTokenCredentials()
- {
- return s3UseWebIdentityTokenCredentials;
- }
-
- @Config("exchange.s3.use-web-identity-token-credentials")
- public ExchangeS3Config setS3UseWebIdentityTokenCredentials(boolean s3UseWebIdentityTokenCredentials)
- {
- this.s3UseWebIdentityTokenCredentials = s3UseWebIdentityTokenCredentials;
- return this;
- }
-
@Min(0)
public int getS3MaxErrorRetries()
{
diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java
index 3f71b86df307..fa8c272972da 100644
--- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java
+++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java
@@ -41,7 +41,6 @@
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.WebIdentityTokenFileCredentialsProvider;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
import software.amazon.awssdk.core.retry.RetryPolicy;
@@ -76,6 +75,10 @@
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable;
import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Publisher;
+import software.amazon.awssdk.services.sts.StsClient;
+import software.amazon.awssdk.services.sts.StsClientBuilder;
+import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider;
+import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
import javax.annotation.PreDestroy;
import javax.annotation.concurrent.GuardedBy;
@@ -452,12 +455,46 @@ private static boolean isDirectory(URI uri)
private static AwsCredentialsProvider createAwsCredentialsProvider(ExchangeS3Config config)
{
- if (config.getS3AwsAccessKey() != null && config.getS3AwsSecretKey() != null) {
- return StaticCredentialsProvider.create(AwsBasicCredentials.create(config.getS3AwsAccessKey(), config.getS3AwsSecretKey()));
+ String accessKey = config.getS3AwsAccessKey();
+ String secretKey = config.getS3AwsSecretKey();
+
+ if (accessKey == null && secretKey != null) {
+ throw new IllegalArgumentException("AWS access key set but secret is not set; make sure you set exchange.s3.aws-secret-key config property");
+ }
+
+ if (accessKey != null && secretKey == null) {
+ throw new IllegalArgumentException("AWS secret key set but access is not set; make sure you set exchange.s3.aws-access-key config property");
+ }
+
+ if (accessKey != null) {
+ checkArgument(
+ config.getS3IamRole().isEmpty(),
+ "IAM role is not compatible with access key based authentication; make sure you set only one of exchange.s3.aws-access-key, exchange.s3.iam-role config properties");
+ checkArgument(
+ config.getS3ExternalId().isEmpty(),
+ "External ID is not compatible with access key based authentication; make sure you set only one of exchange.s3.aws-access-key, exchange.s3.external-id config properties");
+
+ return StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, secretKey));
}
- if (config.isS3UseWebIdentityTokenCredentials()) {
- return WebIdentityTokenFileCredentialsProvider.create();
+ if (config.getS3ExternalId().isPresent() && config.getS3IamRole().isEmpty()) {
+ throw new IllegalArgumentException("External ID can only be used with IAM role based authentication; make sure you set exchange.s3.iam-role config property");
+ }
+
+ if (config.getS3IamRole().isPresent()) {
+ AssumeRoleRequest.Builder assumeRoleRequest = AssumeRoleRequest.builder()
+ .roleArn(config.getS3IamRole().get())
+ .roleSessionName("trino-exchange");
+ config.getS3ExternalId().ifPresent(assumeRoleRequest::externalId);
+
+ StsClientBuilder stsClientBuilder = StsClient.builder();
+ config.getS3Region().ifPresent(stsClientBuilder::region);
+
+ return StsAssumeRoleCredentialsProvider.builder()
+ .stsClient(stsClientBuilder.build())
+ .refreshRequest(assumeRoleRequest.build())
+ .asyncCredentialUpdateEnabled(true)
+ .build();
}
return DefaultCredentialsProvider.create();
diff --git a/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java b/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java
index 3c06c2ed09e3..5dc520c876dd 100644
--- a/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java
+++ b/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java
@@ -36,9 +36,10 @@ public void testDefaults()
assertRecordedDefaults(recordDefaults(ExchangeS3Config.class)
.setS3AwsAccessKey(null)
.setS3AwsSecretKey(null)
+ .setS3IamRole(null)
+ .setS3ExternalId(null)
.setS3Region(null)
.setS3Endpoint(null)
- .setS3UseWebIdentityTokenCredentials(false)
.setS3MaxErrorRetries(10)
.setS3UploadPartSize(DataSize.of(5, MEGABYTE))
.setStorageClass(StorageClass.STANDARD)
@@ -55,9 +56,10 @@ public void testExplicitPropertyMappings()
Map properties = ImmutableMap.builder()
.put("exchange.s3.aws-access-key", "access")
.put("exchange.s3.aws-secret-key", "secret")
+ .put("exchange.s3.iam-role", "roleArn")
+ .put("exchange.s3.external-id", "externalId")
.put("exchange.s3.region", "us-west-1")
.put("exchange.s3.endpoint", "https://s3.us-east-1.amazonaws.com")
- .put("exchange.s3.use-web-identity-token-credentials", "true")
.put("exchange.s3.max-error-retries", "8")
.put("exchange.s3.upload.part-size", "10MB")
.put("exchange.s3.storage-class", "REDUCED_REDUNDANCY")
@@ -71,9 +73,10 @@ public void testExplicitPropertyMappings()
ExchangeS3Config expected = new ExchangeS3Config()
.setS3AwsAccessKey("access")
.setS3AwsSecretKey("secret")
+ .setS3IamRole("roleArn")
+ .setS3ExternalId("externalId")
.setS3Region("us-west-1")
.setS3Endpoint("https://s3.us-east-1.amazonaws.com")
- .setS3UseWebIdentityTokenCredentials(true)
.setS3MaxErrorRetries(8)
.setS3UploadPartSize(DataSize.of(10, MEGABYTE))
.setStorageClass(StorageClass.REDUCED_REDUNDANCY)