HADOOP-12537 S3A to support Amazon STS temporary credentials. Contributed by Sean Mackrory.

This commit is contained in:
Steve Loughran 2016-06-09 20:58:30 +01:00
parent 9378d9428f
commit 31ffaf76f2
9 changed files with 357 additions and 4 deletions

View File

@ -805,6 +805,11 @@
</description> </description>
</property> </property>
<property>
<name>fs.s3a.session.token</name>
<description>The session token used with temporary credentials. Used only with provider org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider.</description>
</property>
<property> <property>
<name>fs.s3a.connection.maximum</name> <name>fs.s3a.connection.maximum</name>
<value>15</value> <value>15</value>

View File

@ -116,6 +116,7 @@
<make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version> <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
<native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version> <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
<surefire.fork.timeout>900</surefire.fork.timeout> <surefire.fork.timeout>900</surefire.fork.timeout>
<aws-java-sdk.version>1.10.6</aws-java-sdk.version>
</properties> </properties>
<dependencyManagement> <dependencyManagement>
@ -690,7 +691,12 @@
<dependency> <dependency>
<groupId>com.amazonaws</groupId> <groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-s3</artifactId> <artifactId>aws-java-sdk-s3</artifactId>
<version>1.10.6</version> <version>${aws-java-sdk.version}</version>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-sts</artifactId>
<version>${aws-java-sdk.version}</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.mina</groupId> <groupId>org.apache.mina</groupId>

View File

@ -230,6 +230,11 @@
<artifactId>aws-java-sdk-s3</artifactId> <artifactId>aws-java-sdk-s3</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-sts</artifactId>
<scope>test</scope>
</dependency>
<dependency> <dependency>
<groupId>junit</groupId> <groupId>junit</groupId>
<artifactId>junit</artifactId> <artifactId>junit</artifactId>

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.fs.s3a; package org.apache.hadoop.fs.s3a;
import com.amazonaws.AmazonClientException;
import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentials;
@ -49,7 +48,7 @@ public AWSCredentials getCredentials() {
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) { if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
return new BasicAWSCredentials(accessKey, secretKey); return new BasicAWSCredentials(accessKey, secretKey);
} }
throw new AmazonClientException( throw new CredentialInitializationException(
"Access key or secret key is null"); "Access key or secret key is null");
} }

View File

