HADOOP-13252. Tune S3A provider plugin mechanism. Contributed by Steve Loughran.

(cherry picked from commit 763f0497bb)
(cherry picked from commit 6dba38f08b)
This commit is contained in:
Chris Nauroth 2016-08-19 10:48:10 -07:00
parent 5d91f9fe35
commit dd6c149c4a
14 changed files with 850 additions and 193 deletions

View File

@ -779,11 +779,14 @@
<property>
<name>fs.s3a.aws.credentials.provider</name>
<description>
Class name of a credentials provider that implements
com.amazonaws.auth.AWSCredentialsProvider. Omit if using access/secret keys
or another authentication mechanism. The specified class must provide an
accessible constructor accepting java.net.URI and
org.apache.hadoop.conf.Configuration, or an accessible default constructor.
Comma-separated class names of credential provider classes which implement
com.amazonaws.auth.AWSCredentialsProvider.
These are loaded and queried in sequence for a valid set of credentials.
Each listed class must provide either an accessible constructor accepting
java.net.URI and org.apache.hadoop.conf.Configuration, or an accessible
default constructor.
Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
anonymous access to a publicly accessible S3 bucket without any credentials.
Please note that allowing anonymous access to an S3 bucket compromises
@ -794,7 +797,9 @@
<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>
<description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
as one of the providers.
</description>
</property>
<property>

View File

