Skip to content

Commit 15d78f4

Browse files
sidsethdeepakdamri
authored andcommitted
HADOOP-16445. Allow separate custom signing algorithms for S3 and DDB (apache#1332)
1 parent b94e1de commit 15d78f4

File tree

9 files changed

+313
-7
lines changed

9 files changed

+313
-7
lines changed

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,8 @@ public AmazonS3 createS3Client(URI name,
5555
final AWSCredentialsProvider credentials,
5656
final String userAgentSuffix) throws IOException {
5757
Configuration conf = getConf();
58-
final ClientConfiguration awsConf = S3AUtils.createAwsConf(getConf(), bucket);
58+
final ClientConfiguration awsConf = S3AUtils
59+
.createAwsConf(getConf(), bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
5960
if (!StringUtils.isEmpty(userAgentSuffix)) {
6061
awsConf.setUserAgentSuffix(userAgentSuffix);
6162
}
Lines changed: 99 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,99 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.fs.s3a;
19+
20+
import com.amazonaws.auth.Signer;
21+
import com.amazonaws.auth.SignerFactory;
22+
import java.io.Closeable;
23+
import java.io.IOException;
24+
import org.slf4j.Logger;
25+
import org.slf4j.LoggerFactory;
26+
27+
import org.apache.hadoop.conf.Configuration;
28+
29+
import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS;
30+
31+
/**
32+
* Class to handle custom signers.
33+
*/
34+
public class SignerManager implements Closeable {
35+
36+
private static final Logger LOG = LoggerFactory
37+
.getLogger(SignerManager.class);
38+
39+
40+
public SignerManager() {
41+
}
42+
43+
/**
44+
* Initialize custom signers and register them with the AWS SDK.
45+
*
46+
* @param conf Hadoop configuration
47+
*/
48+
public void initCustomSigners(Configuration conf) {
49+
String[] customSigners = conf.getTrimmedStrings(CUSTOM_SIGNERS);
50+
if (customSigners == null || customSigners.length == 0) {
51+
// No custom signers specified, nothing to do.
52+
LOG.debug("No custom signers specified");
53+
return;
54+
}
55+
56+
for (String customSigner : customSigners) {
57+
String[] parts = customSigner.split(":");
58+
if (parts.length != 2) {
59+
String message =
60+
"Invalid format (Expected name:SignerClass) for CustomSigner: ["
61+
+ customSigner
62+
+ "]";
63+
LOG.error(message);
64+
throw new IllegalArgumentException(message);
65+
}
66+
maybeRegisterSigner(parts[0], parts[1], conf);
67+
}
68+
}
69+
70+
/*
71+
* Make sure the signer class is registered once with the AWS SDK
72+
*/
73+
private static void maybeRegisterSigner(String signerName,
74+
String signerClassName, Configuration conf) {
75+
try {
76+
SignerFactory.getSignerByTypeAndService(signerName, null);
77+
} catch (IllegalArgumentException e) {
78+
// Signer is not registered with the AWS SDK.
79+
// Load the class and register the signer.
80+
Class<? extends Signer> clazz = null;
81+
try {
82+
clazz = (Class<? extends Signer>) conf.getClassByName(signerClassName);
83+
} catch (ClassNotFoundException cnfe) {
84+
throw new RuntimeException(String
85+
.format("Signer class [%s] not found for signer [%s]",
86+
signerClassName, signerName), cnfe);
87+
}
88+
LOG.debug("Registering Custom Signer - [{}->{}]", signerName,
89+
clazz.getName());
90+
synchronized (SignerManager.class) {
91+
SignerFactory.registerSigner(signerName, clazz);
92+
}
93+
}
94+
}
95+
96+
@Override
97+
public void close() throws IOException {
98+
}
99+
}

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,12 +31,14 @@
3131
import com.amazonaws.services.securitytoken.model.Credentials;
3232
import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
3333
import com.google.common.base.Preconditions;
34+
3435
import org.slf4j.Logger;
3536
import org.slf4j.LoggerFactory;
3637

3738
import org.apache.hadoop.classification.InterfaceAudience;
3839
import org.apache.hadoop.classification.InterfaceStability;
3940
import org.apache.hadoop.conf.Configuration;
41+
import org.apache.hadoop.fs.s3a.Constants;
4042
import org.apache.hadoop.fs.s3a.Invoker;
4143
import org.apache.hadoop.fs.s3a.Retries;
4244
import org.apache.hadoop.fs.s3a.S3AUtils;
@@ -73,7 +75,8 @@ public static AWSSecurityTokenServiceClientBuilder builder(
7375
final Configuration conf,
7476
final String bucket,
7577
final AWSCredentialsProvider credentials) throws IOException {
76-
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
78+
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket,
79+
Constants.AWS_SERVICE_IDENTIFIER_STS);
7780
String endpoint = conf.getTrimmed(DELEGATION_TOKEN_ENDPOINT,
7881
DEFAULT_DELEGATION_TOKEN_ENDPOINT);
7982
String region = conf.getTrimmed(DELEGATION_TOKEN_REGION,
@@ -99,7 +102,8 @@ public static AWSSecurityTokenServiceClientBuilder builder(
99102
final AWSCredentialsProvider credentials,
100103
final String stsEndpoint,
101104
final String stsRegion) throws IOException {
102-
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
105+
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket,
106+
Constants.AWS_SERVICE_IDENTIFIER_STS);
103107
return builder(credentials, awsConf, stsEndpoint, stsRegion);
104108
}
105109

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636

3737
import org.apache.hadoop.conf.Configuration;
3838
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
39+
import org.apache.hadoop.fs.s3a.Constants;
3940
import org.apache.hadoop.fs.s3a.Invoker;
4041
import org.apache.hadoop.fs.s3a.Retries;
4142
import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
@@ -301,7 +302,8 @@ private synchronized Optional<STSClientFactory.STSClient> maybeInitSTS()
301302

302303
invoker = new Invoker(new S3ARetryPolicy(conf), LOG_EVENT);
303304
ClientConfiguration awsConf =
304-
S3AUtils.createAwsConf(conf, uri.getHost());
305+
S3AUtils.createAwsConf(conf, uri.getHost(),
306+
Constants.AWS_SERVICE_IDENTIFIER_STS);
305307
AWSSecurityTokenService tokenService =
306308
STSClientFactory.builder(parentAuthChain,
307309
awsConf,

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.hadoop.conf.Configurable;
3535
import org.apache.hadoop.conf.Configuration;
3636
import org.apache.hadoop.conf.Configured;
37+
import org.apache.hadoop.fs.s3a.Constants;
3738
import org.apache.hadoop.fs.s3a.S3AUtils;
3839

3940
import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
@@ -80,7 +81,8 @@ public AmazonDynamoDB createDynamoDBClient(String defaultRegion,
8081
"Should have been configured before usage");
8182

8283
final Configuration conf = getConf();
83-
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
84+
final ClientConfiguration awsConf = S3AUtils
85+
.createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_DDB);
8486

8587
final String region = getRegion(conf, defaultRegion);
8688
LOG.debug("Creating DynamoDB client in region {}", region);

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import com.amazonaws.services.s3.AmazonS3;
2323
import com.amazonaws.services.s3.S3ClientOptions;
2424

25+
2526
import org.apache.commons.lang3.StringUtils;
2627
import org.apache.commons.lang3.reflect.FieldUtils;
2728
import org.apache.hadoop.conf.Configuration;
@@ -30,12 +31,14 @@
3031
import org.apache.hadoop.fs.contract.ContractTestUtils;
3132
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
3233
import org.apache.hadoop.test.GenericTestUtils;
34+
import org.junit.Assert;
3335
import org.junit.Rule;
3436
import org.junit.Test;
3537
import org.junit.rules.Timeout;
3638
import org.slf4j.Logger;
3739
import org.slf4j.LoggerFactory;
3840

41+
import java.io.IOException;
3942
import java.io.File;
4043
import java.net.URI;
4144
import java.security.PrivilegedExceptionAction;
@@ -612,4 +615,69 @@ public void testSecurityCredentialPropagationEndToEnd() throws Exception {
612615
"override,base");
613616
}
614617

618+
@Test(timeout = 10_000L)
619+
public void testS3SpecificSignerOverride() throws IOException {
620+
ClientConfiguration clientConfiguration = null;
621+
Configuration config;
622+
623+
String signerOverride = "testSigner";
624+
String s3SignerOverride = "testS3Signer";
625+
626+
// Default SIGNING_ALGORITHM, overridden for S3 only
627+
config = new Configuration();
628+
config.set(SIGNING_ALGORITHM_S3, s3SignerOverride);
629+
clientConfiguration = S3AUtils
630+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3);
631+
Assert.assertEquals(s3SignerOverride,
632+
clientConfiguration.getSignerOverride());
633+
clientConfiguration = S3AUtils
634+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_DDB);
635+
Assert.assertNull(clientConfiguration.getSignerOverride());
636+
637+
// Configured base SIGNING_ALGORITHM, overridden for S3 only
638+
config = new Configuration();
639+
config.set(SIGNING_ALGORITHM, signerOverride);
640+
config.set(SIGNING_ALGORITHM_S3, s3SignerOverride);
641+
clientConfiguration = S3AUtils
642+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3);
643+
Assert.assertEquals(s3SignerOverride,
644+
clientConfiguration.getSignerOverride());
645+
clientConfiguration = S3AUtils
646+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_DDB);
647+
Assert
648+
.assertEquals(signerOverride, clientConfiguration.getSignerOverride());
649+
}
650+
651+
@Test(timeout = 10_000L)
652+
public void testDdbSpecificSignerOverride() throws IOException {
653+
ClientConfiguration clientConfiguration = null;
654+
Configuration config;
655+
656+
String signerOverride = "testSigner";
657+
String ddbSignerOverride = "testDdbSigner";
658+
659+
// Default SIGNING_ALGORITHM, overridden for S3
660+
config = new Configuration();
661+
config.set(SIGNING_ALGORITHM_DDB, ddbSignerOverride);
662+
clientConfiguration = S3AUtils
663+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_DDB);
664+
Assert.assertEquals(ddbSignerOverride,
665+
clientConfiguration.getSignerOverride());
666+
clientConfiguration = S3AUtils
667+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3);
668+
Assert.assertNull(clientConfiguration.getSignerOverride());
669+
670+
// Configured base SIGNING_ALGORITHM, overridden for S3
671+
config = new Configuration();
672+
config.set(SIGNING_ALGORITHM, signerOverride);
673+
config.set(SIGNING_ALGORITHM_DDB, ddbSignerOverride);
674+
clientConfiguration = S3AUtils
675+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_DDB);
676+
Assert.assertEquals(ddbSignerOverride,
677+
clientConfiguration.getSignerOverride());
678+
clientConfiguration = S3AUtils
679+
.createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3);
680+
Assert
681+
.assertEquals(signerOverride, clientConfiguration.getSignerOverride());
682+
}
615683
}

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -368,7 +368,7 @@ public <E extends Exception> E expectedSessionRequestFailure(
368368
DurationInfo ignored = new DurationInfo(LOG, "requesting credentials")) {
369369
Configuration conf = new Configuration(getContract().getConf());
370370
ClientConfiguration awsConf =
371-
S3AUtils.createAwsConf(conf, null);
371+
S3AUtils.createAwsConf(conf, null, AWS_SERVICE_IDENTIFIER_STS);
372372
return intercept(clazz, exceptionText,
373373
() -> {
374374
AWSSecurityTokenService tokenService =

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -619,7 +619,7 @@ public static MarshalledCredentials requestSessionCredentials(
619619
MarshalledCredentials sc = MarshalledCredentialBinding
620620
.requestSessionCredentials(
621621
buildAwsCredentialsProvider(conf),
622-
S3AUtils.createAwsConf(conf, bucket),
622+
S3AUtils.createAwsConf(conf, bucket, AWS_SERVICE_IDENTIFIER_STS),
623623
conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT,
624624
DEFAULT_ASSUMED_ROLE_STS_ENDPOINT),
625625
conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT_REGION,

0 commit comments

Comments
 (0)