From f70d9c11c011d53b1743763df7b648b6d87ac6ba Mon Sep 17 00:00:00 2001 From: Lapshin Dmitry Date: Wed, 1 Oct 2025 20:41:38 +0200 Subject: [PATCH] HADOOP-19282. STSClientFactory: do not use URIBuilder (#7966) URIBuilder was used from the AWS SDK for Java v2, from the shaded Apache HTTP Client. It is a problem if a user would like not to use the AWS SDK bundle, since more or less only 3 modules are needed (s3, s3-transfer & sts), but that may cause problems on unshaded dependency versions. Since a URI constructor can achieve the same here I switched it as a preferred option. Contributed by Lapshin Dmitry --- .../java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java index dcfc0a7ee8416..ab15174496969 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java @@ -35,7 +35,6 @@ import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; import software.amazon.awssdk.services.sts.model.Credentials; import software.amazon.awssdk.services.sts.model.GetSessionTokenRequest; -import software.amazon.awssdk.thirdparty.org.apache.http.client.utils.URIBuilder; import org.apache.hadoop.fs.s3a.impl.AWSClientConfig; import org.apache.hadoop.util.Preconditions; @@ -167,7 +166,7 @@ public static StsClientBuilder builder(final AwsCredentialsProvider credentials, */ private static URI getSTSEndpoint(String endpoint) { try { - return new URIBuilder().setScheme("https").setHost(endpoint).build(); + return new URI("https", endpoint, null, null); } catch (URISyntaxException e) { throw new IllegalArgumentException(e); }