@ -41,6 +41,9 @@ private Constants() {
public static final String AWS_CREDENTIALS_PROVIDER = public static final String AWS_CREDENTIALS_PROVIDER =
"fs.s3a.aws.credentials.provider"; "fs.s3a.aws.credentials.provider";
// session token for when using TemporaryAWSCredentialsProvider
public static final String SESSION_TOKEN = "fs.s3a.session.token";
// number of simultaneous connections to s3 // number of simultaneous connections to s3
public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15; public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15;

View File

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a;
import com.amazonaws.AmazonClientException;
/**
* Exception which Hadoop's AWSCredentialsProvider implementations should
* throw when there is a problem with the credential setup. This
* is a subclass of {@link AmazonClientException} which sets
* {@link #isRetryable()} to false, so as to fail fast.
*/
public class CredentialInitializationException extends AmazonClientException {
public CredentialInitializationException(String message, Throwable t) {
super(message, t);
}
public CredentialInitializationException(String message) {
super(message);
}
/**
* This exception is not going to go away if you try calling it again.
* @return false, always.
*/
@Override
public boolean isRetryable() {
return false;
}
}

View File

@ -0,0 +1,70 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.BasicSessionCredentials;
import com.amazonaws.auth.AWSCredentials;
import org.apache.commons.lang.StringUtils;
import java.net.URI;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import static org.apache.hadoop.fs.s3a.Constants.*;
/**
* Support session credentials for authenticating with AWS.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class TemporaryAWSCredentialsProvider implements AWSCredentialsProvider {
public static final String NAME
= "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
private final String accessKey;
private final String secretKey;
private final String sessionToken;
public TemporaryAWSCredentialsProvider(URI uri, Configuration conf) {
this.accessKey = conf.get(ACCESS_KEY, null);
this.secretKey = conf.get(SECRET_KEY, null);
this.sessionToken = conf.get(SESSION_TOKEN, null);
}
public AWSCredentials getCredentials() {
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
&& !StringUtils.isEmpty(sessionToken)) {
return new BasicSessionCredentials(accessKey, secretKey, sessionToken);
}
throw new CredentialInitializationException(
"Access key, secret key or session token is unset");
}
@Override
public void refresh() {}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -201,6 +201,46 @@ If you do any of these: change your credentials immediately!
</description> </description>
</property> </property>
<property>
<name>fs.s3a.session.token</name>
<description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider as the providers.</description>
</property>
#### Authentication methods
The standard way to authenticate is with an access key and secret key using the
properties above. You can also avoid configuring credentials if the EC2
instances in your cluster are configured with IAM instance profiles that grant
the appropriate S3 access.
A temporary set of credentials can also be obtained from Amazon STS; these
consist of an access key, a secret key, and a session token. To use these
temporary credentials you must include the `aws-java-sdk-sts` JAR in your
classpath (consult the POM for the current version) and set the
`TemporaryAWSCredentialsProvider` class as the provider. The session key
must be set in the property `fs.s3a.session.token` —and the access and secret
key properties to those of this temporary session.
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</value>
</property>
<property>
<name>fs.s3a.access.key</name>
<value>SESSION-ACCESS-KEY</value>
</property>
<property>
<name>fs.s3a.secret.key</name>
<value>SESSION-SECRET-KEY</value>
</property>
<property>
<name>fs.s3a.session.token</name>
<value>SECRET-SESSION-TOKEN</value>
</property>
#### Protecting the AWS Credentials in S3A #### Protecting the AWS Credentials in S3A
To protect the access/secret keys from prying eyes, it is recommended that you To protect the access/secret keys from prying eyes, it is recommended that you
@ -605,6 +645,13 @@ Example:
<description>AWS secret key. Omit for IAM role-based authentication.</description> <description>AWS secret key. Omit for IAM role-based authentication.</description>
<value>DONOTEVERSHARETHISSECRETKEY!</value> <value>DONOTEVERSHARETHISSECRETKEY!</value>
</property> </property>
<property>
<name>test.sts.endpoint</name>
<description>Specific endpoint to use for STS requests.</description>
<value>sts.amazonaws.com</value>
</property>
</configuration> </configuration>
### File `contract-test-options.xml` ### File `contract-test-options.xml`
@ -714,8 +761,30 @@ that the file `contract-test-options.xml` does not contain any
secret credentials itself. As the auth keys XML file is kept out of the secret credentials itself. As the auth keys XML file is kept out of the
source code tree, it is not going to get accidentally committed. source code tree, it is not going to get accidentally committed.
### Running Performance Tests against non-AWS storage infrastructures ### Running Tests against non-AWS storage infrastructures
### S3A session tests
The test `TestS3ATemporaryCredentials` requests a set of temporary
credentials from the STS service, then uses them to authenticate with S3.
If an S3 implementation does not support STS, then the functional test
cases must be disabled:
<property>
<name>test.fs.s3a.sts.enabled</name>
<value>false</value>
</property>
These tests reqest a temporary set of credentials from the STS service endpoint.
An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
<property>
<name>test.fs.s3a.sts.endpoint</name>
<value>https://sts.example.org/</value>
</property>
The default is ""; meaning "use the amazon default value".
#### CSV Data source #### CSV Data source

View File

@ -0,0 +1,150 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a;
import java.io.IOException;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProviderChain;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
import com.amazonaws.services.securitytoken.model.GetSessionTokenResult;
import com.amazonaws.services.securitytoken.model.Credentials;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
import org.apache.hadoop.fs.contract.s3a.S3AContract;
import org.apache.hadoop.conf.Configuration;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
import static org.apache.hadoop.fs.s3a.Constants.*;
/**
* Tests use of temporary credentials (for example, AWS STS & S3).
* This test extends a class that "does things to the root directory", and
* should only be used against transient filesystems where you don't care about
* the data.
*/
public class TestS3ATemporaryCredentials extends AbstractFSContractTestBase {
public static final String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
public static final String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
private static final Logger LOG =
LoggerFactory.getLogger(TestS3ATemporaryCredentials.class);
private S3AFileSystem fs;
private static final String PROVIDER_CLASS =
"org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
private static final long TEST_FILE_SIZE = 1024;
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new S3AContract(conf);
}
/**
* Test use of STS for requesting temporary credentials.
*
* The property test.sts.endpoint can be set to point this at different
* STS endpoints. This test will use the AWS credentials (if provided) for
* S3A tests to request temporary credentials, then attempt to use those
* credentials instead.
*
* @throws IOException
*/
@Test
public void testSTS() throws IOException {
Configuration conf = getContract().getConf();
if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
skip("STS functional tests disabled");
}
String parentAccessKey = conf.getTrimmed(ACCESS_KEY, null);
String parentSecretKey = conf.getTrimmed(SECRET_KEY, null);
String stsEndpoint = conf.getTrimmed(TEST_STS_ENDPOINT, "");
AWSCredentialsProviderChain parentCredentials;
parentCredentials = new AWSCredentialsProviderChain(
new BasicAWSCredentialsProvider(parentAccessKey, parentSecretKey),
new InstanceProfileCredentialsProvider()
);
AWSSecurityTokenServiceClient stsClient;
stsClient = new AWSSecurityTokenServiceClient(parentCredentials);
if (!stsEndpoint.isEmpty()) {
LOG.debug("STS Endpoint ={}", stsEndpoint);
stsClient.setEndpoint(stsEndpoint);
}
GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
sessionTokenRequest.setDurationSeconds(900);
GetSessionTokenResult sessionTokenResult;
sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
Credentials sessionCreds = sessionTokenResult.getCredentials();
String childAccessKey = sessionCreds.getAccessKeyId();
conf.set(ACCESS_KEY, childAccessKey);
String childSecretKey = sessionCreds.getSecretAccessKey();
conf.set(SECRET_KEY, childSecretKey);
String sessionToken = sessionCreds.getSessionToken();
conf.set(SESSION_TOKEN, sessionToken);
conf.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
}
// now create an invalid set of credentials by changing the session
// token
conf.set(SESSION_TOKEN, "invalid-" + sessionToken);
try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE);
fail("Expected an access exception, but file access to "
+ fs.getUri() + " was allowed: " + fs);
} catch (AWSS3IOException ex) {
LOG.info("Expected Exception: {}", ex.toString());
LOG.debug("Expected Exception: {}", ex, ex);
}
}
@Test
public void testTemporaryCredentialValidation() throws Throwable {
Configuration conf = new Configuration();
conf.set(ACCESS_KEY, "accesskey");
conf.set(SECRET_KEY, "secretkey");
conf.set(SESSION_TOKEN, "");
TemporaryAWSCredentialsProvider provider
= new TemporaryAWSCredentialsProvider(getFileSystem().getUri(), conf);
try {
AWSCredentials credentials = provider.getCredentials();
fail("Expected a CredentialInitializationException,"
+ " got " + credentials);
} catch (CredentialInitializationException expected) {
// expected
}
}
}