Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -343,9 +343,37 @@ private Constants() {
public static final String SERVER_SIDE_ENCRYPTION_KEY =
"fs.s3a.server-side-encryption.key";

//override signature algorithm used for signing requests
/**
* List of custom Signers. The signer class will be loaded, and the signer
* name will be associated with this signer class in the S3 SDK. e.g. Single
* CustomSigner -> 'CustomSigner:org.apache...CustomSignerClass Multiple
* CustomSigners -> 'CSigner1:CustomSignerClass1,CSigner2:CustomerSignerClass2
*/
public static final String CUSTOM_SIGNERS = "fs.s3a.custom.signers";

/**
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you are going to have to add these to the aws docs I'm afraid. This is probably time to start a new "signing" section

* There's 3 parameters that can be used to specify a non-default signing
* algorithm. fs.s3a.signing-algorithm - This property has existed for the
* longest time. If specified, without either of the other 2 properties being
* specified, this signing algorithm will be used for S3 and DDB (S3Guard).
* The other 2 properties override this value for S3 or DDB.
Copy link
Contributor

@steveloughran steveloughran Sep 20, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also: other uses like STS. Maybe say "non S3 requests, such as to DDB (for S3Guard) or to STS."

* fs.s3a.s3.signing-algorithm - Allows overriding the S3 Signing algorithm.
* This does not affect DDB. Specifying this property without specifying
* fs.s3a.signing-algorithm will only update the signing algorithm for S3
* requests, and the default will be used for DDB fs.s3a.ddb.signing-algorithm
* - Allows overriding the DDB Signing algorithm. This does not affect S3.
* Specifying this property without specifying fs.s3a.signing-algorithm will
* only update the signing algorithm for DDB requests, and the default will be
* used for S3
*/
public static final String SIGNING_ALGORITHM = "fs.s3a.signing-algorithm";

public static final String SIGNING_ALGORITHM_S3 =
"fs.s3a.s3.signing-algorithm";

public static final String SIGNING_ALGORITHM_DDB =
"fs.s3a.ddb.signing-algorithm";

public static final String S3N_FOLDER_SUFFIX = "_$folder$";
public static final String FS_S3A_BLOCK_SIZE = "fs.s3a.block.size";
public static final String FS_S3A = "s3a";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,8 @@ public AmazonS3 createS3Client(URI name,
final AWSCredentialsProvider credentials,
final String userAgentSuffix) throws IOException {
Configuration conf = getConf();
final ClientConfiguration awsConf = S3AUtils.createAwsConf(getConf(), bucket);
final ClientConfiguration awsConf = S3AUtils
.createAwsConfForS3(getConf(), bucket);
if (!StringUtils.isEmpty(userAgentSuffix)) {
awsConf.setUserAgentSuffix(userAgentSuffix);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,8 @@ public void initialize(URI name, Configuration originalConf)
LOG.debug("Initializing S3AFileSystem for {}", bucket);
// clone the configuration into one with propagated bucket options
Configuration conf = propagateBucketOptions(originalConf, bucket);
// Initialize any custom signers
initCustomSigners(conf);
// patch the Hadoop security providers
patchSecurityCredentialProviders(conf);
// look for delegation token support early.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import com.amazonaws.SdkBaseException;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.Signer;
import com.amazonaws.auth.SignerFactory;
import com.amazonaws.retry.RetryUtils;
import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
import com.amazonaws.services.dynamodbv2.model.LimitExceededException;
Expand Down Expand Up @@ -1195,9 +1197,10 @@ public static void deleteWithWarning(FileSystem fs,
}

/**
* Create a new AWS {@code ClientConfiguration}.
* All clients to AWS services <i>MUST</i> use this for consistent setup
* of connectivity, UA, proxy settings.
* Create a new AWS {@code ClientConfiguration}. All clients to AWS services
* <i>MUST</i> use this or the equivalents for the specific service for
* consistent setup of connectivity, UA, proxy settings.
*
* @param conf The Hadoop configuration
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @return new AWS client configuration
Expand All @@ -1213,6 +1216,38 @@ public static ClientConfiguration createAwsConf(Configuration conf,
return awsConf;
}

/**
* Create a new AWS {#link ClientConfiguration} S3 clients <i>MUST</i> use
* this for consistent setup of connectivity, UA, proxy settings.
*/
public static ClientConfiguration createAwsConfForS3(Configuration conf,
String bucket)
throws IOException {
ClientConfiguration awsConf = createAwsConf(conf, bucket);
String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM_S3, "");
if (!signerOverride.isEmpty()) {
LOG.debug("Signer override for S3 = {}", signerOverride);
awsConf.setSignerOverride(signerOverride);
}
return awsConf;
}

/**
* Create a new AWS {#link ClientConfiguration} DynamoDB clients <i>MUST</i>
* use this for consistent setup of connectivity, UA, proxy settings.
*/
public static ClientConfiguration createAwsConfForDdb(Configuration conf,
String bucket)
throws IOException {
ClientConfiguration awsConf = createAwsConf(conf, bucket);
String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM_DDB, "");
if (!signerOverride.isEmpty()) {
LOG.debug("Signer override for DDB = {}", signerOverride);
awsConf.setSignerOverride(signerOverride);
}
return awsConf;
}

/**
* Initializes all AWS SDK settings related to connection management.
*
Expand Down Expand Up @@ -1303,6 +1338,40 @@ public static void initProxySupport(Configuration conf,
}
}

/**
* Initialize custom signers and register them with the AWS SDK.
*
* @param conf Hadoop configuration
*/
public static void initCustomSigners(Configuration conf) {
String[] customSigners = conf.getTrimmedStrings(CUSTOM_SIGNERS);
if (customSigners == null || customSigners.length == 0) {
// No custom signers specified, nothing to do.
LOG.debug("No custom signers specified");
return;
}

for (String customSigner : customSigners) {
String[] parts = customSigner.split(":");
if (parts.length != 2) {
String message =
"Invalid format (name:class) for CustomSigner: [" + customSigner
+ "]";
LOG.error(message);
throw new IllegalArgumentException(message);
}
Class<? extends Signer> clazz = null;
try {
clazz = (Class<? extends Signer>) conf.getClassByName(parts[1]);
} catch (ClassNotFoundException e) {
throw new RuntimeException(e);
}
LOG.debug("Registering Custom Signer - [{}->{}]", parts[0],
clazz.getName());
SignerFactory.registerSigner(parts[0], clazz);
}
}

/**
* Initializes the User-Agent header to send in HTTP requests to AWS
* services. We always include the Hadoop version number. The user also
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,8 @@ public AmazonDynamoDB createDynamoDBClient(String defaultRegion,
"Should have been configured before usage");

final Configuration conf = getConf();
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
final ClientConfiguration awsConf = S3AUtils
.createAwsConfForDdb(conf, bucket);

final String region = getRegion(conf, defaultRegion);
LOG.debug("Creating DynamoDB client in region {}", region);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,17 +19,24 @@
package org.apache.hadoop.fs.s3a;

import com.amazonaws.ClientConfiguration;
import com.amazonaws.SignableRequest;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.Signer;
import com.amazonaws.auth.SignerFactory;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.S3ClientOptions;

import java.io.IOException;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
Expand Down Expand Up @@ -617,4 +624,135 @@ public void testSecurityCredentialPropagationEndToEnd() throws Exception {
"override,base");
}

@Test(timeout = 10_000L)
public void testS3SpecificSignerOverride() throws IOException {
ClientConfiguration clientConfiguration = null;
Configuration config;

String signerOverride = "testSigner";
String s3SignerOverride = "testS3Signer";

// Default SIGNING_ALGORITHM, overridden for S3 only
config = new Configuration();
config.set(SIGNING_ALGORITHM_S3, s3SignerOverride);
clientConfiguration = S3AUtils.createAwsConfForS3(config, "dontcare");
Assert.assertEquals(s3SignerOverride,
clientConfiguration.getSignerOverride());
clientConfiguration = S3AUtils.createAwsConfForDdb(config, "dontcare");
Assert.assertNull(clientConfiguration.getSignerOverride());

// Configured base SIGNING_ALGORITHM, overridden for S3 only
config = new Configuration();
config.set(SIGNING_ALGORITHM, signerOverride);
config.set(SIGNING_ALGORITHM_S3, s3SignerOverride);
clientConfiguration = S3AUtils.createAwsConfForS3(config, "dontcare");
Assert.assertEquals(s3SignerOverride,
clientConfiguration.getSignerOverride());
clientConfiguration = S3AUtils.createAwsConfForDdb(config, "dontcare");
Assert
.assertEquals(signerOverride, clientConfiguration.getSignerOverride());
}

@Test(timeout = 10_000L)
public void testDdbSpecificSignerOverride() throws IOException {
ClientConfiguration clientConfiguration = null;
Configuration config;

String signerOverride = "testSigner";
String ddbSignerOverride = "testDdbSigner";

// Default SIGNING_ALGORITHM, overridden for S3
config = new Configuration();
config.set(SIGNING_ALGORITHM_DDB, ddbSignerOverride);
clientConfiguration = S3AUtils.createAwsConfForDdb(config, "dontcare");
Assert.assertEquals(ddbSignerOverride,
clientConfiguration.getSignerOverride());
clientConfiguration = S3AUtils.createAwsConfForS3(config, "dontcare");
Assert.assertNull(clientConfiguration.getSignerOverride());

// Configured base SIGNING_ALGORITHM, overridden for S3
config = new Configuration();
config.set(SIGNING_ALGORITHM, signerOverride);
config.set(SIGNING_ALGORITHM_DDB, ddbSignerOverride);
clientConfiguration = S3AUtils.createAwsConfForDdb(config, "dontcare");
Assert.assertEquals(ddbSignerOverride,
clientConfiguration.getSignerOverride());
clientConfiguration = S3AUtils.createAwsConfForS3(config, "dontcare");
Assert
.assertEquals(signerOverride, clientConfiguration.getSignerOverride());
}

// Expecting generic Exception.class to handle future implementation changes.
// For now, this is an NPE
@Test(timeout = 10_000L, expected = Exception.class)
public void testCustomSignerFailureIfNotRegistered() {
Signer s1 = SignerFactory.createSigner("testsigner1", null);
}

@Test(timeout = 10_000L)
public void testCustomSignerInitialization() {
Configuration config = new Configuration();
SignerForTest1.reset();
SignerForTest2.reset();
config.set(CUSTOM_SIGNERS, "testsigner1:" + SignerForTest1.class.getName());
initCustomSigners(config);
Signer s1 = SignerFactory.createSigner("testsigner1", null);
s1.sign(null, null);
Assert.assertEquals(true, SignerForTest1.initialized);
}

@Test(timeout = 10_000L)
public void testMultipleCustomSignerInitialization() {
Configuration config = new Configuration();
SignerForTest1.reset();
SignerForTest2.reset();
config.set(CUSTOM_SIGNERS,
"testsigner1:" + SignerForTest1.class.getName() + "," + "testsigner2:"
+ SignerForTest2.class.getName());
initCustomSigners(config);
Signer s1 = SignerFactory.createSigner("testsigner1", null);
s1.sign(null, null);
Assert.assertEquals(true, SignerForTest1.initialized);

Signer s2 = SignerFactory.createSigner("testsigner2", null);
s2.sign(null, null);
Assert.assertEquals(true, SignerForTest2.initialized);
}


/**
* SignerForTest1.
*/
@Private
public static class SignerForTest1 implements Signer {

private static boolean initialized = false;

@Override
public void sign(SignableRequest<?> request, AWSCredentials credentials) {
initialized = true;
}

public static void reset() {
initialized = false;
}
}

/**
* SignerForTest2.
*/
@Private
public static class SignerForTest2 implements Signer {

private static boolean initialized = false;

@Override
public void sign(SignableRequest<?> request, AWSCredentials credentials) {
initialized = true;
}

public static void reset() {
initialized = false;
}
}
}