@ -109,10 +109,12 @@ public void setUp() throws Exception {
@After
public void tearDown() throws Exception {
if (fc != null) {
boolean del = fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
assertTrue(del);
fc.delete(localFsRootPath, true);
}
}
protected Path getDefaultWorkingDirectory() throws IOException {

View File

@ -63,8 +63,10 @@ public void setUp() throws Exception {
@After
public void tearDown() throws Exception {
if (fc != null) {
fc.delete(fileContextTestHelper.getTestRootPath(fc), true);
}
}
@Test
public void testFcCopy() throws Exception{

View File

@ -0,0 +1,188 @@
/*
* 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;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AnonymousAWSCredentials;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
/**
* A list of providers.
*
* This is similar to the AWS SDK {@code AWSCredentialsProviderChain},
* except that:
* <ol>
* <li>Allows extra providers to be added dynamically.</li>
* <li>If any provider in the chain throws an exception other than
* an {@link AmazonClientException}, that is rethrown, rather than
* swallowed.</li>
* <li>Has some more diagnostics.</li>
* <li>On failure, the last AmazonClientException raised is rethrown.</li>
* <li>Special handling of {@link AnonymousAWSCredentials}.</li>
* </ol>
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class AWSCredentialProviderList implements AWSCredentialsProvider {
private static final Logger LOG = LoggerFactory.getLogger(
AWSCredentialProviderList.class);
public static final String NO_AWS_CREDENTIAL_PROVIDERS
= "No AWS Credential Providers";
private final List<AWSCredentialsProvider> providers = new ArrayList<>(1);
private boolean reuseLastProvider = true;
private AWSCredentialsProvider lastProvider;
/**
* Empty instance. This is not ready to be used.
*/
public AWSCredentialProviderList() {
}
/**
* Create with an initial list of providers.
* @param providers provider list.
*/
public AWSCredentialProviderList(
Collection<AWSCredentialsProvider> providers) {
this.providers.addAll(providers);
}
/**
* Add a new provider.
* @param p provider
*/
public void add(AWSCredentialsProvider p) {
providers.add(p);
}
/**
* Reuse the last provider?
* @param reuseLastProvider flag to indicate the last provider should
* be re-used
*/
public void setReuseLastProvider(boolean reuseLastProvider) {
this.reuseLastProvider = reuseLastProvider;
}
/**
* query the {@link #reuseLastProvider} flag.
* @return the current flag state.
*/
public boolean isReuseLastProvider() {
return reuseLastProvider;
}
/**
* Refresh all child entries.
*/
@Override
public void refresh() {
for (AWSCredentialsProvider provider : providers) {
provider.refresh();
}
}
/**
* Iterate through the list of providers, to find one with credentials.
* If {@link #reuseLastProvider} is true, then it is re-used.
* @return a set of credentials (possibly anonymous), for authenticating.
*/
@Override
public AWSCredentials getCredentials() {
checkNotEmpty();
if (reuseLastProvider && lastProvider != null) {
return lastProvider.getCredentials();
}
AmazonClientException lastException = null;
for (AWSCredentialsProvider provider : providers) {
try {
AWSCredentials credentials = provider.getCredentials();
if ((credentials.getAWSAccessKeyId() != null &&
credentials.getAWSSecretKey() != null)
|| (credentials instanceof AnonymousAWSCredentials)) {
lastProvider = provider;
LOG.debug("Using credentials from {}", provider);
return credentials;
}
} catch (AmazonClientException e) {
lastException = e;
LOG.debug("No credentials provided by {}: {}",
provider, e.toString(), e);
}
}
// no providers had any credentials. Rethrow the last exception
// or create a new one.
String message = "No AWS Credentials provided by "
+ listProviderNames();
if (lastException != null) {
message += ": " + lastException;
}
throw new AmazonClientException(message, lastException);
}
/**
* Verify that the provider list is not empty.
* @throws AmazonClientException if there are no providers.
*/
public void checkNotEmpty() {
if (providers.isEmpty()) {
throw new AmazonClientException(NO_AWS_CREDENTIAL_PROVIDERS);
}
}
/**
* List all the providers' names.
* @return a list of names, separated by spaces (with a trailing one).
* If there are no providers, "" is returned.
*/
public String listProviderNames() {
StringBuilder sb = new StringBuilder(providers.size() * 32);
for (AWSCredentialsProvider provider : providers) {
sb.append(provider.getClass().getSimpleName());
sb.append(' ');
}
return sb.toString();
}
/**
* The string value is this class name and the string values of nested
* providers.
* @return a string value for debugging.
*/
@Override
public String toString() {
return "AWSCredentialProviderList: " +
StringUtils.join(providers, " ");
}
}

View File

@ -38,6 +38,10 @@
@InterfaceAudience.Private
@InterfaceStability.Stable
public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider {
public static final String NAME
= "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
public AWSCredentials getCredentials() {
return new AnonymousAWSCredentials();
}

View File

@ -29,16 +29,16 @@
* BasicAWSCredentialsProvider supports static configuration of access key ID
* and secret access key for use with the AWS SDK.
*
* Please note that users may reference this class name from configuration
* property fs.s3a.aws.credentials.provider. Therefore, changing the class name
* would be a backward-incompatible change.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class BasicAWSCredentialsProvider implements AWSCredentialsProvider {
public static final String NAME
= "org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider";
private final String accessKey;
private final String secretKey;
public BasicAWSCredentialsProvider(String accessKey, String secretKey) {
this.accessKey = accessKey;
this.secretKey = secretKey;

View File

@ -33,14 +33,11 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.Protocol;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSCredentialsProviderChain;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.S3ClientOptions;
import com.amazonaws.services.s3.model.AmazonS3Exception;
@ -152,7 +149,7 @@ public void initialize(URI name, Configuration conf) throws IOException {
bucket = name.getHost();
AWSCredentialsProvider credentials =
getAWSCredentialsProvider(name, conf);
createAWSCredentialProviderSet(name, conf, uri);
ClientConfiguration awsConf = new ClientConfiguration();
awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
@ -417,57 +414,6 @@ private void initMultipartUploads(Configuration conf) throws IOException {
}
}
/**
* Create the standard credential provider, or load in one explicitly
* identified in the configuration.
* @param binding the S3 binding/bucket.
* @param conf configuration
* @return a credential provider
* @throws IOException on any problem. Class construction issues may be
* nested inside the IOE.
*/
private AWSCredentialsProvider getAWSCredentialsProvider(URI binding,
Configuration conf) throws IOException {
AWSCredentialsProvider credentials;
String className = conf.getTrimmed(AWS_CREDENTIALS_PROVIDER);
if (StringUtils.isEmpty(className)) {
S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
credentials = new AWSCredentialsProviderChain(
new BasicAWSCredentialsProvider(
creds.getUser(), creds.getPassword()),
new InstanceProfileCredentialsProvider(),
new EnvironmentVariableCredentialsProvider());
} else {
try {
LOG.debug("Credential provider class is {}", className);
Class<?> credClass = Class.forName(className);
try {
credentials =
(AWSCredentialsProvider)credClass.getDeclaredConstructor(
URI.class, Configuration.class).newInstance(this.uri, conf);
} catch (NoSuchMethodException | SecurityException e) {
credentials =
(AWSCredentialsProvider)credClass.getDeclaredConstructor()
.newInstance();
}
} catch (ClassNotFoundException e) {
throw new IOException(className + " not found.", e);
} catch (NoSuchMethodException | SecurityException e) {
throw new IOException(String.format("%s constructor exception. A "
+ "class specified in %s must provide an accessible constructor "
+ "accepting URI and Configuration, or an accessible default "
+ "constructor.", className, AWS_CREDENTIALS_PROVIDER), e);
} catch (ReflectiveOperationException | IllegalArgumentException e) {
throw new IOException(className + " instantiation exception.", e);
}
LOG.debug("Using {} for {}.", credentials, this.uri);
}
return credentials;
}
/**
* Return the protocol scheme for the FileSystem.
*

View File

@ -20,6 +20,9 @@
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import org.apache.commons.lang.StringUtils;
@ -29,6 +32,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.security.ProviderUtils;
import org.slf4j.Logger;
import java.io.EOFException;
import java.io.FileNotFoundException;
@ -40,6 +44,7 @@
import java.util.concurrent.ExecutionException;
import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
/**
@ -49,6 +54,14 @@
@InterfaceStability.Evolving
public final class S3AUtils {
/** Reuse the S3AFileSystem log. */
private static final Logger LOG = S3AFileSystem.LOG;
static final String CONSTRUCTOR_EXCEPTION = "constructor exception";
static final String INSTANTIATION_EXCEPTION
= "instantiation exception";
static final String NOT_AWS_PROVIDER =
"does not implement AWSCredentialsProvider";
private S3AUtils() {
}
@ -243,6 +256,85 @@ public static long dateToLong(final Date date) {
return date.getTime();
}
/**
* Create the AWS credentials from the providers and the URI.
* @param binding Binding URI, may contain user:pass login details
* @param conf filesystem configuration
* @param fsURI fS URI after any login details have been stripped.
* @return a credentials provider list
* @throws IOException Problems loading the providers (including reading
* secrets from credential files).
*/
public static AWSCredentialProviderList createAWSCredentialProviderSet(
URI binding,
Configuration conf,
URI fsURI) throws IOException {
AWSCredentialProviderList credentials = new AWSCredentialProviderList();
Class<?>[] awsClasses;
try {
awsClasses = conf.getClasses(AWS_CREDENTIALS_PROVIDER);
} catch (RuntimeException e) {
Throwable c = e.getCause() != null ? e.getCause() : e;
throw new IOException("From option " + AWS_CREDENTIALS_PROVIDER +
' ' + c, c);
}
if (awsClasses.length == 0) {
S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
credentials.add(new BasicAWSCredentialsProvider(
creds.getUser(), creds.getPassword()));
credentials.add(new EnvironmentVariableCredentialsProvider());
credentials.add(new InstanceProfileCredentialsProvider());
} else {
for (Class<?> aClass : awsClasses) {
credentials.add(createAWSCredentialProvider(conf,
aClass,
fsURI));
}
}
return credentials;
}
/**
* Create an AWS credential provider.
* @param conf configuration
* @param credClass credential class
* @param uri URI of the FS
* @return the instantiated class
* @throws IOException on any instantiation failure.
*/
static AWSCredentialsProvider createAWSCredentialProvider(
Configuration conf,
Class<?> credClass,
URI uri) throws IOException {
AWSCredentialsProvider credentials;
String className = credClass.getName();
if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
}
try {
LOG.debug("Credential provider class is {}", className);
try {
credentials =
(AWSCredentialsProvider) credClass.getDeclaredConstructor(
URI.class, Configuration.class).newInstance(uri, conf);
} catch (NoSuchMethodException | SecurityException e) {
credentials =
(AWSCredentialsProvider) credClass.getDeclaredConstructor()
.newInstance();
}
} catch (NoSuchMethodException | SecurityException e) {
throw new IOException(String.format("%s " + CONSTRUCTOR_EXCEPTION
+". A class specified in %s must provide an accessible constructor "
+ "accepting URI and Configuration, or an accessible default "
+ "constructor.", className, AWS_CREDENTIALS_PROVIDER), e);
} catch (ReflectiveOperationException | IllegalArgumentException e) {
throw new IOException(className + " " + INSTANTIATION_EXCEPTION +".", e);
}
LOG.debug("Using {} for {}.", credentials, uri);
return credentials;
}
/**
* Return the access key and secret for S3 API use.
* Credentials may exist in configuration, within credential providers
@ -263,21 +355,40 @@ public static S3xLoginHelper.Login getAWSAccessKeys(URI name,
return new S3xLoginHelper.Login(accessKey, secretKey);
}
private static String getPassword(Configuration conf, String key, String val)
/**
* Get a password from a configuration, or, if a value is passed in,
* pick that up instead.
* @param conf configuration
* @param key key to look up
* @param val current value: if non empty this is used instead of
* querying the configuration.
* @return a password or "".
* @throws IOException on any problem
*/
static String getPassword(Configuration conf, String key, String val)
throws IOException {
return StringUtils.isEmpty(val)
? lookupPassword(conf, key, "")
: val;
}
/**
* Get a password from a configuration/configured credential providers.
* @param conf configuration
* @param key key to look up
* @param defVal value to return if there is no password
* @return a password or the value in {@code defVal}
* @throws IOException on any problem
*/
static String lookupPassword(Configuration conf, String key, String defVal)
throws IOException {
if (StringUtils.isEmpty(val)) {
try {
final char[] pass = conf.getPassword(key);
if (pass != null) {
return (new String(pass)).trim();
} else {
return "";
}
return pass != null ?
new String(pass).trim()
: defVal;
} catch (IOException ioe) {
throw new IOException("Cannot find password option " + key, ioe);
}
} else {
return val;
}
}
}

View File

@ -0,0 +1,86 @@
/**
* 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.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.ProviderUtils;
import java.io.IOException;
import java.net.URI;
import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
/**
* Support simple credentials for authenticating with AWS.
* Keys generated in URLs are not supported.
*
* Please note that users may reference this class name from configuration
* property fs.s3a.aws.credentials.provider. Therefore, changing the class name
* would be a backward-incompatible change.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
public static final String NAME
= "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider";
private String accessKey;
private String secretKey;
private IOException lookupIOE;
public SimpleAWSCredentialsProvider(URI uri, Configuration conf) {
try {
Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
conf, S3AFileSystem.class);
this.accessKey = S3AUtils.lookupPassword(c, ACCESS_KEY, null);
this.secretKey = S3AUtils.lookupPassword(c, SECRET_KEY, null);
} catch (IOException e) {
lookupIOE = e;
}
}
public AWSCredentials getCredentials() {
if (lookupIOE != null) {
// propagate any initialization problem
throw new CredentialInitializationException(lookupIOE.toString(),
lookupIOE);
}
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
return new BasicAWSCredentials(accessKey, secretKey);
}
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

@ -23,34 +23,52 @@
import com.amazonaws.auth.AWSCredentials;
import org.apache.commons.lang.StringUtils;
import java.io.IOException;
import java.net.URI;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.ProviderUtils;
import static org.apache.hadoop.fs.s3a.Constants.*;
/**
* Support session credentials for authenticating with AWS.
*
* Please note that users may reference this class name from configuration
* property fs.s3a.aws.credentials.provider. Therefore, changing the class name
* would be a backward-incompatible change.
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@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;
private String accessKey;
private String secretKey;
private String sessionToken;
private IOException lookupIOE;
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);
try {
Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
conf, S3AFileSystem.class);
this.accessKey = S3AUtils.lookupPassword(c, ACCESS_KEY, null);
this.secretKey = S3AUtils.lookupPassword(c, SECRET_KEY, null);
this.sessionToken = S3AUtils.lookupPassword(c, SESSION_TOKEN, null);
} catch (IOException e) {
lookupIOE = e;
}
}
public AWSCredentials getCredentials() {
if (lookupIOE != null) {
// propagate any initialization problem
throw new CredentialInitializationException(lookupIOE.toString(),
lookupIOE);
}
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
&& !StringUtils.isEmpty(sessionToken)) {
return new BasicSessionCredentials(accessKey, secretKey, sessionToken);

View File

@ -273,26 +273,37 @@ one where patches for functionality and performance are very welcome.
* `httpclient` jar.
* Jackson `jackson-core`, `jackson-annotations`, `jackson-databind` jars.
### S3A Authentication methods
S3A supports multiple authentication mechanisms, and can be configured as to
which mechanisms to use, and the order to use them. Custom implementations
of `com.amazonaws.auth.AWSCredentialsProvider` may also be used.
### Authentication properties
<property>
<name>fs.s3a.access.key</name>
<description>AWS access key ID. Omit for IAM role-based or provider-based authentication.</description>
<description>AWS access key ID.
Omit for IAM role-based or provider-based authentication.</description>
</property>
<property>
<name>fs.s3a.secret.key</name>
<description>AWS secret key. Omit for IAM role-based or provider-based authentication.</description>
<description>AWS secret key.
Omit for IAM role-based or provider-based authentication.</description>
</property>
<property>
<name>fs.s3a.aws.credentials.provider</name>
<description>
Class name of a credentials provider that implements
com.amazonaws.auth.AWSCredentialsProvider. Omit if using access/secret keys
or another authentication mechanism. The specified class must provide an
accessible constructor accepting java.net.URI and
org.apache.hadoop.conf.Configuration, or an accessible default constructor.
Comma-separated class names of credential provider classes which implement
com.amazonaws.auth.AWSCredentialsProvider.
These are loaded and queried in sequence for a valid set of credentials.
Each listed class must provide either an accessible constructor accepting
java.net.URI and org.apache.hadoop.conf.Configuration, or an accessible
default constructor.
Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
anonymous access to a publicly accessible S3 bucket without any credentials.
Please note that allowing anonymous access to an S3 bucket compromises
@ -303,24 +314,106 @@ one where patches for functionality and performance are very welcome.
<property>
<name>fs.s3a.session.token</name>
<description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider as the providers.</description>
<description>
Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
as one of the providers.
</description>
</property>
#### Authentication methods
#### Authenticating via environment variables
S3A supports configuration via [the standard AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment).
The core environment variables are for the access key and associated secret:
```
export AWS_ACCESS_KEY_ID=my.aws.key
export AWS_SECRET_ACCESS_KEY=my.secret.key
```
If the environment variable `AWS_SESSION_TOKEN` is set, session authentication
using "Temporary Security Credentials" is enabled; the Key ID and secret key
must be set to the credentials for that specific sesssion.
```
export AWS_SESSION_TOKEN=SECRET-SESSION-TOKEN
export AWS_ACCESS_KEY_ID=SESSION-ACCESS-KEY
export AWS_SECRET_ACCESS_KEY=SESSION-SECRET-KEY
```
These environment variables can be used to set the authentication credentials
instead of properties in the Hadoop configuration.
*Important:*
These environment variables are not propagated from client to server when
YARN applications are launched. That is: having the AWS environment variables
set when an application is launched will not permit the launched application
to access S3 resources. The environment variables must (somehow) be set
on the hosts/processes where the work is executed.
#### Changing Authentication Providers
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.
properties in the configuration file.
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.
The S3A client follows the following authentication chain:
1. If login details were provided in the filesystem URI, a warning is printed
and then the username and password extracted for the AWS key and secret respectively.
1. The `fs.s3a.access.key` and `fs.s3a.secret.key` are looked for in the Hadoop
XML configuration.
1. The [AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment),
are then looked for.
1. An attempt is made to query the Amazon EC2 Instance Metadata Service to
retrieve credentials published to EC2 VMs.
S3A can be configured to obtain client authentication providers from classes
which integrate with the AWS SDK by implementing the `com.amazonaws.auth.AWSCredentialsProvider`
Interface. This is done by listing the implementation classes, in order of
preference, in the configuration option `fs.s3a.aws.credentials.provider`.
*Important*: AWS Credential Providers are distinct from _Hadoop Credential Providers_.
As will be covered later, Hadoop Credential Providers allow passwords and other secrets
to be stored and transferred more securely than in XML configuration files.
AWS Credential Providers are classes which can be used by the Amazon AWS SDK to
obtain an AWS login from a different source in the system, including environment
variables, JVM properties and configuration files.
There are three AWS Credential Providers inside the `hadoop-aws` JAR:
| classname | description |
|-----------|-------------|
| `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider`| Session Credentials |
| `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider`| Simple name/secret credentials |
| `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider`| Anonymous Login |
There are also many in the Amazon SDKs, in particular two which are automatically
set up in the authentication chain:
| classname | description |
|-----------|-------------|
| `com.amazonaws.auth.InstanceProfileCredentialsProvider`| EC2 Metadata Credentials |
| `com.amazonaws.auth.EnvironmentVariableCredentialsProvider`| AWS Environment Variables |
*Session Credentials with `TemporaryAWSCredentialsProvider`*
[Temporary Security Credentials](http://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp.html)
can be obtained from the Amazon Security Token Service; these
consist of an access key, a secret key, and a session token.
To authenticate with these:
1. Declare `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider` as the
provider.
1. Set the session key in the property `fs.s3a.session.token`,
and the access and secret key properties to those of this temporary session.
Example:
```xml
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</value>
@ -340,71 +433,174 @@ key properties to those of this temporary session.
<name>fs.s3a.session.token</name>
<value>SECRET-SESSION-TOKEN</value>
</property>
```
#### Protecting the AWS Credentials in S3A
The lifetime of session credentials are fixed when the credentials
are issued; once they expire the application will no longer be able to
authenticate to AWS.
*Anonymous Login with `AnonymousAWSCredentialsProvider`*
Specifying `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` allows
anonymous access to a publicly accessible S3 bucket without any credentials.
It can be useful for accessing public data sets without requiring AWS credentials.
```xml
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider</value>
</property>
```
Once this is done, there's no need to supply any credentials
in the Hadoop configuration or via environment variables.
This option can be used to verify that an object store does
not permit unauthenticated access: that is, if an attempt to list
a bucket is made using the anonymous credentials, it should fail —unless
explicitly opened up for broader access.
```bash
hadoop fs -ls \
-D fs.s3a.aws.credentials.provider=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider \
s3a://landsat-pds/
```
1. Allowing anonymous access to an S3 bucket compromises
security and therefore is unsuitable for most use cases.
1. If a list of credential providers is given in `fs.s3a.aws.credentials.provider`,
then the Anonymous Credential provider *must* come last. If not, credential
providers listed after it will be ignored.
*Simple name/secret credentials with `SimpleAWSCredentialsProvider`*
This is is the standard credential provider, which
supports the secret key in `fs.s3a.access.key` and token in `fs.s3a.secret.key`
values. It does not support authentication with logins credentials declared
in the URLs.
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
</property>
Apart from its lack of support of user:password details being included in filesystem
URLs (a dangerous practise that is strongly discouraged), this provider acts
exactly at the basic authenticator used in the default authentication chain.
This means that the default S3A authentication chain can be defined as
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>
org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
com.amazonaws.auth.InstanceProfileCredentialsProvider
</value>
</property>
#### Protecting the AWS Credentials
To protect the access/secret keys from prying eyes, it is recommended that you
use either IAM role-based authentication (such as EC2 instance profile) or
the credential provider framework securely storing them and accessing them
through configuration. The following describes using the latter for AWS
credentials in S3AFileSystem.
credentials in the S3A FileSystem.
For additional reading on the credential provider API see:
##### Storing secrets with Hadoop Credential Providers
The Hadoop Credential Provider Framework allows secure "Credential Providers"
to keep secrets outside Hadoop configuration files, storing them in encrypted
files in local or Hadoop filesystems, and including them in requests.
The S3A configuration options with sensitive data
(`fs.s3a.secret.key`, `fs.s3a.access.key` and `fs.s3a.session.token`) can
have their data saved to a binary file stored, with the values being read in
when the S3A filesystem URL is used for data access. The reference to this
credential provider is all that is passed as a direct configuration option.
For additional reading on the Hadoop Credential Provider API see:
[Credential Provider API](../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html).
#### Authenticating via environment variables
S3A supports configuration via [the standard AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment).
###### Create a credential file
The core environment variables are for the access key and associated secret:
A credential file can be created on any Hadoop filesystem; when creating one on HDFS or
a Unix filesystem the permissions are automatically set to keep the file
private to the reader —though as directory permissions are not touched,
users should verify that the directory containing the file is readable only by
the current user.
```
export AWS_ACCESS_KEY_ID=my.aws.key
export AWS_SECRET_ACCESS_KEY=my.secret.key
```bash
hadoop credential create fs.s3a.access.key -value 123 \
-provider jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks
hadoop credential create fs.s3a.secret.key -value 456 \
-provider jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks
```
These environment variables can be used to set the authentication credentials
instead of properties in the Hadoop configuration. *Important:* these
environment variables are not propagated from client to server when
YARN applications are launched. That is: having the AWS environment variables
set when an application is launched will not permit the launched application
to access S3 resources. The environment variables must (somehow) be set
on the hosts/processes where the work is executed.
A credential file can be listed, to see what entries are kept inside it
##### End to End Steps for Distcp and S3 with Credential Providers
###### provision
```bash
hadoop credential list -provider jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks
Listing aliases for CredentialProvider: jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks
fs.s3a.secret.key
fs.s3a.access.key
```
% hadoop credential create fs.s3a.access.key -value 123
-provider localjceks://file/home/lmccay/aws.jceks
```
At this point, the credentials are ready for use.
```
% hadoop credential create fs.s3a.secret.key -value 456
-provider localjceks://file/home/lmccay/aws.jceks
```
###### Configure the `hadoop.security.credential.provider.path` property
###### configure core-site.xml or command line system property
The URL to the provider must be set in the configuration property
`hadoop.security.credential.provider.path`, either on the command line or
in XML configuration files.
```
```xml
<property>
<name>hadoop.security.credential.provider.path</name>
<value>localjceks://file/home/lmccay/aws.jceks</value>
<value>jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks</value>
<description>Path to interrogate for protected credentials.</description>
</property>
```
###### distcp
Because this property only supplies the path to the secrets file, the configuration
option itself is no longer a sensitive item.
###### Using the credentials
Once the provider is set in the Hadoop configuration, hadoop commands
work exactly as if the secrets were in an XML file.
```bash
hadoop distcp \
hdfs://nn1.example.com:9001/user/backup/007020615 s3a://glacier1/
hadoop fs -ls s3a://glacier1/
```
% hadoop distcp
[-D hadoop.security.credential.provider.path=localjceks://file/home/lmccay/aws.jceks]
hdfs://hostname:9001/user/lmccay/007020615 s3a://lmccay/
The path to the provider can also be set on the command line:
```bash
hadoop distcp \
-D hadoop.security.credential.provider.path=jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks \
hdfs://nn1.example.com:9001/user/backup/007020615 s3a://glacier1/
hadoop fs \
-D hadoop.security.credential.provider.path=jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks \
-ls s3a://glacier1/
```
NOTE: You may optionally add the provider path property to the distcp command line instead of
added job specific configuration to a generic core­site.xml. The square brackets above illustrate
this capability.
Because the provider path is not itself a sensitive secret, there is no risk
from placing its declaration on the command line.
### Other properties
@ -821,7 +1017,7 @@ the classpath.
### `ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client`
(or other `com.amazonaws` class.)
`
This means that one or more of the `aws-*-sdk` JARs are missing. Add them.
### Missing method in AWS class
@ -848,21 +1044,15 @@ classpath. All Jackson JARs on the classpath *must* be of the same version.
### Authentication failure
One authentication problem is caused by classpath mismatch; see the joda time
issue above.
Otherwise, the general cause is: you have the wrong credentials —or somehow
The general cause is: you have the wrong credentials —or somehow
the credentials were not readable on the host attempting to read or write
the S3 Bucket.
There's not much that Hadoop can do/does for diagnostics here,
though enabling debug logging for the package `org.apache.hadoop.fs.s3a`
can help.
There's not much that Hadoop can do for diagnostics here.
Enabling debug logging for the package `org.apache.hadoop.fs.s3a`
can help somewhat.
There is also some logging in the AWS libraries which provide some extra details.
In particular, the setting the log `com.amazonaws.auth.AWSCredentialsProviderChain`
to log at DEBUG level will mean the invidual reasons for the (chained)
authentication clients to fail will be printed.
Most common: there's an error in the key or secret.
Otherwise, try to use the AWS command line tools with the same credentials.
If you set the environment variables, you can take advantage of S3A's support
@ -870,15 +1060,38 @@ of environment-variable authentication by attempting to use the `hdfs fs` comman
to read or write data on S3. That is: comment out the `fs.s3a` secrets and rely on
the environment variables.
### Authentication failure when using URLs with embedded secrets
If using the (strongly discouraged) mechanism of including the
AWS Key and secret in a URL, then both "+" and "/" symbols need
to encoded in the URL. As many AWS secrets include these characters,
encoding problems are not uncommon.
| symbol | encoded value|
|-----------|-------------|
| `+` | `%2B` |
| `/` | `%2F` |
That is, a URL for `bucket` with AWS ID `user1` and secret `a+b/c` would
be represented as
```
s3a://user1:a%2Bb%2Fc@bucket
```
This technique is only needed when placing secrets in the URL. Again,
this is something users are strongly advised against using.
### Authentication failures running on Java 8u60+
A change in the Java 8 JVM broke some of the `toString()` string generation
of Joda Time 2.8.0, which stopped the amazon s3 client from being able to
of Joda Time 2.8.0, which stopped the Amazon S3 client from being able to
generate authentication headers suitable for validation by S3.
Fix: make sure that the version of Joda Time is 2.8.1 or later.
### "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or elsewhere
### "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or other "V4" endpoint
S3 Frankfurt and Seoul *only* support
@ -920,12 +1133,14 @@ ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Ex
```
This happens when trying to work with any S3 service which only supports the
"V4" signing API —and he client is configured to use the default S3A service
"V4" signing API —but the client is configured to use the default S3A service
endpoint.
The S3A client needs to be given the endpoint to use via the `fs.s3a.endpoint`
property.
As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
```xml
<property>
<name>fs.s3a.endpoint</name>

View File

@ -18,10 +18,6 @@
package org.apache.hadoop.fs.s3a;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
import static org.junit.Assert.*;
import java.io.IOException;
import java.net.URI;
import java.nio.file.AccessDeniedException;
@ -43,6 +39,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.junit.Assert.*;
/**
* Tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic.
*
@ -57,6 +58,18 @@ public class TestS3AAWSCredentialsProvider {
@Rule
public ExpectedException exception = ExpectedException.none();
/**
* Declare what exception to raise, and the text which must be found
* in it.
* @param exceptionClass class of exception
* @param text text in exception
*/
private void expectException(Class<? extends Throwable> exceptionClass,
String text) {
exception.expect(exceptionClass);
exception.expectMessage(text);
}
@Test
public void testBadConfiguration() throws IOException {
Configuration conf = new Configuration();
@ -142,10 +155,35 @@ public void testAnonymousProvider() throws Exception {
assertEquals(testFile, stat.getPath());
}
static class ConstructorErrorProvider implements AWSCredentialsProvider {
/**
* A credential provider whose constructor signature doesn't match.
*/
static class ConstructorSignatureErrorProvider
implements AWSCredentialsProvider {
@SuppressWarnings("unused")
public ConstructorErrorProvider(String str) {
public ConstructorSignatureErrorProvider(String str) {
}
@Override
public AWSCredentials getCredentials() {
return null;
}
@Override
public void refresh() {
}
}
/**
* A credential provider whose constructor raises an NPE.
*/
static class ConstructorFailureProvider
implements AWSCredentialsProvider {
@SuppressWarnings("unused")
public ConstructorFailureProvider() {
throw new NullPointerException("oops");
}
@Override
@ -159,14 +197,54 @@ public void refresh() {
}
@Test
public void testProviderConstructorError() throws Exception {
public void testProviderWrongClass() throws Exception {
expectProviderInstantiationFailure(this.getClass().getName(),
NOT_AWS_PROVIDER);
}
@Test
public void testProviderNotAClass() throws Exception {
expectProviderInstantiationFailure("NoSuchClass",
"ClassNotFoundException");
}
private void expectProviderInstantiationFailure(String option,
String expectedErrorText) throws IOException {
Configuration conf = new Configuration();
conf.set(AWS_CREDENTIALS_PROVIDER,
ConstructorErrorProvider.class.getName());
conf.set(AWS_CREDENTIALS_PROVIDER, option);
Path testFile = new Path(
conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
exception.expect(IOException.class);
exception.expectMessage("constructor exception");
FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
expectException(IOException.class, expectedErrorText);
URI uri = testFile.toUri();
S3AUtils.createAWSCredentialProviderSet(uri, conf, uri);
}
@Test
public void testProviderConstructorError() throws Exception {
expectProviderInstantiationFailure(
ConstructorSignatureErrorProvider.class.getName(),
CONSTRUCTOR_EXCEPTION);
}
@Test
public void testProviderFailureError() throws Exception {
expectProviderInstantiationFailure(
ConstructorFailureProvider.class.getName(),
INSTANTIATION_EXCEPTION);
}
@Test
public void testInstantiationChain() throws Throwable {
Configuration conf = new Configuration();
conf.set(AWS_CREDENTIALS_PROVIDER,
TemporaryAWSCredentialsProvider.NAME
+ ", \t" + SimpleAWSCredentialsProvider.NAME
+ " ,\n " + AnonymousAWSCredentialsProvider.NAME);
Path testFile = new Path(
conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
URI uri = testFile.toUri();
S3AUtils.createAWSCredentialProviderSet(uri, conf, uri);
}
}

View File

@ -55,8 +55,8 @@ public class TestS3ATemporaryCredentials extends AbstractFSContractTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestS3ATemporaryCredentials.class);
private static final String PROVIDER_CLASS =
"org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
private static final String PROVIDER_CLASS
= TemporaryAWSCredentialsProvider.NAME;
private static final long TEST_FILE_SIZE = 1024;

View File

@ -50,8 +50,10 @@ public void setUp() throws Exception {
@After
public void tearDown() throws Exception {
if (fc != null) {
fc.delete(getTestPath(), true);
}
}
protected Path getTestPath() {
return new Path("/tests3afc");