HADOOP-14556. S3A to support Delegation Tokens.

Contributed by Steve Loughran and Daryn Sharp.
This commit is contained in:
Steve Loughran 2019-01-14 17:59:27 +00:00
parent c4a00d1ad3
commit 6d0bffe17e
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
100 changed files with 11342 additions and 772 deletions

View File

@ -396,8 +396,11 @@ public void checkPath(Path path) {
thatPort = this.getUriDefaultPort();
}
if (thisPort != thatPort) {
throw new InvalidPathException("Wrong FS: " + path + ", expected: "
+ this.getUri());
throw new InvalidPathException("Wrong FS: " + path
+ " and port=" + thatPort
+ ", expected: "
+ this.getUri()
+ " with port=" + thisPort);
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.util.Iterator;
@ -37,9 +38,13 @@ public abstract class StorageStatistics {
*
* When adding new common statistic name constants, please make them unique.
* By convention, they are implicitly unique:
* - the name of the constants are uppercase, words separated by underscores.
* - the value of the constants are lowercase of the constant names.
* <ul>
* <li>the name of the constants are uppercase, words separated by
* underscores.</li>
* <li>the value of the constants are lowercase of the constant names.</li>
* </ul>
*/
@InterfaceStability.Evolving
public interface CommonStatisticNames {
// The following names are for file system operation invocations
String OP_APPEND = "op_append";
@ -49,6 +54,7 @@ public interface CommonStatisticNames {
String OP_DELETE = "op_delete";
String OP_EXISTS = "op_exists";
String OP_GET_CONTENT_SUMMARY = "op_get_content_summary";
String OP_GET_DELEGATION_TOKEN = "op_get_delegation_token";
String OP_GET_FILE_CHECKSUM = "op_get_file_checksum";
String OP_GET_FILE_STATUS = "op_get_file_status";
String OP_GET_STATUS = "op_get_status";

View File

@ -1024,19 +1024,33 @@
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>
org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
</value>
<description>
Comma-separated class names of credential provider classes which implement
com.amazonaws.auth.AWSCredentialsProvider.
When S3A delegation tokens are not enabled, this list will be used
to directly authenticate with S3 and DynamoDB services.
When S3A Delegation tokens are enabled, depending upon the delegation
token binding it may be used
to communicate wih the STS endpoint to request session/role
credentials.
These are loaded and queried in sequence for a valid set of credentials.
Each listed class must implement one of the following means of
construction, which are attempted in order:
1. a public constructor accepting java.net.URI and
* a public constructor accepting java.net.URI and
org.apache.hadoop.conf.Configuration,
2. a public static method named getInstance that accepts no
* a public constructor accepting org.apache.hadoop.conf.Configuration,
* a public static method named getInstance that accepts no
arguments and returns an instance of
com.amazonaws.auth.AWSCredentialsProvider, or
3. a public default constructor.
* a public default constructor.
Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
anonymous access to a publicly accessible S3 bucket without any credentials.
@ -1046,13 +1060,15 @@
If unspecified, then the default list of credential provider classes,
queried in sequence, is:
1. org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
* org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider: looks
for session login secrets in the Hadoop configuration.
* org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
Uses the values of fs.s3a.access.key and fs.s3a.secret.key.
2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
* com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
configuration of AWS access key ID and secret access key in
environment variables named AWS_ACCESS_KEY_ID and
AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
3. com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
* com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
of instance profile credentials if running in an EC2 VM.
</description>
</property>
@ -1108,7 +1124,7 @@
<value>30m</value>
<description>
Duration of assumed roles before a refresh is attempted.
Only used if AssumedRoleCredentialProvider is the AWS credential provider.
Used when session tokens are requested.
Range: 15m to 1h
</description>
</property>
@ -1120,17 +1136,20 @@
AWS Security Token Service Endpoint.
If unset, uses the default endpoint.
Only used if AssumedRoleCredentialProvider is the AWS credential provider.
Used by the AssumedRoleCredentialProvider and in Session and Role delegation
tokens.
</description>
</property>
<property>
<name>fs.s3a.assumed.role.sts.endpoint.region</name>
<value>us-west-1</value>
<value></value>
<description>
AWS Security Token Service Endpoint's region;
Needed if fs.s3a.assumed.role.sts.endpoint points to an endpoint
other than the default one and the v4 signature is used.
Only used if AssumedRoleCredentialProvider is the AWS credential provider.
Used by the AssumedRoleCredentialProvider and in Session and Role delegation
tokens.
</description>
</property>
@ -1145,6 +1164,29 @@
</description>
</property>
<property>
<name>fs.s3a.delegation.tokens.enabled</name>
<value>false</value>
<description></description>
</property>
<property>
<name>fs.s3a.delegation.token.binding</name>
<value></value>
<description>
The name of a class to provide delegation tokens support in S3A.
If unset: delegation token support is disabled.
Note: for job submission to actually collect these tokens,
Kerberos must be enabled.
Options are:
org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding
org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding
and org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding
</description>
</property>
<property>
<name>fs.s3a.connection.maximum</name>
<value>15</value>

View File

@ -20,6 +20,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.fs.FileStatus;
@ -519,7 +520,8 @@ private FileStatus[] verifyListStatus(int expected,
Path path,
PathFilter filter) throws IOException {
FileStatus[] result = getFileSystem().listStatus(path, filter);
assertEquals("length of listStatus(" + path + ", " + filter + " )",
assertEquals("length of listStatus(" + path + ", " + filter + " ) " +
Arrays.toString(result),
expected, result.length);
return result;
}

View File

@ -23,8 +23,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.Optional;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeoutException;
@ -645,6 +648,48 @@ public static void eval(VoidCallable closure) {
}
}
/**
* Evaluate a closure and return the result, after verifying that it is
* not null.
* @param message message to use in assertion text if the result is null
* @param eval closure to evaluate
* @param <T> type of response
* @return the evaluated result
* @throws Exception on any problem
*/
public static<T> T notNull(String message, Callable<T> eval)
throws Exception {
T t = eval.call();
Assert.assertNotNull(message, t);
return t;
}
/**
* Execute a closure as the given user.
* @param user user to invoke the closure as
* @param eval closure to evaluate
* @param <T> return type
* @return the result of calling the closure under the identity of the user.
* @throws IOException IO failure
* @throws InterruptedException interrupted operation.
*/
public static<T> T doAs(UserGroupInformation user, Callable<T> eval)
throws IOException, InterruptedException {
return user.doAs(new PrivilegedOperation<>(eval));
}
/**
* Execute a closure as the given user.
* @param user user to invoke the closure as
* @param eval closure to evaluate
* @throws IOException IO failure
* @throws InterruptedException interrupted operation.
*/
public static void doAs(UserGroupInformation user, VoidCallable eval)
throws IOException, InterruptedException {
user.doAs(new PrivilegedVoidOperation(eval));
}
/**
* Returns {@code TimeoutException} on a timeout. If
* there was a inner class passed in, includes it as the
@ -812,4 +857,50 @@ public Void call() throws Exception {
}
}
/**
* A lambda-invoker for doAs use; invokes the callable provided
* in the constructor.
* @param <T> return type.
*/
public static class PrivilegedOperation<T>
implements PrivilegedExceptionAction<T> {
private final Callable<T> callable;
/**
* Constructor.
* @param callable a non-null callable/closure.
*/
public PrivilegedOperation(final Callable<T> callable) {
this.callable = Preconditions.checkNotNull(callable);
}
@Override
public T run() throws Exception {
return callable.call();
}
}
/**
* VoidCaller variant of {@link PrivilegedOperation}: converts
* a void-returning closure to an action which {@code doAs} can call.
*/
public static class PrivilegedVoidOperation
implements PrivilegedExceptionAction<Void> {
private final Callable<Void> callable;
/**
* Constructor.
* @param callable a non-null callable/closure.
*/
public PrivilegedVoidOperation(final VoidCallable callable) {
this.callable = new VoidCaller(callable);
}
@Override
public Void run() throws Exception {
return callable.call();
}
}
}

View File

@ -42,6 +42,8 @@
import org.apache.hadoop.mapreduce.util.ConfigUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ReservationId;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -1529,7 +1531,10 @@ public static Map<String, Boolean> getArchiveSharedCacheUploadPolicies(
return getSharedCacheUploadPolicies(conf, false);
}
private synchronized void connect()
/** Only for mocking via unit tests. */
@Private
@VisibleForTesting
synchronized void connect()
throws IOException, InterruptedException, ClassNotFoundException {
if (cluster == null) {
cluster =
@ -1549,7 +1554,8 @@ boolean isConnected() {
/** Only for mocking via unit tests. */
@Private
public JobSubmitter getJobSubmitter(FileSystem fs,
@VisibleForTesting
JobSubmitter getJobSubmitter(FileSystem fs,
ClientProtocol submitClient) throws IOException {
return new JobSubmitter(fs, submitClient);
}

View File

@ -1469,6 +1469,11 @@
<artifactId>snakeyaml</artifactId>
<version>${snakeyaml.version}</version>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-library</artifactId>
<version>1.3</version>
</dependency>
<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>

View File

@ -490,6 +490,11 @@
<artifactId>hadoop-minikdc</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-library</artifactId>
<scope>test</scope>
</dependency>
<!-- Used to create SSL certs for a secure Keystore -->
<dependency>
<groupId>org.bouncycastle</groupId>

View File

@ -21,6 +21,7 @@
import java.io.Closeable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -39,6 +40,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
import org.apache.hadoop.io.IOUtils;
/**
@ -52,7 +54,8 @@
* 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>On failure, the last "relevant" AmazonClientException raised is
* rethrown; exceptions other than 'no credentials' have priority.</li>
* <li>Special handling of {@link AnonymousAWSCredentials}.</li>
* </ol>
*/
@ -78,6 +81,12 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
private final AtomicBoolean closed = new AtomicBoolean(false);
/**
* The name, which is empty by default.
* Uses in the code assume if non empty there's a trailing space.
*/
private String name = "";
/**
* Empty instance. This is not ready to be used.
*/
@ -93,6 +102,29 @@ public AWSCredentialProviderList(
this.providers.addAll(providers);
}
/**
* Create with an initial list of providers.
* @param name name for error messages, may be ""
* @param providerArgs provider list.
*/
public AWSCredentialProviderList(final String name,
final AWSCredentialsProvider... providerArgs) {
setName(name);
Collections.addAll(providers, providerArgs);
}
/**
* Set the name; adds a ": " if needed.
* @param name name to add, or "" for no name.
*/
public void setName(final String name) {
if (!name.isEmpty() && !name.endsWith(": ")) {
this.name = name + ": ";
} else {
this.name = name;
}
}
/**
* Add a new provider.
* @param p provider
@ -101,6 +133,14 @@ public void add(AWSCredentialsProvider p) {
providers.add(p);
}
/**
* Add all providers from another list to this one.
* @param other the other list.
*/
public void addAll(AWSCredentialProviderList other) {
providers.addAll(other.providers);
}
/**
* Refresh all child entries.
*/
@ -123,7 +163,7 @@ public void refresh() {
public AWSCredentials getCredentials() {
if (isClosed()) {
LOG.warn(CREDENTIALS_REQUESTED_WHEN_CLOSED);
throw new NoAuthWithAWSException(
throw new NoAuthWithAWSException(name +
CREDENTIALS_REQUESTED_WHEN_CLOSED);
}
checkNotEmpty();
@ -135,6 +175,8 @@ public AWSCredentials getCredentials() {
for (AWSCredentialsProvider provider : providers) {
try {
AWSCredentials credentials = provider.getCredentials();
Preconditions.checkNotNull(credentials,
"Null credentials returned by %s", provider);
if ((credentials.getAWSAccessKeyId() != null &&
credentials.getAWSSecretKey() != null)
|| (credentials instanceof AnonymousAWSCredentials)) {
@ -142,6 +184,18 @@ public AWSCredentials getCredentials() {
LOG.debug("Using credentials from {}", provider);
return credentials;
}
} catch (NoAwsCredentialsException e) {
// don't bother with the stack trace here as it is usually a
// minor detail.
// only update the last exception if it isn't set.
// Why so? Stops delegation token issues being lost on the fallback
// values.
if (lastException == null) {
lastException = e;
}
LOG.debug("No credentials from {}: {}",
provider, e.toString());
} catch (AmazonClientException e) {
lastException = e;
LOG.debug("No credentials provided by {}: {}",
@ -151,13 +205,17 @@ public AWSCredentials getCredentials() {
// no providers had any credentials. Rethrow the last exception
// or create a new one.
String message = "No AWS Credentials provided by "
String message = name + "No AWS Credentials provided by "
+ listProviderNames();
if (lastException != null) {
message += ": " + lastException;
}
if (lastException instanceof CredentialInitializationException) {
throw lastException;
} else {
throw new NoAuthWithAWSException(message, lastException);
}
}
/**
* Returns the underlying list of providers.
@ -175,7 +233,7 @@ List<AWSCredentialsProvider> getProviders() {
*/
public void checkNotEmpty() {
if (providers.isEmpty()) {
throw new NoAuthWithAWSException(NO_AWS_CREDENTIAL_PROVIDERS);
throw new NoAuthWithAWSException(name + NO_AWS_CREDENTIAL_PROVIDERS);
}
}
@ -198,8 +256,10 @@ public String listProviderNames() {
@Override
public String toString() {
return "AWSCredentialProviderList[" +
name +
"refcount= " + refCount.get() + ": [" +
StringUtils.join(providers, ", ") + ']';
StringUtils.join(providers, ", ") + ']'
+ (lastProvider != null ? (" last provider: " + lastProvider) : "");
}
/**
@ -265,4 +325,12 @@ public void close() {
}
}
}
/**
* Get the size of this list.
* @return the number of providers in the list.
*/
public int size() {
return providers.size();
}
}

View File

@ -51,7 +51,7 @@ private Constants() {
// s3 secret key
public static final String SECRET_KEY = "fs.s3a.secret.key";
// aws credentials provider
// aws credentials providers
public static final String AWS_CREDENTIALS_PROVIDER =
"fs.s3a.aws.credentials.provider";
@ -63,18 +63,20 @@ private Constants() {
public static final String S3A_SECURITY_CREDENTIAL_PROVIDER_PATH =
"fs.s3a.security.credential.provider.path";
// session token for when using TemporaryAWSCredentialsProvider
/**
* session token for when using TemporaryAWSCredentialsProvider: : {@value}.
*/
public static final String SESSION_TOKEN = "fs.s3a.session.token";
/**
* AWS Role to request.
* ARN of AWS Role to request: {@value}.
*/
public static final String ASSUMED_ROLE_ARN =
"fs.s3a.assumed.role.arn";
/**
* Session name for the assumed role, must be valid characters according
* to the AWS APIs.
* to the AWS APIs: {@value}.
* If not set, one is generated from the current Hadoop/Kerberos username.
*/
public static final String ASSUMED_ROLE_SESSION_NAME =
@ -86,34 +88,50 @@ private Constants() {
public static final String ASSUMED_ROLE_SESSION_DURATION =
"fs.s3a.assumed.role.session.duration";
/** Security Token Service Endpoint. If unset, uses the default endpoint. */
/**
* Security Token Service Endpoint: {@value}.
* If unset, uses the default endpoint.
*/
public static final String ASSUMED_ROLE_STS_ENDPOINT =
"fs.s3a.assumed.role.sts.endpoint";
/**
* Region for the STS endpoint; only relevant if the endpoint
* is set.
* Default endpoint for session tokens: {@value}.
* This is the central STS endpoint which, for v3 signing, can
* issue STS tokens for any region.
*/
public static final String DEFAULT_ASSUMED_ROLE_STS_ENDPOINT = "";
/**
* Region for the STS endpoint; needed if the endpoint
* is set to anything other then the central one.: {@value}.
*/
public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION =
"fs.s3a.assumed.role.sts.endpoint.region";
/**
* Default value for the STS endpoint region; needed for
* v4 signing.
* v4 signing: {@value}.
*/
public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT =
"us-west-1";
public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT = "";
/**
* Default duration of an assumed role.
* Default duration of an assumed role: {@value}.
*/
public static final String ASSUMED_ROLE_SESSION_DURATION_DEFAULT = "30m";
public static final String ASSUMED_ROLE_SESSION_DURATION_DEFAULT = "1h";
/** list of providers to authenticate for the assumed role. */
/**
* List of providers to authenticate for the assumed role: {@value}.
*/
public static final String ASSUMED_ROLE_CREDENTIALS_PROVIDER =
"fs.s3a.assumed.role.credentials.provider";
/** JSON policy containing the policy to apply to the role. */
/**
* JSON policy containing the policy to apply to the role: {@value}.
* This is not used for delegation tokens, which generate the policy
* automatically, and restrict it to the S3, KMS and S3Guard services
* needed.
*/
public static final String ASSUMED_ROLE_POLICY =
"fs.s3a.assumed.role.policy";
@ -320,7 +338,10 @@ private Constants() {
/** Prefix for S3A bucket-specific properties: {@value}. */
public static final String FS_S3A_BUCKET_PREFIX = "fs.s3a.bucket.";
public static final int S3A_DEFAULT_PORT = -1;
/**
* Default port for this is 443: HTTPS.
*/
public static final int S3A_DEFAULT_PORT = 443;
public static final String USER_AGENT_PREFIX = "fs.s3a.user.agent.prefix";

View File

@ -28,6 +28,9 @@
import com.amazonaws.services.s3.S3ClientOptions;
import org.slf4j.Logger;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@ -39,6 +42,8 @@
* This which calls the AWS SDK to configure and create an
* {@link AmazonS3Client} that communicates with the S3 service.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class DefaultS3ClientFactory extends Configured
implements S3ClientFactory {
@ -47,9 +52,13 @@ public class DefaultS3ClientFactory extends Configured
@Override
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentials) throws IOException {
final AWSCredentialsProvider credentials,
final String userAgentSuffix) throws IOException {
Configuration conf = getConf();
final ClientConfiguration awsConf = S3AUtils.createAwsConf(getConf(), bucket);
if (!StringUtils.isEmpty(userAgentSuffix)) {
awsConf.setUserAgentSuffix(userAgentSuffix);
}
return configureAmazonS3Client(
newAmazonS3Client(credentials, awsConf), conf);
}

View File

@ -476,7 +476,7 @@ public void onFailure(String text,
};
/**
* Log summary at info, full stack at debug.
* Log retries at debug.
*/
public static final Retried LOG_EVENT = new Retried() {
@Override

View File

@ -42,6 +42,16 @@ public S3A(URI theUri, Configuration conf)
@Override
public int getUriDefaultPort() {
return Constants.S3A_DEFAULT_PORT;
// return Constants.S3A_DEFAULT_PORT;
return super.getUriDefaultPort();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("S3A{");
sb.append("URI =").append(fsImpl.getUri());
sb.append("; fsImpl=").append(fsImpl);
sb.append('}');
return sb.toString();
}
}

View File

@ -25,27 +25,44 @@
/**
* This enum is to centralize the encryption methods and
* the value required in the configuration.
*
* There's two enum values for the two client encryption mechanisms the AWS
* S3 SDK supports, even though these are not currently supported in S3A.
* This is to aid supporting CSE in some form in future, fundamental
* issues about file length of encrypted data notwithstanding.
*
*/
public enum S3AEncryptionMethods {
SSE_S3("AES256"),
SSE_KMS("SSE-KMS"),
SSE_C("SSE-C"),
NONE("");
NONE("", false),
SSE_S3("AES256", true),
SSE_KMS("SSE-KMS", true),
SSE_C("SSE-C", true),
CSE_KMS("CSE-KMS", false),
CSE_CUSTOM("CSE-CUSTOM", false);
static final String UNKNOWN_ALGORITHM
= "Unknown Server Side Encryption algorithm ";
= "Unknown encryption algorithm ";
private String method;
private boolean serverSide;
S3AEncryptionMethods(String method) {
S3AEncryptionMethods(String method, final boolean serverSide) {
this.method = method;
this.serverSide = serverSide;
}
public String getMethod() {
return method;
}
/**
* Flag to indicate this is a server-side encryption option.
* @return true if this is server side.
*/
public boolean isServerSide() {
return serverSide;
}
/**
* Get the encryption mechanism from the value provided.
@ -57,16 +74,12 @@ public static S3AEncryptionMethods getMethod(String name) throws IOException {
if(StringUtils.isBlank(name)) {
return NONE;
}
switch(name) {
case "AES256":
return SSE_S3;
case "SSE-KMS":
return SSE_KMS;
case "SSE-C":
return SSE_C;
default:
throw new IOException(UNKNOWN_ALGORITHM + name);
for (S3AEncryptionMethods v : values()) {
if (v.getMethod().equals(name)) {
return v;
}
}
throw new IOException(UNKNOWN_ALGORITHM + name);
}
}

View File

@ -36,6 +36,7 @@
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.Objects;
import java.util.concurrent.ExecutorService;
@ -101,6 +102,12 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
import org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.PutTracker;
import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration;
@ -114,6 +121,7 @@
import org.apache.hadoop.fs.store.EtagChecksum;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
@ -122,8 +130,12 @@
import static org.apache.hadoop.fs.s3a.Invoker.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* The core S3A Filesystem implementation.
@ -140,7 +152,8 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class S3AFileSystem extends FileSystem implements StreamCapabilities {
public class S3AFileSystem extends FileSystem implements StreamCapabilities,
AWSPolicyProvider {
/**
* Default blocksize as used in blocksize and FS status queries.
*/
@ -183,7 +196,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
private LocalDirAllocator directoryAllocator;
private CannedAccessControlList cannedACL;
private S3AEncryptionMethods serverSideEncryptionAlgorithm;
/**
* This must never be null; until initialized it just declares that there
* is no encryption.
*/
private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
private S3AInstrumentation instrumentation;
private final S3AStorageStatistics storageStatistics =
createStorageStatistics();
@ -194,6 +212,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
private MetadataStore metadataStore;
private boolean allowAuthoritative;
/** Delegation token integration; non-empty when DT support is enabled. */
private Optional<S3ADelegationTokens> delegationTokens = Optional.empty();
/** Principal who created the FS; recorded during initialization. */
private UserGroupInformation owner;
// The maximum number of entries that can be deleted in any call to s3
private static final int MAX_ENTRIES_TO_DELETE = 1000;
private String blockOutputBuffer;
@ -234,32 +258,40 @@ private static void addDeprecatedKeys() {
*/
public void initialize(URI name, Configuration originalConf)
throws IOException {
setUri(name);
// get the host; this is guaranteed to be non-null, non-empty
bucket = name.getHost();
LOG.debug("Initializing S3AFileSystem for {}", bucket);
// clone the configuration into one with propagated bucket options
Configuration conf = propagateBucketOptions(originalConf, bucket);
// patch the Hadoop security providers
patchSecurityCredentialProviders(conf);
super.initialize(name, conf);
// look for delegation token support early.
boolean delegationTokensEnabled = hasDelegationTokenBinding(conf);
if (delegationTokensEnabled) {
LOG.debug("Using delegation tokens");
}
// set the URI, this will do any fixup of the URI to remove secrets,
// canonicalize.
setUri(name, delegationTokensEnabled);
super.initialize(uri, conf);
setConf(conf);
try {
instrumentation = new S3AInstrumentation(name);
// look for encryption data
// DT Bindings may override this
setEncryptionSecrets(new EncryptionSecrets(
getEncryptionAlgorithm(bucket, conf),
getServerSideEncryptionKey(bucket, getConf())));
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
instrumentation = new S3AInstrumentation(uri);
// Username is the current user at the time the FS was instantiated.
username = UserGroupInformation.getCurrentUser().getShortUserName();
owner = UserGroupInformation.getCurrentUser();
username = owner.getShortUserName();
workingDir = new Path("/user", username)
.makeQualified(this.uri, this.getWorkingDirectory());
Class<? extends S3ClientFactory> s3ClientFactoryClass = conf.getClass(
S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
S3ClientFactory.class);
credentials = createAWSCredentialProviderSet(name, conf);
s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
.createS3Client(name, bucket, credentials);
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
onRetry);
writeHelper = new WriteOperationHelper(this, getConf());
@ -306,13 +338,18 @@ public void initialize(URI name, Configuration originalConf)
}
useListV1 = (listVersion == 1);
// creates the AWS client, including overriding auth chain if
// the FS came with a DT
// this may do some patching of the configuration (e.g. setting
// the encryption algorithms)
bindAWSClient(name, delegationTokensEnabled);
initTransferManager();
initCannedAcls(conf);
verifyBucketExists();
serverSideEncryptionAlgorithm = getEncryptionAlgorithm(bucket, conf);
inputPolicy = S3AInputPolicy.getPolicy(
conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
LOG.debug("Input fadvise policy = {}", inputPolicy);
@ -391,6 +428,80 @@ public S3AInstrumentation getInstrumentation() {
return instrumentation;
}
/**
* Set up the client bindings.
* If delegation tokens are enabled, the FS first looks for a DT
* ahead of any other bindings;.
* If there is a DT it uses that to do the auth
* and switches to the DT authenticator automatically (and exclusively)
* @param name URI of the FS
* @param dtEnabled are delegation tokens enabled?
* @throws IOException failure.
*/
private void bindAWSClient(URI name, boolean dtEnabled) throws IOException {
Configuration conf = getConf();
credentials = null;
String uaSuffix = "";
if (dtEnabled) {
// Delegation support.
// Create and start the DT integration.
// Then look for an existing DT for this bucket, switch to authenticating
// with it if so.
LOG.debug("Using delegation tokens");
S3ADelegationTokens tokens = new S3ADelegationTokens();
this.delegationTokens = Optional.of(tokens);
tokens.bindToFileSystem(getCanonicalUri(), this);
tokens.init(conf);
tokens.start();
// switch to the DT provider and bypass all other configured
// providers.
if (tokens.isBoundToDT()) {
// A DT was retrieved.
LOG.debug("Using existing delegation token");
// and use the encryption settings from that client, whatever they were
} else {
LOG.debug("No delegation token for this instance");
}
// Get new credential chain
credentials = tokens.getCredentialProviders();
// and any encryption secrets which came from a DT
tokens.getEncryptionSecrets()
.ifPresent(this::setEncryptionSecrets);
// and update the UA field with any diagnostics provided by
// the DT binding.
uaSuffix = tokens.getUserAgentField();
} else {
// DT support is disabled, so create the normal credential chain
credentials = createAWSCredentialProviderSet(name, conf);
}
LOG.debug("Using credential provider {}", credentials);
Class<? extends S3ClientFactory> s3ClientFactoryClass = conf.getClass(
S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
S3ClientFactory.class);
s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
.createS3Client(getUri(), bucket, credentials, uaSuffix);
}
/**
* Set the encryption secrets for requests.
* @param secrets secrets
*/
protected void setEncryptionSecrets(final EncryptionSecrets secrets) {
this.encryptionSecrets = secrets;
}
/**
* Get the encryption secrets.
* This potentially sensitive information and must be treated with care.
* @return the current encryption secrets.
*/
public EncryptionSecrets getEncryptionSecrets() {
return encryptionSecrets;
}
private void initTransferManager() {
TransferManagerConfiguration transferConfiguration =
new TransferManagerConfiguration();
@ -466,18 +577,30 @@ public URI getUri() {
}
/**
* Set the URI field through {@link S3xLoginHelper}.
* Set the URI field through {@link S3xLoginHelper} and
* optionally {@link #canonicalizeUri(URI)}
* Exported for testing.
* @param uri filesystem URI.
* @param fsUri filesystem URI.
* @param canonicalize true if the URI should be canonicalized.
*/
@VisibleForTesting
protected void setUri(URI uri) {
this.uri = S3xLoginHelper.buildFSURI(uri);
protected void setUri(URI fsUri, boolean canonicalize) {
URI u = S3xLoginHelper.buildFSURI(fsUri);
this.uri = canonicalize ? u : canonicalizeUri(u);
}
/**
* Get the canonical URI.
* @return the canonical URI of this FS.
*/
public URI getCanonicalUri() {
return uri;
}
@VisibleForTesting
@Override
public int getDefaultPort() {
return Constants.S3A_DEFAULT_PORT;
return 0;
}
/**
@ -558,7 +681,7 @@ public S3AInputPolicy getInputPolicy() {
* @return the encryption algorithm.
*/
public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
return serverSideEncryptionAlgorithm;
return encryptionSecrets.getEncryptionMethod();
}
/**
@ -690,6 +813,13 @@ public void checkPath(Path path) {
S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
}
/**
* Override the base canonicalization logic and relay to
* {@link S3xLoginHelper#canonicalizeUri(URI, int)}.
* This allows for the option of changing this logic for better DT handling.
* @param rawUri raw URI.
* @return the canonical URI to use in delegation tokens and file context.
*/
@Override
protected URI canonicalizeUri(URI rawUri) {
return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
@ -719,8 +849,8 @@ public FSDataInputStream open(Path f, int bufferSize)
fileStatus),
new S3ObjectAttributes(bucket,
pathToKey(f),
serverSideEncryptionAlgorithm,
getServerSideEncryptionKey(bucket, getConf())),
getServerSideEncryptionAlgorithm(),
encryptionSecrets.getEncryptionKey()),
fileStatus.getLen(),
s3,
readAhead,
@ -1092,9 +1222,26 @@ private boolean innerRename(Path source, Path dest)
* @throws IOException IO and object access problems.
*/
@VisibleForTesting
@Retries.RetryRaw
@Retries.RetryTranslated
public ObjectMetadata getObjectMetadata(Path path) throws IOException {
return getObjectMetadata(pathToKey(path));
return once("getObjectMetadata", path.toString(),
() ->
// this always does a full HEAD to the object
getObjectMetadata(pathToKey(path)));
}
/**
* Get all the headers of the object of a path, if the object exists.
* @param path path to probe
* @return an immutable map of object headers.
* @throws IOException failure of the query
*/
@Retries.RetryTranslated
public Map<String, Object> getObjectHeaders(Path path) throws IOException {
LOG.debug("getObjectHeaders({})", path);
checkNotClosed();
incrementReadOperations();
return getObjectMetadata(path).getRawMetadata();
}
/**
@ -1244,10 +1391,7 @@ protected ObjectMetadata getObjectMetadata(String key) throws IOException {
GetObjectMetadataRequest request =
new GetObjectMetadataRequest(bucket, key);
//SSE-C requires to be filled in if enabled for object metadata
if(S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
isNotBlank(getServerSideEncryptionKey(bucket, getConf()))){
request.setSSECustomerKey(generateSSECustomerKey());
}
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
ObjectMetadata meta = invoker.retryUntranslated("GET " + key, true,
() -> {
incrementStatistic(OBJECT_METADATA_REQUESTS);
@ -2012,6 +2156,14 @@ public String getUsername() {
return username;
}
/**
* Get the owner of this FS: who created it?
* @return the owner of the FS.
*/
public UserGroupInformation getOwner() {
return owner;
}
/**
*
* Make the given path and all non-existent parents into
@ -2508,6 +2660,7 @@ public void close() throws IOException {
metadataStore = null;
instrumentation = null;
closeAutocloseables(LOG, credentials);
cleanupWithLogger(LOG, delegationTokens.orElse(null));
credentials = null;
}
}
@ -2524,13 +2677,89 @@ private void checkNotClosed() throws IOException {
}
/**
* Override getCanonicalServiceName because we don't support token in S3A.
* Get the delegation token support for this filesystem;
* not null iff delegation support is enabled.
* @return the token support, or an empty option.
*/
@VisibleForTesting
public Optional<S3ADelegationTokens> getDelegationTokens() {
return delegationTokens;
}
/**
* Return a service name iff delegation tokens are enabled and the
* token binding is issuing delegation tokens.
* @return the canonical service name or null
*/
@Override
public String getCanonicalServiceName() {
// Does not support Token
// this could all be done in map statements, but it'd be harder to
// understand and maintain.
// Essentially: no DTs, no canonical service name.
if (!delegationTokens.isPresent()) {
return null;
}
// DTs present: ask the binding if it is willing to
// serve tokens (or fail noisily).
S3ADelegationTokens dt = delegationTokens.get();
return dt.getTokenIssuingPolicy() != NoTokensAvailable
? dt.getCanonicalServiceName()
: null;
}
/**
* Get a delegation token if the FS is set up for them.
* If the user already has a token, it is returned,
* <i>even if it has expired</i>.
* @param renewer the account name that is allowed to renew the token.
* @return the delegation token or null
* @throws IOException IO failure
*/
@Override
public Token<AbstractS3ATokenIdentifier> getDelegationToken(String renewer)
throws IOException {
entryPoint(Statistic.INVOCATION_GET_DELEGATION_TOKEN);
LOG.debug("Delegation token requested");
if (delegationTokens.isPresent()) {
return delegationTokens.get().getBoundOrNewDT(encryptionSecrets);
} else {
// Delegation token support is not set up
LOG.debug("Token support is not enabled");
return null;
}
}
/**
* Build the AWS policy for restricted access to the resources needed
* by this bucket.
* The policy generated includes S3 access, S3Guard access
* if needed, and KMS operations.
* @param access access level desired.
* @return a policy for use in roles
*/
@Override
public List<RoleModel.Statement> listAWSPolicyRules(
final Set<AccessLevel> access) {
if (access.isEmpty()) {
return Collections.emptyList();
}
List<RoleModel.Statement> statements = new ArrayList<>(
allowS3Operations(bucket,
access.contains(AccessLevel.WRITE)
|| access.contains(AccessLevel.ADMIN)));
// no attempt is made to qualify KMS access; there's no
// way to predict read keys, and not worried about granting
// too much encryption access.
statements.add(STATEMENT_ALLOW_SSE_KMS_RW);
// add any metastore policies
if (metadataStore instanceof AWSPolicyProvider) {
statements.addAll(
((AWSPolicyProvider) metadataStore).listAWSPolicyRules(access));
}
return statements;
}
/**
* Copy a single object in the bucket via a COPY operation.
@ -2581,20 +2810,15 @@ private void copyFile(String srcKey, String dstKey, long size)
});
}
/**
* Set the optional parameters when initiating the request (encryption,
* headers, storage, etc).
* @param request request to patch.
*/
protected void setOptionalMultipartUploadRequestParameters(
InitiateMultipartUploadRequest req) {
switch (serverSideEncryptionAlgorithm) {
case SSE_KMS:
req.setSSEAwsKeyManagementParams(generateSSEAwsKeyParams());
break;
case SSE_C:
if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
//at the moment, only supports copy using the same key
req.setSSECustomerKey(generateSSECustomerKey());
}
break;
default:
}
InitiateMultipartUploadRequest request) {
generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
}
/**
@ -2604,14 +2828,7 @@ protected void setOptionalMultipartUploadRequestParameters(
*/
protected void setOptionalUploadPartRequestParameters(
UploadPartRequest request) {
switch (serverSideEncryptionAlgorithm) {
case SSE_C:
if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
request.setSSECustomerKey(generateSSECustomerKey());
}
break;
default:
}
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
}
/**
@ -2632,71 +2849,53 @@ InitiateMultipartUploadResult initiateMultipartUpload(
protected void setOptionalCopyObjectRequestParameters(
CopyObjectRequest copyObjectRequest) throws IOException {
switch (serverSideEncryptionAlgorithm) {
switch (getServerSideEncryptionAlgorithm()) {
case SSE_KMS:
copyObjectRequest.setSSEAwsKeyManagementParams(
generateSSEAwsKeyParams()
);
generateSSEAwsKeyParams().ifPresent(
copyObjectRequest::setSSEAwsKeyManagementParams);
break;
case SSE_C:
if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
//at the moment, only supports copy using the same key
SSECustomerKey customerKey = generateSSECustomerKey();
generateSSECustomerKey().ifPresent(customerKey -> {
copyObjectRequest.setSourceSSECustomerKey(customerKey);
copyObjectRequest.setDestinationSSECustomerKey(customerKey);
}
});
break;
default:
}
}
private void setOptionalPutRequestParameters(PutObjectRequest request) {
switch (serverSideEncryptionAlgorithm) {
case SSE_KMS:
request.setSSEAwsKeyManagementParams(generateSSEAwsKeyParams());
break;
case SSE_C:
if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
request.setSSECustomerKey(generateSSECustomerKey());
}
break;
default:
}
generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
}
private void setOptionalObjectMetadata(ObjectMetadata metadata) {
if (S3AEncryptionMethods.SSE_S3.equals(serverSideEncryptionAlgorithm)) {
metadata.setSSEAlgorithm(serverSideEncryptionAlgorithm.getMethod());
final S3AEncryptionMethods algorithm
= getServerSideEncryptionAlgorithm();
if (S3AEncryptionMethods.SSE_S3.equals(algorithm)) {
metadata.setSSEAlgorithm(algorithm.getMethod());
}
}
/**
* Create the AWS SDK structure used to configure SSE, based on the
* configuration.
* @return an instance of the class, which main contain the encryption key
* Create the AWS SDK structure used to configure SSE,
* if the encryption secrets contain the information/settings for this.
* @return an optional set of KMS Key settings
*/
@Retries.OnceExceptionsSwallowed
private SSEAwsKeyManagementParams generateSSEAwsKeyParams() {
//Use specified key, otherwise default to default master aws/s3 key by AWS
SSEAwsKeyManagementParams sseAwsKeyManagementParams =
new SSEAwsKeyManagementParams();
String encryptionKey = getServerSideEncryptionKey(bucket, getConf());
if (isNotBlank(encryptionKey)) {
sseAwsKeyManagementParams = new SSEAwsKeyManagementParams(encryptionKey);
}
return sseAwsKeyManagementParams;
private Optional<SSEAwsKeyManagementParams> generateSSEAwsKeyParams() {
return EncryptionSecretOperations.createSSEAwsKeyManagementParams(
encryptionSecrets);
}
/**
* Create the SSE-C structure for the AWS SDK.
* Create the SSE-C structure for the AWS SDK, if the encryption secrets
* contain the information/settings for this.
* This will contain a secret extracted from the bucket/configuration.
* @return the customer key.
* @return an optional customer key.
*/
@Retries.OnceExceptionsSwallowed
private SSECustomerKey generateSSECustomerKey() {
SSECustomerKey customerKey = new SSECustomerKey(
getServerSideEncryptionKey(bucket, getConf()));
return customerKey;
private Optional<SSECustomerKey> generateSSECustomerKey() {
return EncryptionSecretOperations.createSSECustomerKey(
encryptionSecrets);
}
/**
@ -2902,9 +3101,9 @@ public String toString() {
sb.append(", blockSize=").append(getDefaultBlockSize());
}
sb.append(", multiPartThreshold=").append(multiPartThreshold);
if (serverSideEncryptionAlgorithm != null) {
if (getServerSideEncryptionAlgorithm() != null) {
sb.append(", serverSideEncryptionAlgorithm='")
.append(serverSideEncryptionAlgorithm)
.append(getServerSideEncryptionAlgorithm())
.append('\'');
}
if (blockFactory != null) {
@ -2919,6 +3118,8 @@ public String toString() {
sb.append(", boundedExecutor=").append(boundedThreadPool);
sb.append(", unboundedExecutor=").append(unboundedThreadPool);
sb.append(", credentials=").append(credentials);
sb.append(", delegation tokens=")
.append(delegationTokens.map(Objects::toString).orElse("disabled"));
sb.append(", statistics {")
.append(statistics)
.append("}");
@ -3056,13 +3257,9 @@ public EtagChecksum getFileChecksum(Path f, final long length)
ETAG_CHECKSUM_ENABLED_DEFAULT)) {
Path path = qualify(f);
LOG.debug("getFileChecksum({})", path);
return once("getFileChecksum", path.toString(),
() -> {
// this always does a full HEAD to the object
ObjectMetadata headers = getObjectMetadata(path);
String eTag = headers.getETag();
return eTag != null ? new EtagChecksum(eTag) : null;
});
} else {
// disabled
return null;

View File

@ -139,6 +139,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
INVOCATION_CREATE_NON_RECURSIVE,
INVOCATION_DELETE,
INVOCATION_EXISTS,
INVOCATION_GET_DELEGATION_TOKEN,
INVOCATION_GET_FILE_CHECKSUM,
INVOCATION_GET_FILE_STATUS,
INVOCATION_GLOB_STATUS,
@ -181,7 +182,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
S3GUARD_METADATASTORE_INITIALIZATION,
S3GUARD_METADATASTORE_RETRY,
S3GUARD_METADATASTORE_THROTTLED,
STORE_IO_THROTTLED
STORE_IO_THROTTLED,
DELEGATION_TOKENS_ISSUED
};
private static final Statistic[] GAUGES_TO_CREATE = {
@ -1100,6 +1102,30 @@ public void jobCompleted(boolean success) {
}
}
/**
* Create a delegation token statistics instance.
* @return an instance of delegation token statistics
*/
public DelegationTokenStatistics newDelegationTokenStatistics() {
return new DelegationTokenStatistics();
}
/**
* Instrumentation exported to S3A Delegation Token support.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class DelegationTokenStatistics {
private DelegationTokenStatistics() {
}
/** A token has been issued. */
public void tokenIssued() {
incrementCounter(DELEGATION_TOKENS_ISSUED, 1);
}
}
/**
* Copy all the metrics to a map of (name, long-value).
* @return a map of the metrics

View File

@ -26,7 +26,6 @@
import com.amazonaws.SdkBaseException;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.amazonaws.retry.RetryUtils;
import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
import com.amazonaws.services.dynamodbv2.model.LimitExceededException;
@ -36,6 +35,7 @@
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.StringUtils;
@ -47,6 +47,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.net.ConnectTimeoutException;
@ -71,11 +72,15 @@
import java.net.URI;
import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import static org.apache.hadoop.fs.s3a.Constants.*;
@ -126,6 +131,13 @@ public final class S3AUtils {
private static final String BUCKET_PATTERN = FS_S3A_BUCKET_PREFIX + "%s.%s";
/**
* Error message when the AWS provider list built up contains a forbidden
* entry.
*/
@VisibleForTesting
public static final String E_FORBIDDEN_AWS_PROVIDER
= "AWS provider class cannot be used";
private S3AUtils() {
}
@ -168,7 +180,7 @@ public static IOException translateException(@Nullable String operation,
SdkBaseException exception) {
String message = String.format("%s%s: %s",
operation,
path != null ? (" on " + path) : "",
StringUtils.isNotEmpty(path)? (" on " + path) : "",
exception);
if (!(exception instanceof AmazonServiceException)) {
Exception innerCause = containsInterruptedException(exception);
@ -580,36 +592,40 @@ public static long dateToLong(final Date date) {
return date.getTime();
}
/**
* The standard AWS provider list for AWS connections.
*/
public static final List<Class<?>>
STANDARD_AWS_PROVIDERS = Collections.unmodifiableList(
Arrays.asList(
TemporaryAWSCredentialsProvider.class,
SimpleAWSCredentialsProvider.class,
EnvironmentVariableCredentialsProvider.class,
IAMInstanceCredentialsProvider.class));
/**
* Create the AWS credentials from the providers, the URI and
* the key {@link Constants#AWS_CREDENTIALS_PROVIDER} in the configuration.
* @param binding Binding URI, may contain user:pass login details;
* may be null
* @param binding Binding URI -may be null
* @param conf filesystem configuration
* @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) throws IOException {
AWSCredentialProviderList credentials = new AWSCredentialProviderList();
Class<?>[] awsClasses = loadAWSProviderClasses(conf,
AWS_CREDENTIALS_PROVIDER);
if (awsClasses.length == 0) {
credentials.add(new SimpleAWSCredentialsProvider(binding, conf));
credentials.add(new EnvironmentVariableCredentialsProvider());
credentials.add(InstanceProfileCredentialsProvider.getInstance());
} else {
for (Class<?> aClass : awsClasses) {
credentials.add(createAWSCredentialProvider(conf,
aClass,
binding));
}
}
@Nullable URI binding,
Configuration conf) throws IOException {
// this will reject any user:secret entries in the URI
S3xLoginHelper.rejectSecretsInURIs(binding);
AWSCredentialProviderList credentials =
buildAWSProviderList(binding,
conf,
AWS_CREDENTIALS_PROVIDER,
STANDARD_AWS_PROVIDERS,
new HashSet<>());
// make sure the logging message strips out any auth details
LOG.debug("For URI {}, using credentials {}",
S3xLoginHelper.toString(binding), credentials);
binding, credentials);
return credentials;
}
@ -621,17 +637,60 @@ public static AWSCredentialProviderList createAWSCredentialProviderSet(
* @return the list of classes, possibly empty
* @throws IOException on a failure to load the list.
*/
public static Class<?>[] loadAWSProviderClasses(Configuration conf,
public static List<Class<?>> loadAWSProviderClasses(Configuration conf,
String key,
Class<?>... defaultValue) throws IOException {
try {
return conf.getClasses(key, defaultValue);
return Arrays.asList(conf.getClasses(key, defaultValue));
} catch (RuntimeException e) {
Throwable c = e.getCause() != null ? e.getCause() : e;
throw new IOException("From option " + key + ' ' + c, c);
}
}
/**
* Load list of AWS credential provider/credential provider factory classes;
* support a forbidden list to prevent loops, mandate full secrets, etc.
* @param binding Binding URI -may be null
* @param conf configuration
* @param key key
* @param forbidden a possibly empty set of forbidden classes.
* @param defaultValues list of default providers.
* @return the list of classes, possibly empty
* @throws IOException on a failure to load the list.
*/
public static AWSCredentialProviderList buildAWSProviderList(
@Nullable final URI binding,
final Configuration conf,
final String key,
final List<Class<?>> defaultValues,
final Set<Class<?>> forbidden) throws IOException {
// build up the base provider
List<Class<?>> awsClasses = loadAWSProviderClasses(conf,
key,
defaultValues.toArray(new Class[defaultValues.size()]));
// and if the list is empty, switch back to the defaults.
// this is to address the issue that configuration.getClasses()
// doesn't return the default if the config value is just whitespace.
if (awsClasses.isEmpty()) {
awsClasses = defaultValues;
}
// iterate through, checking for blacklists and then instantiating
// each provider
AWSCredentialProviderList providers = new AWSCredentialProviderList();
for (Class<?> aClass : awsClasses) {
if (forbidden.contains(aClass)) {
throw new IOException(E_FORBIDDEN_AWS_PROVIDER
+ " in option " + key + ": " + aClass);
}
providers.add(createAWSCredentialProvider(conf,
aClass, binding));
}
return providers;
}
/**
* Create an AWS credential provider from its class by using reflection. The
* class must implement one of the following means of construction, which are
@ -640,6 +699,8 @@ public static Class<?>[] loadAWSProviderClasses(Configuration conf,
* <ol>
* <li>a public constructor accepting java.net.URI and
* org.apache.hadoop.conf.Configuration</li>
* <li>a public constructor accepting
* org.apache.hadoop.conf.Configuration</li>
* <li>a public static method named getInstance that accepts no
* arguments and returns an instance of
* com.amazonaws.auth.AWSCredentialsProvider, or</li>
@ -652,11 +713,11 @@ public static Class<?>[] loadAWSProviderClasses(Configuration conf,
* @return the instantiated class
* @throws IOException on any instantiation failure.
*/
public static AWSCredentialsProvider createAWSCredentialProvider(
private static AWSCredentialsProvider createAWSCredentialProvider(
Configuration conf,
Class<?> credClass,
URI uri) throws IOException {
AWSCredentialsProvider credentials;
@Nullable URI uri) throws IOException {
AWSCredentialsProvider credentials = null;
String className = credClass.getName();
if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
@ -699,9 +760,9 @@ public static AWSCredentialsProvider createAWSCredentialProvider(
// no supported constructor or factory method found
throw new IOException(String.format("%s " + CONSTRUCTOR_EXCEPTION
+ ". A class specified in %s must provide a public constructor "
+ "accepting Configuration, or a public factory method named "
+ "getInstance that accepts no arguments, or a public default "
+ "constructor.", className, AWS_CREDENTIALS_PROVIDER));
+ "of a supported signature, or a public factory method named "
+ "getInstance that accepts no arguments.",
className, AWS_CREDENTIALS_PROVIDER));
} catch (InvocationTargetException e) {
Throwable targetException = e.getTargetException();
if (targetException == null) {
@ -726,6 +787,24 @@ public static AWSCredentialsProvider createAWSCredentialProvider(
}
}
/**
* Set a key if the value is non-empty.
* @param config config to patch
* @param key key to set
* @param val value to probe and set
* @param origin origin
* @return true if the property was set
*/
public static boolean setIfDefined(Configuration config, String key,
String val, String origin) {
if (StringUtils.isNotEmpty(val)) {
config.set(key, val, origin);
return true;
} else {
return false;
}
}
/**
* Return the access key and secret for S3 API use.
* or indicated in the UserInfo of the name URI param.
@ -1399,7 +1478,7 @@ static void patchSecurityCredentialProviders(Configuration conf) {
* @return the encryption key or ""
* @throws IllegalArgumentException bad arguments.
*/
static String getServerSideEncryptionKey(String bucket,
public static String getServerSideEncryptionKey(String bucket,
Configuration conf) {
try {
return lookupPassword(bucket, conf, SERVER_SIDE_ENCRYPTION_KEY);
@ -1420,7 +1499,7 @@ static String getServerSideEncryptionKey(String bucket,
* one is set.
* @throws IOException on any validation problem.
*/
static S3AEncryptionMethods getEncryptionAlgorithm(String bucket,
public static S3AEncryptionMethods getEncryptionAlgorithm(String bucket,
Configuration conf) throws IOException {
S3AEncryptionMethods sse = S3AEncryptionMethods.getMethod(
lookupPassword(bucket, conf,
@ -1430,6 +1509,7 @@ static S3AEncryptionMethods getEncryptionAlgorithm(String bucket,
String diagnostics = passwordDiagnostics(sseKey, "key");
switch (sse) {
case SSE_C:
LOG.debug("Using SSE-C with {}", diagnostics);
if (sseKeyLen == 0) {
throw new IOException(SSE_C_NO_KEY_ERROR);
}
@ -1452,7 +1532,6 @@ static S3AEncryptionMethods getEncryptionAlgorithm(String bucket,
LOG.debug("Data is unencrypted");
break;
}
LOG.debug("Using SSE-C with {}", diagnostics);
return sse;
}

View File

@ -40,11 +40,13 @@ public interface S3ClientFactory {
* @param name raw input S3A file system URI
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @param credentialSet credentials to use
* @param userAgentSuffix optional suffix for the UA field.
* @return S3 client
* @throws IOException IO problem
*/
AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentialSet) throws IOException;
String bucket,
AWSCredentialsProvider credentialSet,
String userAgentSuffix) throws IOException;
}

View File

@ -21,19 +21,18 @@
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
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;
import static org.apache.hadoop.fs.s3a.S3AUtils.getAWSAccessKeys;
/**
@ -49,13 +48,29 @@ public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
public static final String NAME
= "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider";
private String accessKey;
private String secretKey;
private final String accessKey;
private final String secretKey;
public SimpleAWSCredentialsProvider(URI uri, Configuration conf)
/**
* Build the credentials from a filesystem URI and configuration.
* @param uri FS URI
* @param conf configuration containing secrets/references to.
* @throws IOException failure
*/
public SimpleAWSCredentialsProvider(final URI uri, final Configuration conf)
throws IOException {
this(getAWSAccessKeys(uri, conf));
}
S3xLoginHelper.Login login = getAWSAccessKeys(uri, conf);
/**
* Instantiate from a login tuple.
* For testing, hence package-scoped.
* @param login login secrets
* @throws IOException failure
*/
@VisibleForTesting
SimpleAWSCredentialsProvider(final S3xLoginHelper.Login login)
throws IOException {
this.accessKey = login.getUser();
this.secretKey = login.getPassword();
}
@ -65,8 +80,8 @@ public AWSCredentials getCredentials() {
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
return new BasicAWSCredentials(accessKey, secretKey);
}
throw new CredentialInitializationException(
"Access key or secret key is unset");
throw new NoAwsCredentialsException("SimpleAWSCredentialsProvider",
"No AWS credentials in the Hadoop configuration");
}
@Override

View File

@ -57,6 +57,8 @@ public enum Statistic {
"Calls of delete()"),
INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
"Calls of exists()"),
INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN,
"Calls of getDelegationToken()"),
INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
"Calls of getFileChecksum()"),
INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
@ -209,7 +211,10 @@ public enum Statistic {
"s3guard_metadatastore_throttle_rate",
"S3Guard metadata store throttle rate"),
STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried");
STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"),
DELEGATION_TOKENS_ISSUED("delegation_tokens_issued",
"Number of delegation tokens issued");
private static final Map<String, Statistic> SYMBOL_MAP =
new HashMap<>(Statistic.values().length);

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -18,21 +18,21 @@
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.lang3.StringUtils;
import javax.annotation.Nullable;
import java.io.IOException;
import com.amazonaws.auth.AWSCredentials;
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.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.lookupPassword;
import org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
/**
* Support session credentials for authenticating with AWS.
@ -40,50 +40,65 @@
* 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.
*
* This credential provider must not fail in creation because that will
* break a chain of credential providers.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class TemporaryAWSCredentialsProvider implements AWSCredentialsProvider {
public class TemporaryAWSCredentialsProvider extends
AbstractSessionCredentialsProvider {
public static final String NAME
= "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
private String accessKey;
private String secretKey;
private String sessionToken;
public TemporaryAWSCredentialsProvider(Configuration conf)
public static final String COMPONENT
= "Session credentials in Hadoop configuration";
/**
* Construct from just a configuration.
* @param conf configuration.
*/
public TemporaryAWSCredentialsProvider(final Configuration conf)
throws IOException {
this(null, conf);
}
public TemporaryAWSCredentialsProvider(URI uri, Configuration conf)
/**
* Constructor: the URI will be null if the provider is inited unbonded
* to a filesystem.
* @param uri binding to a filesystem URI.
* @param conf configuration.
*/
public TemporaryAWSCredentialsProvider(
@Nullable final URI uri,
final Configuration conf)
throws IOException {
// determine the bucket
String bucket = uri != null ? uri.getHost(): "";
Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
conf, S3AFileSystem.class);
this.accessKey = lookupPassword(bucket, c, ACCESS_KEY);
this.secretKey = lookupPassword(bucket, c, SECRET_KEY);
this.sessionToken = lookupPassword(bucket, c, SESSION_TOKEN);
super(uri, conf);
}
/**
* The credentials here must include a session token, else this operation
* will raise an exception.
* @param config the configuration
* @return temporary credentials.
* @throws IOException on any failure to load the credentials.
* @throws NoAuthWithAWSException validation failure
* @throws NoAwsCredentialsException the credentials are actually empty.
*/
@Override
public AWSCredentials getCredentials() {
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
&& !StringUtils.isEmpty(sessionToken)) {
return new BasicSessionCredentials(accessKey, secretKey, sessionToken);
protected AWSCredentials createCredentials(Configuration config)
throws IOException {
MarshalledCredentials creds = MarshalledCredentialBinding.fromFileSystem(
getUri(), config);
MarshalledCredentials.CredentialTypeRequired sessionOnly
= MarshalledCredentials.CredentialTypeRequired.SessionOnly;
// treat only having non-session creds as empty.
if (!creds.isValid(sessionOnly)) {
throw new NoAwsCredentialsException(COMPONENT);
}
throw new CredentialInitializationException(
"Access key, secret key or session token is unset");
}
@Override
public void refresh() {}
@Override
public String toString() {
return getClass().getSimpleName();
return MarshalledCredentialBinding.toAWSCredentials(creds,
sessionOnly, COMPONENT);
}
}

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.auth;
import javax.annotation.Nullable;
import java.net.URI;
import com.amazonaws.auth.AWSCredentialsProvider;
import org.apache.hadoop.conf.Configuration;
/**
* Base class for AWS credential providers which
* take a URI and config in their constructor.
*/
public abstract class AbstractAWSCredentialProvider
implements AWSCredentialsProvider {
private final URI binding;
private final Configuration conf;
/**
* Construct from URI + configuration.
* @param uri URI: may be null.
* @param conf configuration.
*/
protected AbstractAWSCredentialProvider(
@Nullable final URI uri,
final Configuration conf) {
this.conf = conf;
this.binding = uri;
}
public Configuration getConf() {
return conf;
}
/**
* Get the binding URI: may be null.
* @return the URI this instance was constructed with,
* if any.
*/
public URI getUri() {
return binding;
}
/**
* Refresh is a no-op by default.
*/
@Override
public void refresh() {
}
}

View File

@ -0,0 +1,170 @@
/*
* 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.auth;
import javax.annotation.Nullable;
import java.net.URI;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import com.amazonaws.SdkBaseException;
import com.amazonaws.auth.AWSCredentials;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.Retries;
/**
* Base class for session credential support.
*/
@InterfaceAudience.Private
public abstract class AbstractSessionCredentialsProvider
extends AbstractAWSCredentialProvider {
/** Credentials, created in {@link #init()}. */
private AWSCredentials awsCredentials;
/** Atomic flag for on-demand initialization. */
private final AtomicBoolean initialized = new AtomicBoolean(false);
/**
* The (possibly translated) initialization exception.
* Used for testing.
*/
private IOException initializationException;
/**
* Constructor.
* @param uri possibly null filesystem URI.
* @param conf configuration.
*/
public AbstractSessionCredentialsProvider(
@Nullable final URI uri,
final Configuration conf) {
super(uri, conf);
}
/**
* Initialize the credentials by calling
* {@link #createCredentials(Configuration)} with the current config.
*/
@Retries.OnceTranslated
protected void init() throws IOException {
// stop re-entrant attempts
if (initialized.getAndSet(true)) {
return;
}
try {
awsCredentials = Invoker.once("create credentials", "",
() -> createCredentials(getConf()));
} catch (IOException e) {
initializationException = e;
throw e;
}
}
/**
* Has an attempt to initialize the credentials been attempted?
* @return true if {@code init()} was called.
*/
public boolean isInitialized() {
return initialized.get();
}
/**
* Implementation point: whatever the subclass must do to load credentials.
* This is called from {@link #init()} and then the credentials are cached,
* along with any exception.
* @param config the configuration
* @return the credentials
* @throws IOException on any failure.
*/
protected abstract AWSCredentials createCredentials(Configuration config)
throws IOException;
/**
* Get the credentials.
* Any exception raised in
* {@link #createCredentials(Configuration)}
* is thrown here before any attempt to return the credentials
* is made.
* @return credentials, if set.
* @throws SdkBaseException if one was raised during init
* @throws CredentialInitializationException on other failures.
*/
public AWSCredentials getCredentials() throws SdkBaseException {
// do an on-demand init then raise an AWS SDK exception if
// there was a failure.
try {
if (!isInitialized()) {
init();
}
} catch (IOException e) {
if (e.getCause() instanceof SdkBaseException) {
throw (SdkBaseException) e.getCause();
} else {
throw new CredentialInitializationException(e.getMessage(), e);
}
}
if (awsCredentials == null) {
throw new CredentialInitializationException(
"Provider " + this + " has no credentials");
}
return awsCredentials;
}
public final boolean hasCredentials() {
return awsCredentials == null;
}
@Override
public String toString() {
return getClass().getSimpleName();
}
/**
* Get any IOE raised during initialization.
* Null if {@link #init()} hasn't been called, or it actually worked.
* @return an exception or null.
*/
@VisibleForTesting
public IOException getInitializationException() {
return initializationException;
}
/**
* A special set of null credentials which are not the anonymous class.
* This will be interpreted as "this provider has no credentials to offer",
* rather than an explicit error or anonymous access.
*/
protected static final class NoCredentials implements AWSCredentials {
@Override
public String getAWSAccessKeyId() {
return null;
}
@Override
public String getAWSSecretKey() {
return null;
}
}
}

View File

@ -18,19 +18,22 @@
package org.apache.hadoop.fs.s3a.auth;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
import java.util.Locale;
import java.util.concurrent.TimeUnit;
import com.amazonaws.AmazonClientException;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Sets;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -39,6 +42,8 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
@ -46,8 +51,7 @@
import org.apache.hadoop.security.UserGroupInformation;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider;
import static org.apache.hadoop.fs.s3a.S3AUtils.loadAWSProviderClasses;
import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
/**
* Support IAM Assumed roles by instantiating an instance of
@ -67,10 +71,6 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
public static final String NAME
= "org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider";
static final String E_FORBIDDEN_PROVIDER =
"AssumedRoleCredentialProvider cannot be in "
+ ASSUMED_ROLE_CREDENTIALS_PROVIDER;
public static final String E_NO_ROLE = "Unset property "
+ ASSUMED_ROLE_ARN;
@ -90,13 +90,13 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
* Instantiate.
* This calls {@link #getCredentials()} to fail fast on the inner
* role credential retrieval.
* @param fsUri URI of the filesystem.
* @param fsUri possibly null URI of the filesystem.
* @param conf configuration
* @throws IOException on IO problems and some parameter checking
* @throws IllegalArgumentException invalid parameters
* @throws AWSSecurityTokenServiceException problems getting credentials
*/
public AssumedRoleCredentialProvider(URI fsUri, Configuration conf)
public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf)
throws IOException {
arn = conf.getTrimmed(ASSUMED_ROLE_ARN, "");
@ -105,16 +105,12 @@ public AssumedRoleCredentialProvider(URI fsUri, Configuration conf)
}
// build up the base provider
Class<?>[] awsClasses = loadAWSProviderClasses(conf,
credentialsToSTS = buildAWSProviderList(fsUri, conf,
ASSUMED_ROLE_CREDENTIALS_PROVIDER,
SimpleAWSCredentialsProvider.class);
credentialsToSTS = new AWSCredentialProviderList();
for (Class<?> aClass : awsClasses) {
if (this.getClass().equals(aClass)) {
throw new IOException(E_FORBIDDEN_PROVIDER);
}
credentialsToSTS.add(createAWSCredentialProvider(conf, aClass, fsUri));
}
Arrays.asList(
SimpleAWSCredentialsProvider.class,
EnvironmentVariableCredentialsProvider.class),
Sets.newHashSet(this.getClass()));
LOG.debug("Credentials to obtain role credentials: {}", credentialsToSTS);
// then the STS binding
@ -132,13 +128,13 @@ public AssumedRoleCredentialProvider(URI fsUri, Configuration conf)
LOG.debug("Scope down policy {}", policy);
builder.withScopeDownPolicy(policy);
}
String endpoint = conf.get(ASSUMED_ROLE_STS_ENDPOINT, "");
String region = conf.get(ASSUMED_ROLE_STS_ENDPOINT_REGION,
String endpoint = conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT, "");
String region = conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT_REGION,
ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT);
AWSSecurityTokenServiceClientBuilder stsbuilder =
STSClientFactory.builder(
conf,
fsUri.getHost(),
fsUri != null ? fsUri.getHost() : "",
credentialsToSTS,
endpoint,
region);
@ -164,6 +160,7 @@ public AssumedRoleCredentialProvider(URI fsUri, Configuration conf)
* @throws AWSSecurityTokenServiceException if none could be obtained.
*/
@Override
@Retries.RetryRaw
public AWSCredentials getCredentials() {
try {
return invoker.retryUntranslated("getCredentials",
@ -174,7 +171,7 @@ public AWSCredentials getCredentials() {
// its hard to see how this could be raised, but for
// completeness, it is wrapped as an Amazon Client Exception
// and rethrown.
throw new AmazonClientException(
throw new CredentialInitializationException(
"getCredentials failed: " + e,
e);
} catch (AWSSecurityTokenServiceException e) {

View File

@ -0,0 +1,75 @@
/*
* 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.auth;
import java.io.Closeable;
import java.io.IOException;
import com.amazonaws.AmazonClientException;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* This is going to be an IAM credential provider which performs
* async refresh for lower-latency on IO calls.
* Initially it does not do this, simply shares the single IAM instance
* across all instances. This makes it less expensive to declare.
*
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class IAMInstanceCredentialsProvider
implements AWSCredentialsProvider, Closeable {
private static final InstanceProfileCredentialsProvider INSTANCE =
InstanceProfileCredentialsProvider.getInstance();
public IAMInstanceCredentialsProvider() {
}
/**
* Ask for the credentials.
* as it invariably means "you aren't running on EC2"
* @return the credentials
*/
@Override
public AWSCredentials getCredentials() {
try {
return INSTANCE.getCredentials();
} catch (AmazonClientException e) {
throw new NoAwsCredentialsException("IAMInstanceCredentialsProvider",
e.getMessage(),
e);
}
}
@Override
public void refresh() {
INSTANCE.refresh();
}
@Override
public void close() throws IOException {
// until async, no-op.
}
}

View File

@ -0,0 +1,205 @@
/*
* 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.auth;
import java.io.IOException;
import java.net.URI;
import java.util.Date;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSSessionCredentials;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.auth.BasicSessionCredentials;
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
import com.amazonaws.services.securitytoken.model.Credentials;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.security.ProviderUtils;
import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
import static org.apache.hadoop.fs.s3a.Constants.SESSION_TOKEN;
import static org.apache.hadoop.fs.s3a.S3AUtils.lookupPassword;
/**
* Class to bridge from the serializable/marshallabled
* {@link MarshalledCredentialBinding} class to/from AWS classes.
* This is to keep that class isolated and not dependent on aws-sdk JARs
* to load.
*/
public final class MarshalledCredentialBinding {
private MarshalledCredentialBinding() {
}
/**
* Error text on empty credentials: {@value}.
*/
@VisibleForTesting
public static final String NO_AWS_CREDENTIALS = "No AWS credentials";
/**
* Create a set of marshalled credentials from a set of credentials
* issued by an STS call.
* @param credentials AWS-provided session credentials
*/
public static MarshalledCredentials fromSTSCredentials(
final Credentials credentials) {
MarshalledCredentials marshalled = new MarshalledCredentials(
credentials.getAccessKeyId(),
credentials.getSecretAccessKey(),
credentials.getSessionToken());
Date date = credentials.getExpiration();
marshalled.setExpiration(date != null ? date.getTime() : 0);
return marshalled;
}
/**
* Create from a set of AWS credentials.
* @param credentials source credential.
* @return a set of marshalled credentials.
*/
public static MarshalledCredentials fromAWSCredentials(
final AWSSessionCredentials credentials) {
return new MarshalledCredentials(
credentials.getAWSAccessKeyId(),
credentials.getAWSSecretKey(),
credentials.getSessionToken());
}
/**
* Build a set of credentials from the environment.
* @param env environment.
* @return a possibly incomplete/invalid set of credentials.
*/
public static MarshalledCredentials fromEnvironment(
final Map<String, String> env) {
return new MarshalledCredentials(
nullToEmptyString(env.get("AWS_ACCESS_KEY")),
nullToEmptyString(env.get("AWS_SECRET_KEY")),
nullToEmptyString(env.get("AWS_SESSION_TOKEN")));
}
/**
* Take a string where a null value is remapped to an empty string.
* @param src source string.
* @return the value of the string or ""
*/
private static String nullToEmptyString(final String src) {
return src == null ? "" : src;
}
/**
* Loads the credentials from the owning S3A FS, including
* from Hadoop credential providers.
* There is no validation.
* @param conf configuration to load from
* @return the component
* @throws IOException on any load failure
*/
public static MarshalledCredentials fromFileSystem(
final URI uri,
final Configuration conf) throws IOException {
// determine the bucket
final String bucket = uri != null ? uri.getHost() : "";
final Configuration leanConf =
ProviderUtils.excludeIncompatibleCredentialProviders(
conf, S3AFileSystem.class);
return new MarshalledCredentials(
lookupPassword(bucket, leanConf, ACCESS_KEY),
lookupPassword(bucket, leanConf, SECRET_KEY),
lookupPassword(bucket, leanConf, SESSION_TOKEN));
}
/**
* Create an AWS credential set from a set of marshalled credentials.
*
* This code would seem to fit into (@link MarshalledCredentials}, and
* while it would from a code-hygiene perspective, to keep all AWS
* SDK references out of that class, the logic is implemented here instead,
* @param marshalled marshalled credentials
* @param typeRequired type of credentials required
* @param component component name for exception messages.
* @return a new set of credentials
* @throws NoAuthWithAWSException validation failure
* @throws NoAwsCredentialsException the credentials are actually empty.
*/
public static AWSCredentials toAWSCredentials(
final MarshalledCredentials marshalled,
final MarshalledCredentials.CredentialTypeRequired typeRequired,
final String component)
throws NoAuthWithAWSException, NoAwsCredentialsException {
if (marshalled.isEmpty()) {
throw new NoAwsCredentialsException(component, NO_AWS_CREDENTIALS);
}
if (!marshalled.isValid(typeRequired)) {
throw new NoAuthWithAWSException(component + ":" +
marshalled.buildInvalidCredentialsError(typeRequired));
}
final String accessKey = marshalled.getAccessKey();
final String secretKey = marshalled.getSecretKey();
if (marshalled.hasSessionToken()) {
// a session token was supplied, so return session credentials
return new BasicSessionCredentials(accessKey, secretKey,
marshalled.getSessionToken());
} else {
// these are full credentials
return new BasicAWSCredentials(accessKey, secretKey);
}
}
/**
* Request a set of credentials from an STS endpoint.
* @param parentCredentials the parent credentials needed to talk to STS
* @param stsEndpoint an endpoint, use "" for none
* @param stsRegion region; use if the endpoint isn't the AWS default.
* @param duration duration of the credentials in seconds. Minimum value: 900.
* @param invoker invoker to use for retrying the call.
* @return the credentials
* @throws IOException on a failure of the request
*/
@Retries.RetryTranslated
public static MarshalledCredentials requestSessionCredentials(
final AWSCredentialsProvider parentCredentials,
final ClientConfiguration awsConf,
final String stsEndpoint,
final String stsRegion,
final int duration,
final Invoker invoker) throws IOException {
final AWSSecurityTokenService tokenService =
STSClientFactory.builder(parentCredentials,
awsConf,
stsEndpoint.isEmpty() ? null : stsEndpoint,
stsRegion)
.build();
return fromSTSCredentials(
STSClientFactory.createClientConnection(tokenService, invoker)
.requestSessionCredentials(duration, TimeUnit.SECONDS));
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.auth;
import java.io.IOException;
import java.net.URI;
import com.amazonaws.auth.AWSCredentials;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.toAWSCredentials;
/**
* AWS credential provider driven from marshalled session/full credentials
* (full, simple session or role).
* This is <i>not</i> intended for explicit use in job/app configurations,
* instead it is returned by Delegation Token Bindings, as needed.
* The constructor implicitly prevents explicit use.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class MarshalledCredentialProvider extends
AbstractSessionCredentialsProvider {
/** Name: {@value}. */
public static final String NAME
= "org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider";
private final MarshalledCredentials credentials;
private final MarshalledCredentials.CredentialTypeRequired typeRequired;
private final String component;
/**
* Constructor.
*
* @param component component name for exception messages.
* @param uri filesystem URI: must not be null.
* @param conf configuration.
* @param credentials marshalled credentials.
* @param typeRequired credential type required.
* @throws CredentialInitializationException validation failure
* @throws IOException failure
*/
public MarshalledCredentialProvider(
final String component,
final URI uri,
final Configuration conf,
final MarshalledCredentials credentials,
final MarshalledCredentials.CredentialTypeRequired typeRequired)
throws IOException {
super(checkNotNull(uri, "No filesystem URI"), conf);
this.component = component;
this.typeRequired = typeRequired;
this.credentials = checkNotNull(credentials);
}
/**
* Perform the binding, looking up the DT and parsing it.
* @return true if there were some credentials
* @throws CredentialInitializationException validation failure
* @throws IOException on a failure
*/
@Override
protected AWSCredentials createCredentials(final Configuration config)
throws IOException {
return toAWSCredentials(credentials, typeRequired, component);
}
}

View File

@ -0,0 +1,409 @@
/*
* 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.auth;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.Serializable;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.time.format.DateTimeFormatter;
import java.util.Date;
import java.util.Objects;
import java.util.Optional;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenIOException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import static java.util.Objects.requireNonNull;
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
import static org.apache.hadoop.fs.s3a.Constants.SESSION_TOKEN;
/**
* Stores the credentials for a session or for a full login.
* This structure is {@link Writable}, so can be marshalled inside a
* delegation token.
*
* The class is designed so that keys inside are kept non-null; to be
* unset just set them to the empty string. This is to simplify marshalling.
*
* <i>Important: Add no references to any AWS SDK class, to
* ensure it can be safely deserialized whenever the relevant token
* identifier of a token type declared in this JAR is examined.</i>
*/
@InterfaceAudience.Private
public final class MarshalledCredentials implements Writable, Serializable {
/**
* Error text on invalid non-empty credentials: {@value}.
*/
@VisibleForTesting
public static final String INVALID_CREDENTIALS
= "Invalid AWS credentials";
/**
* How long can any of the secrets be: {@value}.
* This is much longer than the current tokens, but leaves space for
* future enhancements.
*/
private static final int MAX_SECRET_LENGTH = 8192;
private static final long serialVersionUID = 8444610385533920692L;
/**
* Access key of IAM account.
*/
private String accessKey = "";
/**
* Secret key of IAM account.
*/
private String secretKey = "";
/**
* Optional session token.
* If non-empty: the credentials can be converted into
* session credentials.
*/
private String sessionToken = "";
/**
* ARN of a role. Purely for diagnostics.
*/
private String roleARN = "";
/**
* Expiry time milliseconds in UTC; the {@code Java.Util.Date} value.
* 0 means "does not expire/unknown".
*/
private long expiration;
/**
* Constructor.
*/
public MarshalledCredentials() {
}
/**
* Create from a set of properties.
* No expiry time is expected/known here.
* @param accessKey access key
* @param secretKey secret key
* @param sessionToken session token
*/
public MarshalledCredentials(
final String accessKey,
final String secretKey,
final String sessionToken) {
this();
this.accessKey = requireNonNull(accessKey);
this.secretKey = requireNonNull(secretKey);
this.sessionToken = sessionToken == null ? "" : sessionToken;
}
public String getAccessKey() {
return accessKey;
}
public String getSecretKey() {
return secretKey;
}
public String getSessionToken() {
return sessionToken;
}
/**
* Expiration; will be 0 for none known.
* @return any expiration timestamp
*/
public long getExpiration() {
return expiration;
}
public void setExpiration(final long expiration) {
this.expiration = expiration;
}
/**
* Get a temporal representing the time of expiration, if there
* is one.
* This is here to wrap up expectations about timestamps and zones.
* @return the expiration time.
*/
public Optional<OffsetDateTime> getExpirationDateTime() {
return expiration == 0
? Optional.empty()
: Optional.of(
OffsetDateTime.ofInstant(
new Date(expiration).toInstant(),
ZoneOffset.UTC));
}
public String getRoleARN() {
return roleARN;
}
public void setRoleARN(String roleARN) {
this.roleARN = requireNonNull(roleARN);
}
public void setAccessKey(final String accessKey) {
this.accessKey = requireNonNull(accessKey, "access key");
}
public void setSecretKey(final String secretKey) {
this.secretKey = requireNonNull(secretKey, "secret key");
}
public void setSessionToken(final String sessionToken) {
this.sessionToken = requireNonNull(sessionToken, "session token");
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
MarshalledCredentials that = (MarshalledCredentials) o;
return expiration == that.expiration &&
Objects.equals(accessKey, that.accessKey) &&
Objects.equals(secretKey, that.secretKey) &&
Objects.equals(sessionToken, that.sessionToken) &&
Objects.equals(roleARN, that.roleARN);
}
@Override
public int hashCode() {
return Objects.hash(accessKey, secretKey, sessionToken, roleARN,
expiration);
}
/**
* String value MUST NOT include any secrets.
* @return a string value for logging.
*/
@Override
public String toString() {
if (isEmpty()) {
return "Empty credentials";
}
String validity = isValid(CredentialTypeRequired.AnyNonEmpty)
? "valid"
: "invalid";
if (!hasSessionToken()) {
// full credentials have the simplest string value.
return "full credentials (" + validity + ")";
} else {
// session/role credentials may have an expiry and role ARN.
return String.format("session credentials, expiry %s; %s(%s)",
getExpirationDateTime()
.map(x -> x.format(DateTimeFormatter.ISO_DATE))
.orElse("unknown"),
(isNotEmpty(roleARN)
? ("role \"" + roleARN + "\" ")
: ""),
validity);
}
}
/**
* Is this empty: does it contain any credentials at all?
* This test returns true if either the access key or secret key is empty.
* @return true if there are no credentials.
*/
public boolean isEmpty() {
return !(isNotEmpty(accessKey) && isNotEmpty(secretKey));
}
/**
* Is this a valid set of credentials tokens?
* @param required credential type required.
* @return true if the requirements are met.
*/
public boolean isValid(final CredentialTypeRequired required) {
if (accessKey == null || secretKey == null || sessionToken == null) {
// null fields are not permitted, empty is OK for marshalling around.
return false;
}
// now look at whether values are set/unset.
boolean hasAccessAndSecretKeys = isNotEmpty(accessKey)
&& isNotEmpty(secretKey);
boolean hasSessionToken = hasSessionToken();
switch (required) {
case AnyIncludingEmpty:
// this is simplest.
return true;
case Empty:
// empty. ignore session value if the other keys are unset.
return !hasAccessAndSecretKeys;
case AnyNonEmpty:
// just look for the access key and secret key being non-empty
return hasAccessAndSecretKeys;
case FullOnly:
return hasAccessAndSecretKeys && !hasSessionToken;
case SessionOnly:
return hasAccessAndSecretKeys && hasSessionToken();
// this is here to keep the IDE quiet
default:
return false;
}
}
/**
* Does this set of credentials have a session token.
* @return true if there's a session token.
*/
public boolean hasSessionToken() {
return isNotEmpty(sessionToken);
}
/**
* Write the token.
* Only works if valid.
* @param out stream to serialize to.
* @throws IOException if the serialization failed.
*/
@Override
public void write(DataOutput out) throws IOException {
validate("Writing " + this + ": ",
CredentialTypeRequired.AnyIncludingEmpty);
Text.writeString(out, accessKey);
Text.writeString(out, secretKey);
Text.writeString(out, sessionToken);
Text.writeString(out, roleARN);
out.writeLong(expiration);
}
/**
* Read in the fields.
* @throws IOException IO problem
*/
@Override
public void readFields(DataInput in) throws IOException {
accessKey = Text.readString(in, MAX_SECRET_LENGTH);
secretKey = Text.readString(in, MAX_SECRET_LENGTH);
sessionToken = Text.readString(in, MAX_SECRET_LENGTH);
roleARN = Text.readString(in, MAX_SECRET_LENGTH);
expiration = in.readLong();
}
/**
* Verify that a set of credentials is valid.
* @throws DelegationTokenIOException if they aren't
* @param message message to prefix errors;
* @param typeRequired credential type required.
*/
public void validate(final String message,
final CredentialTypeRequired typeRequired) throws IOException {
if (!isValid(typeRequired)) {
throw new DelegationTokenIOException(message
+ buildInvalidCredentialsError(typeRequired));
}
}
/**
* Build an error string for when the credentials do not match
* those required.
* @param typeRequired credential type required.
* @return an error string.
*/
public String buildInvalidCredentialsError(
final CredentialTypeRequired typeRequired) {
if (isEmpty()) {
return " " + MarshalledCredentialBinding.NO_AWS_CREDENTIALS;
} else {
return " " + INVALID_CREDENTIALS
+ " in " + toString() + " required: " + typeRequired;
}
}
/**
* Patch a configuration with the secrets.
* This does not set any per-bucket options (it doesn't know the bucket...).
* <i>Warning: once done the configuration must be considered sensitive.</i>
* @param config configuration to patch
*/
public void setSecretsInConfiguration(Configuration config) {
config.set(ACCESS_KEY, accessKey);
config.set(SECRET_KEY, secretKey);
S3AUtils.setIfDefined(config, SESSION_TOKEN, sessionToken,
"session credentials");
}
/**
* Return a set of empty credentials.
* These can be marshalled, but not used for login.
* @return a new set of credentials.
*/
public static MarshalledCredentials empty() {
return new MarshalledCredentials("", "", "");
}
/**
* Enumeration of credential types for use in validation methods.
*/
public enum CredentialTypeRequired {
/** No entry at all. */
Empty("None"),
/** Any credential type including "unset". */
AnyIncludingEmpty("Full, Session or None"),
/** Any credential type is OK. */
AnyNonEmpty("Full or Session"),
/** The credentials must be session or role credentials. */
SessionOnly("Session"),
/** Full credentials are required. */
FullOnly("Full");
private final String text;
CredentialTypeRequired(final String text) {
this.text = text;
}
public String getText() {
return text;
}
@Override
public String toString() {
return getText();
}
}
}

View File

@ -18,14 +18,14 @@
package org.apache.hadoop.fs.s3a.auth;
import com.amazonaws.AmazonClientException;
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
/**
* A specific subclass of {@code AmazonClientException} which can
* be used in the retry logic to fail fast when there is any
* A specific subclass of {@code AmazonClientException} which is
* used in the S3A retry policy to fail fast when there is any
* authentication problem.
*/
public class NoAuthWithAWSException extends AmazonClientException {
public class NoAuthWithAWSException extends CredentialInitializationException {
public NoAuthWithAWSException(final String message, final Throwable t) {
super(message, t);

View File

@ -0,0 +1,69 @@
/*
* 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.auth;
import javax.annotation.Nonnull;
/**
* A special exception which declares that no credentials were found;
* this can be treated specially in logging, handling, etc.
* As it subclasses {@link NoAuthWithAWSException}, the S3A retry handler
* knows not to attempt to ask for the credentials again.
*/
public class NoAwsCredentialsException extends
NoAuthWithAWSException {
/**
* The default error message: {@value}.
*/
public static final String E_NO_AWS_CREDENTIALS = "No AWS Credentials";
/**
* Construct.
* @param credentialProvider name of the credential provider.
* @param message message.
*/
public NoAwsCredentialsException(
@Nonnull final String credentialProvider,
@Nonnull final String message) {
this(credentialProvider, message, null);
}
/**
* Construct with the default message of {@link #E_NO_AWS_CREDENTIALS}.
* @param credentialProvider name of the credential provider.
*/
public NoAwsCredentialsException(
@Nonnull final String credentialProvider) {
this(credentialProvider, E_NO_AWS_CREDENTIALS, null);
}
/**
* Construct with exception.
* @param credentialProvider name of the credential provider.
* @param message message.
* @param thrown inner exception
*/
public NoAwsCredentialsException(
@Nonnull final String credentialProvider,
@Nonnull final String message,
final Throwable thrown) {
super(credentialProvider + ": " + message, thrown);
}
}

View File

@ -19,7 +19,7 @@
package org.apache.hadoop.fs.s3a.auth;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
@ -35,8 +35,8 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.util.JsonSerialization;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static java.util.Objects.requireNonNull;
/**
* Jackson Role Model for Role Properties, for API clients and tests.
@ -173,6 +173,21 @@ public static Statement statement(boolean allow,
.addResources(scope);
}
/**
* Create a statement.
* @param allow allow or deny
* @param scope scope
* @param actions actions
* @return the formatted json statement
*/
public static Statement statement(boolean allow,
String scope,
Collection<String> actions) {
return new Statement(RoleModel.effect(allow))
.addActions(actions)
.addResources(scope);
}
/**
* Create a statement.
* If {@code isDirectory} is true, a "/" is added to the path.
@ -196,6 +211,29 @@ public static Statement statement(
.addResources(resource(path, isDirectory, wildcards));
}
/**
* Create a statement.
* If {@code isDirectory} is true, a "/" is added to the path.
* This is critical when adding wildcard permissions under
* a directory, and also needed when locking down dir-as-file
* and dir-as-directory-marker access.
* @param allow allow or deny
* @param path path
* @param isDirectory is this a directory?
* @param actions action
* @return the formatted json statement
*/
public static Statement statement(
final boolean allow,
final Path path,
final boolean isDirectory,
final boolean wildcards,
final Collection<String> actions) {
return new Statement(RoleModel.effect(allow))
.addActions(actions)
.addResources(resource(path, isDirectory, wildcards));
}
/**
* From a set of statements, create a policy.
* @param statements statements
@ -264,8 +302,8 @@ public Statement(final Effects effect) {
@Override
public void validate() {
checkNotNull(sid, "Sid");
checkNotNull(effect, "Effect");
requireNonNull(sid, "Sid");
requireNonNull(effect, "Effect");
checkState(!(action.isEmpty()), "Empty Action");
checkState(!(resource.isEmpty()), "Empty Resource");
}
@ -280,11 +318,25 @@ public Statement addActions(String... actions) {
return this;
}
public Statement addActions(Collection<String> actions) {
action.addAll(actions);
return this;
}
public Statement addResources(String... resources) {
Collections.addAll(resource, resources);
return this;
}
/**
* Add a list of resources.
* @param resources resource list
* @return this statement.
*/
public Statement addResources(Collection<String> resources) {
resource.addAll(resources);
return this;
}
}
/**
@ -298,12 +350,20 @@ public static class Policy extends RoleElt {
@JsonProperty("Statement")
public List<Statement> statement;
/**
* Empty constructor: initializes the statements to an empty list.
*/
public Policy() {
statement = new ArrayList<>();
}
public Policy(final List<RoleModel.Statement> statement) {
this.statement = statement;
}
public Policy(RoleModel.Statement... statements) {
statement = Arrays.asList(statements);
statement = new ArrayList<>(statements.length);
Collections.addAll(statement, statements);
}
/**
@ -311,11 +371,34 @@ public Policy(RoleModel.Statement... statements) {
*/
@Override
public void validate() {
checkNotNull(statement, "Statement");
requireNonNull(statement, "Statement");
checkState(VERSION.equals(version), "Invalid Version: %s", version);
statement.stream().forEach((a) -> a.validate());
}
/**
* Add the statements of another policy to this one.
* @param other other policy.
*/
public void add(Policy other) {
add(other.statement);
}
/**
* Add a collection of statements.
* @param statements statements to add.
*/
public void add(Collection<Statement> statements) {
statement.addAll(statements);
}
/**
* Add a single statement.
* @param stat new statement.
*/
public void add(Statement stat) {
statement.add(stat);
}
}

View File

@ -18,12 +18,24 @@
package org.apache.hadoop.fs.s3a.auth;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import com.google.common.collect.Lists;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
/**
* Operations, statements and policies covering the operations
* needed to work with S3 and S3Guard.
*/
@InterfaceAudience.LimitedPrivate("Tests")
@InterfaceStability.Unstable
public final class RolePolicies {
private RolePolicies() {
@ -88,27 +100,36 @@ private RolePolicies() {
*/
public static final String S3_ALL_BUCKETS = "arn:aws:s3:::*";
/**
* All bucket list operations, including
* {@link #S3_BUCKET_LIST_BUCKET} and
* {@link #S3_BUCKET_LIST_MULTIPART_UPLOADS}.
*/
public static final String S3_BUCKET_ALL_LIST = "s3:ListBucket*";
public static final String S3_ALL_LIST_OPERATIONS = "s3:List*";
public static final String S3_ALL_LIST_BUCKET = "s3:ListBucket*";
public static final String S3_LIST_BUCKET = "s3:ListBucket";
/**
* List the contents of a bucket.
* It applies to a bucket, not to a path in a bucket.
*/
public static final String S3_BUCKET_LIST_BUCKET = "s3:ListBucket";
/**
* This is used by the abort operation in S3A commit work.
* It applies to a bucket, not to a path in a bucket.
*/
public static final String S3_LIST_BUCKET_MULTPART_UPLOADS =
public static final String S3_BUCKET_LIST_MULTIPART_UPLOADS =
"s3:ListBucketMultipartUploads";
/**
* List multipart upload is needed for the S3A Commit protocols.
* It applies to a path in a bucket.
*/
public static final String S3_LIST_MULTIPART_UPLOAD_PARTS
= "s3:ListMultipartUploadParts";
/**
* abort multipart upload is needed for the S3A Commit protocols.
* Abort multipart upload is needed for the S3A Commit protocols.
* It applies to a path in a bucket.
*/
public static final String S3_ABORT_MULTIPART_UPLOAD
= "s3:AbortMultipartUpload";
@ -181,64 +202,81 @@ private RolePolicies() {
* Actions needed to read a file in S3 through S3A, excluding
* S3Guard and SSE-KMS.
*/
public static final String[] S3_PATH_READ_OPERATIONS =
private static final String[] S3_PATH_READ_OPERATIONS =
new String[]{
S3_GET_OBJECT,
};
/**
* Base actions needed to read data from S3 through S3A,
* excluding SSE-KMS data and S3Guard-ed buckets.
* excluding:
* <ol>
* <li>bucket-level operations</li>
* <li>SSE-KMS key operations</li>
* <li>DynamoDB operations for S3Guard.</li>
* </ol>
* As this excludes the bucket list operations, it is not sufficient
* to read from a bucket on its own.
*/
public static final String[] S3_ROOT_READ_OPERATIONS =
private static final String[] S3_ROOT_READ_OPERATIONS =
new String[]{
S3_LIST_BUCKET,
S3_LIST_BUCKET_MULTPART_UPLOADS,
S3_ALL_GET,
};
public static final List<String> S3_ROOT_READ_OPERATIONS_LIST =
Collections.unmodifiableList(Arrays.asList(S3_ALL_GET));
/**
* Policies which can be applied to bucket resources for read operations.
* <ol>
* <li>SSE-KMS key operations</li>
* <li>DynamoDB operations for S3Guard.</li>
* </ol>
*/
public static final String[] S3_BUCKET_READ_OPERATIONS =
new String[]{
S3_ALL_GET,
S3_BUCKET_ALL_LIST,
};
/**
* Actions needed to write data to an S3A Path.
* This includes the appropriate read operations, but
* not SSE-KMS or S3Guard support.
*/
public static final String[] S3_PATH_RW_OPERATIONS =
new String[]{
public static final List<String> S3_PATH_RW_OPERATIONS =
Collections.unmodifiableList(Arrays.asList(new String[]{
S3_ALL_GET,
S3_PUT_OBJECT,
S3_DELETE_OBJECT,
S3_ABORT_MULTIPART_UPLOAD,
S3_LIST_MULTIPART_UPLOAD_PARTS,
};
}));
/**
* Actions needed to write data to an S3A Path.
* This is purely the extra operations needed for writing atop
* of the read operation set.
* Deny these and a path is still readable, but not writeable.
* Excludes: SSE-KMS and S3Guard permissions.
* Excludes: bucket-ARN, SSE-KMS and S3Guard permissions.
*/
public static final String[] S3_PATH_WRITE_OPERATIONS =
new String[]{
public static final List<String> S3_PATH_WRITE_OPERATIONS =
Collections.unmodifiableList(Arrays.asList(new String[]{
S3_PUT_OBJECT,
S3_DELETE_OBJECT,
S3_ABORT_MULTIPART_UPLOAD
};
}));
/**
* Actions needed for R/W IO from the root of a bucket.
* Excludes: SSE-KMS and S3Guard permissions.
* Excludes: bucket-ARN, SSE-KMS and S3Guard permissions.
*/
public static final String[] S3_ROOT_RW_OPERATIONS =
new String[]{
S3_LIST_BUCKET,
public static final List<String> S3_ROOT_RW_OPERATIONS =
Collections.unmodifiableList(Arrays.asList(new String[]{
S3_ALL_GET,
S3_PUT_OBJECT,
S3_DELETE_OBJECT,
S3_ABORT_MULTIPART_UPLOAD,
S3_LIST_MULTIPART_UPLOAD_PARTS,
S3_ALL_LIST_BUCKET,
};
}));
/**
* All DynamoDB operations: {@value}.
@ -300,24 +338,15 @@ private RolePolicies() {
/**
* Statement to allow all DDB access.
*/
public static final Statement STATEMENT_ALL_DDB = statement(true,
ALL_DDB_TABLES, DDB_ALL_OPERATIONS);
public static final Statement STATEMENT_ALL_DDB =
allowAllDynamoDBOperations(ALL_DDB_TABLES);
/**
* Statement to allow all client operations needed for S3Guard,
* but none of the admin operations.
*/
public static final Statement STATEMENT_S3GUARD_CLIENT = statement(true,
ALL_DDB_TABLES,
DDB_BATCH_GET_ITEM,
DDB_BATCH_WRITE_ITEM,
DDB_DELETE_ITEM,
DDB_DESCRIBE_TABLE,
DDB_GET_ITEM,
DDB_PUT_ITEM,
DDB_QUERY,
DDB_UPDATE_ITEM
);
public static final Statement STATEMENT_S3GUARD_CLIENT =
allowS3GuardClientOperations(ALL_DDB_TABLES);
/**
* Allow all S3 Operations.
@ -327,13 +356,92 @@ private RolePolicies() {
S3_ALL_BUCKETS,
S3_ALL_OPERATIONS);
/**
* The s3:GetBucketLocation permission is for all buckets, not for
* any named bucket, which complicates permissions.
*/
public static final Statement STATEMENT_ALL_S3_GET_BUCKET_LOCATION =
statement(true,
S3_ALL_BUCKETS,
S3_GET_BUCKET_LOCATION);
/**
* Policy for all S3 and S3Guard operations, and SSE-KMS.
*/
public static final Policy ALLOW_S3_AND_SGUARD = policy(
STATEMENT_ALL_S3,
STATEMENT_ALL_DDB,
STATEMENT_ALLOW_SSE_KMS_RW
STATEMENT_ALLOW_SSE_KMS_RW,
STATEMENT_ALL_S3_GET_BUCKET_LOCATION
);
public static Statement allowS3GuardClientOperations(String tableArn) {
return statement(true,
tableArn,
DDB_BATCH_GET_ITEM,
DDB_BATCH_WRITE_ITEM,
DDB_DELETE_ITEM,
DDB_DESCRIBE_TABLE,
DDB_GET_ITEM,
DDB_PUT_ITEM,
DDB_QUERY,
DDB_UPDATE_ITEM
);
}
public static Statement allowAllDynamoDBOperations(String tableArn) {
return statement(true,
tableArn,
DDB_ALL_OPERATIONS);
}
/**
* From an S3 bucket name, build an ARN to refer to it.
* @param bucket bucket name.
* @param write are write permissions required
* @return return statement granting access.
*/
public static List<Statement> allowS3Operations(String bucket,
boolean write) {
// add the bucket operations for the specific bucket ARN
ArrayList<Statement> statements =
Lists.newArrayList(
statement(true,
bucketToArn(bucket),
S3_GET_BUCKET_LOCATION, S3_BUCKET_ALL_LIST));
// then add the statements for objects in the buckets
if (write) {
statements.add(
statement(true,
bucketObjectsToArn(bucket),
S3_ROOT_RW_OPERATIONS));
} else {
statements.add(
statement(true,
bucketObjectsToArn(bucket),
S3_ROOT_READ_OPERATIONS_LIST));
}
return statements;
}
/**
* From an S3 bucket name, build an ARN to refer to all objects in
* it.
* @param bucket bucket name.
* @return return the ARN to use in statements.
*/
public static String bucketObjectsToArn(String bucket) {
return String.format("arn:aws:s3:::%s/*", bucket);
}
/**
* From an S3 bucket name, build an ARN to refer to it.
* @param bucket bucket name.
* @return return the ARN to use in statements.
*/
public static String bucketToArn(String bucket) {
return String.format("arn:aws:s3:::%s", bucket);
}
}

View File

@ -18,22 +18,33 @@
package org.apache.hadoop.fs.s3a.auth;
import java.io.Closeable;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
import com.amazonaws.services.securitytoken.model.AssumeRoleRequest;
import com.amazonaws.services.securitytoken.model.Credentials;
import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AUtils;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
/**
* Factory for creating STS Clients.
*/
@ -48,31 +59,179 @@ public class STSClientFactory {
* Create the builder ready for any final configuration options.
* Picks up connection settings from the Hadoop configuration, including
* proxy secrets.
* The endpoint comes from the configuration options
* {@link org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants#DELEGATION_TOKEN_ENDPOINT}
* and
* {@link org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants#DELEGATION_TOKEN_REGION}
* @param conf Configuration to act as source of options.
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @param credentials AWS credential chain to use
* @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
* @param stsRegion the region, e.g "us-west-1"
* @return the builder to call {@code build()}
* @throws IOException problem reading proxy secrets
*/
public static AWSSecurityTokenServiceClientBuilder builder(
final Configuration conf,
final String bucket,
final AWSCredentialsProvider credentials, final String stsEndpoint,
final AWSCredentialsProvider credentials) throws IOException {
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
String endpoint = conf.getTrimmed(DELEGATION_TOKEN_ENDPOINT,
DEFAULT_DELEGATION_TOKEN_ENDPOINT);
String region = conf.getTrimmed(DELEGATION_TOKEN_REGION,
DEFAULT_DELEGATION_TOKEN_REGION);
return builder(credentials, awsConf, endpoint, region);
}
/**
* Create the builder ready for any final configuration options.
* Picks up connection settings from the Hadoop configuration, including
* proxy secrets.
* @param conf Configuration to act as source of options.
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @param credentials AWS credential chain to use
* @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
* @param stsRegion AWS recommend setting the endpoint instead.
* @return the builder to call {@code build()}
* @throws IOException problem reading proxy secrets
*/
public static AWSSecurityTokenServiceClientBuilder builder(
final Configuration conf,
final String bucket,
final AWSCredentialsProvider credentials,
final String stsEndpoint,
final String stsRegion) throws IOException {
Preconditions.checkArgument(credentials != null, "No credentials");
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
return builder(credentials, awsConf, stsEndpoint, stsRegion);
}
/**
* Create the builder ready for any final configuration options.
* Picks up connection settings from the Hadoop configuration, including
* proxy secrets.
* @param awsConf AWS configuration.
* @param credentials AWS credential chain to use
* @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
* @param stsRegion the region, e.g "us-west-1". Must be set if endpoint is.
* @return the builder to call {@code build()}
*/
public static AWSSecurityTokenServiceClientBuilder builder(
final AWSCredentialsProvider credentials,
final ClientConfiguration awsConf,
final String stsEndpoint,
final String stsRegion) {
final AWSSecurityTokenServiceClientBuilder builder
= AWSSecurityTokenServiceClientBuilder.standard();
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
Preconditions.checkArgument(credentials != null, "No credentials");
builder.withClientConfiguration(awsConf);
builder.withCredentials(credentials);
if (StringUtils.isNotEmpty(stsEndpoint)) {
LOG.debug("STS Endpoint ={}", stsEndpoint);
boolean destIsStandardEndpoint = STS_STANDARD.equals(stsEndpoint);
if (isNotEmpty(stsEndpoint) && !destIsStandardEndpoint) {
Preconditions.checkArgument(
isNotEmpty(stsRegion),
"STS endpoint is set to %s but no signing region was provided",
stsEndpoint);
LOG.debug("STS Endpoint={}; region='{}'", stsEndpoint, stsRegion);
builder.withEndpointConfiguration(
new AwsClientBuilder.EndpointConfiguration(stsEndpoint, stsRegion));
} else {
Preconditions.checkArgument(isEmpty(stsRegion),
"STS signing region set set to %s but no STS endpoint specified",
stsRegion);
}
return builder;
}
/**
* Create an STS Client instance.
* @param tokenService STS instance
* @param invoker invoker to use
* @return an STS client bonded to that interface.
* @throws IOException on any failure
*/
public static STSClient createClientConnection(
final AWSSecurityTokenService tokenService,
final Invoker invoker)
throws IOException {
return new STSClient(tokenService, invoker);
}
/**
* STS client connection with retries.
*/
public static final class STSClient implements Closeable {
private final AWSSecurityTokenService tokenService;
private final Invoker invoker;
private STSClient(final AWSSecurityTokenService tokenService,
final Invoker invoker) {
this.tokenService = tokenService;
this.invoker = invoker;
}
@Override
public void close() throws IOException {
try {
tokenService.shutdown();
} catch (UnsupportedOperationException ignored) {
// ignore this, as it is what the STS client currently
// does.
}
}
/**
* Request a set of session credentials.
*
* @param duration duration of the credentials
* @param timeUnit time unit of duration
* @return the role result
* @throws IOException on a failure of the request
*/
@Retries.RetryTranslated
public Credentials requestSessionCredentials(
final long duration,
final TimeUnit timeUnit) throws IOException {
int durationSeconds = (int) timeUnit.toSeconds(duration);
LOG.debug("Requesting session token of duration {}", duration);
final GetSessionTokenRequest request = new GetSessionTokenRequest();
request.setDurationSeconds(durationSeconds);
return invoker.retry("request session credentials", "",
true,
() ->{
LOG.info("Requesting Amazon STS Session credentials");
return tokenService.getSessionToken(request).getCredentials();
});
}
/**
* Request a set of role credentials.
*
* @param roleARN ARN to request
* @param sessionName name of the session
* @param policy optional policy; "" is treated as "none"
* @param duration duration of the credentials
* @param timeUnit time unit of duration
* @return the role result
* @throws IOException on a failure of the request
*/
@Retries.RetryTranslated
public Credentials requestRole(
final String roleARN,
final String sessionName,
final String policy,
final long duration,
final TimeUnit timeUnit) throws IOException {
LOG.debug("Requesting role {} with duration {}; policy = {}",
roleARN, duration, policy);
AssumeRoleRequest request = new AssumeRoleRequest();
request.setDurationSeconds((int) timeUnit.toSeconds(duration));
request.setRoleArn(roleARN);
request.setRoleSessionName(sessionName);
if (isNotEmpty(policy)) {
request.setPolicy(policy);
}
return invoker.retry("request role credentials", "", true,
() -> tokenService.assumeRole(request).getCredentials());
}
}
}

View File

@ -0,0 +1,59 @@
/*
* 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.auth.delegation;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
/**
* Interface for providers of AWS policy for accessing data.
* This is used when building up the role permissions for a delegation
* token.
*
* The permissions requested are from the perspective of
* S3A filesystem operations on the data, <i>not</i> the simpler
* model of "permissions on the the remote service".
* As an example, to use S3Guard effectively, the client needs full CRUD
* access to the table, even for {@link AccessLevel#READ}.
*/
public interface AWSPolicyProvider {
/**
* Get the AWS policy statements required for accessing this service.
*
* @param access access level desired.
* @return a possibly empty list of statements to grant access at that
* level.
*/
List<RoleModel.Statement> listAWSPolicyRules(Set<AccessLevel> access);
/**
* Access levels.
*/
enum AccessLevel {
/** Filesystem data read operations. */
READ,
/** Data write, encryption, etc. */
WRITE,
/** Administration of the data, tables, etc. */
ADMIN,
}
}

View File

@ -0,0 +1,154 @@
/*
* 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.auth.delegation;
import java.io.IOException;
import java.net.URI;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.AbstractService;
import static java.util.Objects.requireNonNull;
/**
* This is the base class for both the delegation binding
* code and the back end service created; allows for
* shared methods across both.
*
* The lifecycle sequence is as follows
* <pre>
* - create
* - bindToFileSystem(uri, ownerFS)
* - init
* - start
* ...api calls...
* - stop
* </pre>
*
* As the S3ADelegation mechanism is all configured during the filesystem
* initalize() operation, it is not ready for use through all the start process.
*/
public abstract class AbstractDTService
extends AbstractService {
/**
* URI of the filesystem.
* Valid after {@link #bindToFileSystem(URI, S3AFileSystem)}.
*/
private URI canonicalUri;
/**
* The owning filesystem.
* Valid after {@link #bindToFileSystem(URI, S3AFileSystem)}.
*/
private S3AFileSystem fileSystem;
/**
* Owner of the filesystem.
* Valid after {@link #bindToFileSystem(URI, S3AFileSystem)}.
*/
private UserGroupInformation owner;
/**
* Protected constructor.
* @param name service name.
*/
protected AbstractDTService(final String name) {
super(name);
}
/**
* Bind to the filesystem.
* Subclasses can use this to perform their own binding operations -
* but they must always call their superclass implementation.
* This <i>Must</i> be called before calling {@code init()}.
*
* <b>Important:</b>
* This binding will happen during FileSystem.initialize(); the FS
* is not live for actual use and will not yet have interacted with
* AWS services.
* @param uri the canonical URI of the FS.
* @param fs owning FS.
* @throws IOException failure.
*/
public void bindToFileSystem(
final URI uri,
final S3AFileSystem fs) throws IOException {
requireServiceState(STATE.NOTINITED);
Preconditions.checkState(canonicalUri == null,
"bindToFileSystem called twice");
this.canonicalUri = requireNonNull(uri);
this.fileSystem = requireNonNull(fs);
this.owner = fs.getOwner();
}
/**
* Get the canonical URI of the filesystem, which is what is
* used to identify the tokens.
* @return the URI.
*/
public URI getCanonicalUri() {
return canonicalUri;
}
/**
* Get the owner of the FS.
* @return the owner fs
*/
protected S3AFileSystem getFileSystem() {
return fileSystem;
}
/**
* Get the owner of this Service.
* @return owner; non-null after binding to an FS.
*/
public UserGroupInformation getOwner() {
return owner;
}
/**
* Require that the service is in a given state.
* @param state desired state.
* @throws IllegalStateException if the condition is not met
*/
protected void requireServiceState(final STATE state)
throws IllegalStateException {
Preconditions.checkState(isInState(state),
"Required State: %s; Actual State %s", state, getServiceState());
}
/**
* Require the service to be started.
* @throws IllegalStateException if it is not.
*/
protected void requireServiceStarted() throws IllegalStateException {
requireServiceState(STATE.STARTED);
}
@Override
protected void serviceInit(final Configuration conf) throws Exception {
super.serviceInit(conf);
requireNonNull(canonicalUri, "service does not have a canonical URI");
}
}

View File

@ -0,0 +1,305 @@
/*
* 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.auth.delegation;
import java.io.IOException;
import java.net.URI;
import java.nio.charset.Charset;
import java.util.Optional;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.commit.DurationInfo;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.Token;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DURATION_LOG_AT_INFO;
/**
* An AbstractDelegationTokenBinding implementation is a class which
* handles the binding of its underlying authentication mechanism to the
* Hadoop Delegation token mechanism.
*
* See also {@code org.apache.hadoop.fs.azure.security.WasbDelegationTokenManager}
* but note that it assumes Kerberos tokens for which the renewal mechanism
* is the sole plugin point.
* This class is designed to be more generic.
*
* <b>Lifecycle</b>
*
* It is a Hadoop Service, so has a standard lifecycle: once started
* its lifecycle will follow that of the {@link S3ADelegationTokens}
* instance which created it --which itself follows the lifecycle of the FS.
*
* One big difference is that
* {@link #bindToFileSystem(URI, S3AFileSystem)} will be called
* before the {@link #init(Configuration)} operation, this is where
* the owning FS is passed in.
*
* Implementations are free to start background operations in their
* {@code serviceStart()} method, provided they are safely stopped in
* {@code serviceStop()}.
*
* <b>When to check for the ability to issue tokens</b>
* Implementations MUST start up without actually holding the secrets
* needed to issue tokens (config options, credentials to talk to STS etc)
* as in server-side deployments they are not expected to have these.
*
* <b>Retry Policy</b>
*
* All methods which talk to AWS services are expected to do translation,
* with retries as they see fit.
*/
public abstract class AbstractDelegationTokenBinding extends AbstractDTService {
/** Token kind: must match that of the token identifiers issued. */
private final Text kind;
private SecretManager<AbstractS3ATokenIdentifier> secretManager;
private static final Logger LOG = LoggerFactory.getLogger(
AbstractDelegationTokenBinding.class);
/**
* Constructor.
*
* @param name as passed to superclass for use in log messages.
* @param kind token kind.
*/
protected AbstractDelegationTokenBinding(final String name,
final Text kind) {
super(name);
this.kind = requireNonNull(kind);
}
/**
* Get the kind of the tokens managed here.
* @return the token kind.
*/
public Text getKind() {
return kind;
}
/**
* Return the name of the owner to be used in tokens.
* This may be that of the UGI owner, or it could be related to
* the AWS login.
* @return a text name of the owner.
*/
public Text getOwnerText() {
return new Text(getOwner().getUserName());
}
/**
* Predicate: will this binding issue a DT?
* That is: should the filesystem declare that it is issuing
* delegation tokens? If true
* @return a declaration of what will happen when asked for a token.
*/
public S3ADelegationTokens.TokenIssuingPolicy getTokenIssuingPolicy() {
return S3ADelegationTokens.TokenIssuingPolicy.RequestNewToken;
}
/**
* Create a delegation token for the user.
* This will only be called if a new DT is needed, that is: the
* filesystem has been deployed unbonded.
* @param policy minimum policy to use, if known.
* @param encryptionSecrets encryption secrets for the token.
* @return the token or null if the back end does not want to issue one.
* @throws IOException if one cannot be created
*/
public Token<AbstractS3ATokenIdentifier> createDelegationToken(
final Optional<RoleModel.Policy> policy,
final EncryptionSecrets encryptionSecrets) throws IOException {
requireServiceStarted();
final AbstractS3ATokenIdentifier tokenIdentifier =
createTokenIdentifier(policy, encryptionSecrets);
if (tokenIdentifier != null) {
Token<AbstractS3ATokenIdentifier> token =
new Token<>(tokenIdentifier, secretManager);
token.setKind(getKind());
LOG.debug("Created token {} with token identifier {}",
token, tokenIdentifier);
return token;
} else {
return null;
}
}
/**
* Create a token identifier with all the information needed
* to be included in a delegation token.
* This is where session credentials need to be extracted, etc.
* This will only be called if a new DT is needed, that is: the
* filesystem has been deployed unbonded.
*
* If {@link #createDelegationToken(Optional, EncryptionSecrets)}
* is overridden, this method can be replaced with a stub.
*
* @param policy minimum policy to use, if known.
* @param encryptionSecrets encryption secrets for the token.
* @return the token data to include in the token identifier.
* @throws IOException failure creating the token data.
*/
public abstract AbstractS3ATokenIdentifier createTokenIdentifier(
Optional<RoleModel.Policy> policy,
EncryptionSecrets encryptionSecrets) throws IOException;
/**
* Verify that a token identifier is of a specific class.
* This will reject subclasses (i.e. it is stricter than
* {@code instanceof}, then cast it to that type.
* @param identifier identifier to validate
* @param expectedClass class of the expected token identifier.
* @throws DelegationTokenIOException If the wrong class was found.
*/
protected <T extends AbstractS3ATokenIdentifier> T convertTokenIdentifier(
final AbstractS3ATokenIdentifier identifier,
final Class<T> expectedClass) throws DelegationTokenIOException {
if (!identifier.getClass().equals(expectedClass)) {
throw new DelegationTokenIOException(
DelegationTokenIOException.TOKEN_WRONG_CLASS
+ "; expected a token identifier of type "
+ expectedClass
+ " but got "
+ identifier.getClass()
+ " and kind " + identifier.getKind());
}
return (T) identifier;
}
/**
* Perform any actions when deploying unbonded, and return a list
* of credential providers.
* @return non-empty list of AWS credential providers to use for
* authenticating this client with AWS services.
* @throws IOException any failure.
*/
public abstract AWSCredentialProviderList deployUnbonded()
throws IOException;
/**
* Bind to the token identifier, returning the credential providers to use
* for the owner to talk to S3, DDB and related AWS Services.
* @param retrievedIdentifier the unmarshalled data
* @return non-empty list of AWS credential providers to use for
* authenticating this client with AWS services.
* @throws IOException any failure.
*/
public abstract AWSCredentialProviderList bindToTokenIdentifier(
AbstractS3ATokenIdentifier retrievedIdentifier)
throws IOException;
/**
* Create a new subclass of {@link AbstractS3ATokenIdentifier}.
* This is used in the secret manager.
* @return an empty identifier.
*/
public abstract AbstractS3ATokenIdentifier createEmptyIdentifier();
@Override
public String toString() {
return super.toString()
+ " token kind = " + getKind();
}
/**
* Service startup: create the secret manager.
* @throws Exception failure.
*/
@Override
protected void serviceStart() throws Exception {
super.serviceStart();
secretManager = createSecretMananger();
}
/**
* Return a description.
* This is logged during after service start and binding:
* it should be as informative as possible.
* @return a description to log.
*/
public String getDescription() {
return "Token binding " + getKind().toString();
}
/**
* Create a secret manager.
* @return a secret manager.
* @throws IOException on failure
*/
protected SecretManager<AbstractS3ATokenIdentifier> createSecretMananger()
throws IOException {
return new TokenSecretManager();
}
/**
* Return a string for use in building up the User-Agent field, so
* get into the S3 access logs. Useful for diagnostics.
* @return a string for the S3 logs or "" for "nothing to add"
*/
public String getUserAgentField() {
return "";
}
/**
* Get the password to use in secret managers.
* This is a constant; its just recalculated every time to stop findbugs
* highlighting security risks of shared mutable byte arrays.
* @return a password.
*/
protected static byte[] getSecretManagerPasssword() {
return "non-password".getBytes(Charset.forName("UTF-8"));
}
/**
* The secret manager always uses the same secret; the
* factory for new identifiers is that of the token manager.
*/
protected class TokenSecretManager
extends SecretManager<AbstractS3ATokenIdentifier> {
@Override
protected byte[] createPassword(AbstractS3ATokenIdentifier identifier) {
return getSecretManagerPasssword();
}
@Override
public byte[] retrievePassword(AbstractS3ATokenIdentifier identifier)
throws InvalidToken {
return getSecretManagerPasssword();
}
@Override
public AbstractS3ATokenIdentifier createIdentifier() {
try (DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
"Creating Delegation Token Identifier")) {
return AbstractDelegationTokenBinding.this.createEmptyIdentifier();
}
}
}
}

View File

@ -0,0 +1,305 @@
/*
* 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.auth.delegation;
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.IOException;
import java.net.URI;
import java.util.Objects;
import java.util.UUID;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
import static java.util.Objects.requireNonNull;
/**
* An S3A Delegation Token Identifier: contains the information needed
* to talk to S3A.
*
* These are loaded via the service loader API an used in a map of
* Kind to class, which is then looked up to deserialize token
* identifiers of a given class.
*
* Every non-abstract class must provide
* <ol>
* <li>Their unique token kind.</li>
* <li>An empty constructor.</li>
* <li>An entry in the resource file
* {@code /META-INF/services/org.apache.hadoop.security.token.TokenIdentifier}
* </li>
* </ol>
*
* The base implementation contains
* <ol>
* <li>The URI of the FS.</li>
* <li>Encryption secrets for use in the destination FS.</li>
* </ol>
* Subclasses are required to add whatever information is needed to authenticate
* the user with the credential provider which their binding class will
* provide.
*
* <i>Important: Add no references to any AWS SDK class, to
* ensure it can be safely deserialized whenever the relevant token
* identifier of a token type declared in this JAR is examined.</i>
*/
public abstract class AbstractS3ATokenIdentifier
extends DelegationTokenIdentifier {
/**
* The maximum string length supported for text fields.
*/
protected static final int MAX_TEXT_LENGTH = 8192;
/** Canonical URI of the bucket. */
private URI uri;
/**
* Encryption secrets to also marshall with any credentials.
* Set during creation to ensure it is never null.
*/
private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
/**
* Timestamp of creation.
* This is set to the current time; it will be overridden when
* deserializing data.
*/
private long created = System.currentTimeMillis();
/**
* An origin string for diagnostics.
*/
private String origin = "";
/**
* This marshalled UUID can be used in testing to verify transmission,
* and reuse; as it is printed you can see what is happending too.
*/
private String uuid = UUID.randomUUID().toString();
/**
* Constructor.
* @param kind token kind.
* @param uri filesystem URI.
* @param owner token owner
* @param origin origin text for diagnostics.
* @param encryptionSecrets encryption secrets to set.
*/
protected AbstractS3ATokenIdentifier(
final Text kind,
final URI uri,
final Text owner,
final String origin,
final EncryptionSecrets encryptionSecrets) {
this(kind, owner, new Text(), new Text(), uri);
this.origin = requireNonNull(origin);
this.encryptionSecrets = requireNonNull(encryptionSecrets);
}
/**
* Constructor.
* @param kind token kind.
* @param owner token owner
* @param renewer token renewer
* @param realUser token real user
* @param uri filesystem URI.
*/
protected AbstractS3ATokenIdentifier(
final Text kind,
final Text owner,
final Text renewer,
final Text realUser,
final URI uri) {
super(kind, owner, renewer, realUser);
this.uri = requireNonNull(uri);
}
/**
* Build from a token.
* This has been written for refresh operations;
* if someone implements refresh it will be relevant.
* @param token to to build from
* @throws IOException failure to build the identifier.
*/
protected AbstractS3ATokenIdentifier(
final Text kind,
final Token<AbstractS3ATokenIdentifier> token) throws IOException {
super(kind);
ByteArrayInputStream bais = new ByteArrayInputStream(token.getIdentifier());
readFields(new DataInputStream(bais));
}
/**
* For subclasses to use in their own empty-constructors.
*/
protected AbstractS3ATokenIdentifier(final Text kind) {
super(kind);
}
public String getBucket() {
return uri.getHost();
}
public URI getUri() {
return uri;
}
public String getOrigin() {
return origin;
}
public void setOrigin(final String origin) {
this.origin = origin;
}
public long getCreated() {
return created;
}
/**
* Write state.
* {@link org.apache.hadoop.io.Writable#write(DataOutput)}.
* @param out destination
* @throws IOException failure
*/
@Override
public void write(final DataOutput out) throws IOException {
super.write(out);
Text.writeString(out, uri.toString());
Text.writeString(out, origin);
Text.writeString(out, uuid);
encryptionSecrets.write(out);
out.writeLong(created);
}
/**
* Read state.
* {@link org.apache.hadoop.io.Writable#readFields(DataInput)}.
*
* Note: this operation gets called in toString() operations on tokens, so
* must either always succeed, or throw an IOException to trigger the
* catch and downgrade. RuntimeExceptions (e.g. Preconditions checks) are
* not to be used here for this reason.)
*
* @param in input stream
* @throws DelegationTokenIOException if the token binding is wrong.
* @throws IOException IO problems.
*/
@Override
public void readFields(final DataInput in)
throws DelegationTokenIOException, IOException {
super.readFields(in);
uri = URI.create(Text.readString(in, MAX_TEXT_LENGTH));
origin = Text.readString(in, MAX_TEXT_LENGTH);
uuid = Text.readString(in, MAX_TEXT_LENGTH);
encryptionSecrets.readFields(in);
created = in.readLong();
}
/**
* Validate the token by looking at its fields.
* @throws IOException on failure.
*/
public void validate() throws IOException {
if (uri == null) {
throw new DelegationTokenIOException("No URI in " + this);
}
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"S3ATokenIdentifier{");
sb.append(getKind());
sb.append("; uri=").append(uri);
sb.append("; timestamp=").append(created);
sb.append("; encryption=").append(encryptionSecrets.toString());
sb.append("; ").append(uuid);
sb.append("; ").append(origin);
sb.append('}');
return sb.toString();
}
/**
* Equality check is on superclass and UUID only.
* @param o other.
* @return true if the base class considers them equal and the URIs match.
*/
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
final AbstractS3ATokenIdentifier that = (AbstractS3ATokenIdentifier) o;
return Objects.equals(uuid, that.uuid) &&
Objects.equals(uri, that.uri);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), uri);
}
/**
* Return the expiry time in seconds since 1970-01-01.
* @return the time when the session credential expire.
*/
public long getExpiryTime() {
return 0;
}
/**
* Get the UUID of this token identifier.
* @return a UUID.
*/
public String getUuid() {
return uuid;
}
/**
* Get the encryption secrets.
* @return the encryption secrets within this identifier.
*/
public EncryptionSecrets getEncryptionSecrets() {
return encryptionSecrets;
}
/**
* Create the default origin text message with local hostname and
* timestamp.
* @return a string for token diagnostics.
*/
public static String createDefaultOriginMessage() {
return String.format("Created on %s at time %s.",
NetUtils.getHostname(),
java.time.Instant.now());
}
}

View File

@ -0,0 +1,165 @@
/*
* 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.auth.delegation;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.io.Text;
/**
* All the constants related to delegation tokens.
* Not in the normal S3 constants while unstable.
*
* Where possible, the existing assumed role properties are used to configure
* STS binding, default ARN, etc. This makes documenting everything that
* much easier and avoids trying to debug precisely which sts endpoint
* property should be set.
*
* Most settings here are replicated in {@code core-default.xml}; the
* values MUST be kept in sync.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class DelegationConstants {
/**
* Endpoint for session tokens, used when building delegation tokens:
* {@value}.
* @see <a href="https://docs.aws.amazon.com/general/latest/gr/rande.html#sts_region">STS regions</a>
*/
public static final String DELEGATION_TOKEN_ENDPOINT =
Constants.ASSUMED_ROLE_STS_ENDPOINT;
/**
* Default endpoint for session tokens: {@value}.
*/
public static final String DEFAULT_DELEGATION_TOKEN_ENDPOINT =
Constants.DEFAULT_ASSUMED_ROLE_STS_ENDPOINT;
/**
* Region for DT issuing; must be non-empty if the endpoint is set: {@value}.
*/
public static final String DELEGATION_TOKEN_REGION =
Constants.ASSUMED_ROLE_STS_ENDPOINT_REGION;
/**
* Region default: {@value}.
*/
public static final String DEFAULT_DELEGATION_TOKEN_REGION =
Constants.ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT;
/**
* Duration of tokens in time: {@value}.
*/
public static final String DELEGATION_TOKEN_DURATION =
Constants.ASSUMED_ROLE_SESSION_DURATION;
/**
* Default duration of a delegation token: {@value}.
* Must be in the range supported by STS.
*/
public static final String DEFAULT_DELEGATION_TOKEN_DURATION =
Constants.ASSUMED_ROLE_SESSION_DURATION_DEFAULT;
/**
* Key to list AWS credential providers for Session/role
* credentials: {@value}.
*/
public static final String DELEGATION_TOKEN_CREDENTIALS_PROVIDER =
Constants.AWS_CREDENTIALS_PROVIDER;
/**
* ARN of the delegation token: {@value}.
* Required for the role token.
*/
public static final String DELEGATION_TOKEN_ROLE_ARN =
Constants.ASSUMED_ROLE_ARN;
/**
* Property containing classname for token binding: {@value}.
*/
public static final String DELEGATION_TOKEN_BINDING =
"fs.s3a.delegation.token.binding";
/**
* Session Token binding classname: {@value}.
*/
public static final String DELEGATION_TOKEN_SESSION_BINDING =
"org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding";
/**
* Default token binding {@value}.
*/
public static final String DEFAULT_DELEGATION_TOKEN_BINDING = "";
/**
* Token binding to pass full credentials: {@value}.
*/
public static final String DELEGATION_TOKEN_FULL_CREDENTIALS_BINDING =
"org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding";
/**
* Role DTs: {@value}.
*/
public static final String DELEGATION_TOKEN_ROLE_BINDING =
"org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding";
/** Prefix for token names: {@value}. */
public static final String TOKEN_NAME_PREFIX = "S3ADelegationToken/";
/** Name of session token: {@value}. */
public static final String SESSION_TOKEN_NAME = TOKEN_NAME_PREFIX + "Session";
/** Kind of the session token; value is {@link #SESSION_TOKEN_NAME}. */
public static final Text SESSION_TOKEN_KIND = new Text(SESSION_TOKEN_NAME);
/** Name of full token: {@value}. */
public static final String FULL_TOKEN_NAME = TOKEN_NAME_PREFIX + "Full";
/** Kind of the full token; value is {@link #FULL_TOKEN_NAME}. */
public static final Text FULL_TOKEN_KIND = new Text(FULL_TOKEN_NAME);
/** Name of role token: {@value}. */
public static final String ROLE_TOKEN_NAME = TOKEN_NAME_PREFIX + "Role";
/** Kind of the role token; value is {@link #ROLE_TOKEN_NAME}. */
public static final Text ROLE_TOKEN_KIND = new Text(ROLE_TOKEN_NAME);
/**
* Package-scoped option to control level that duration info on token
* binding operations are logged at.
* Value: {@value}.
*/
static final boolean DURATION_LOG_AT_INFO = true;
/**
* If the token binding auth chain is only session-level auth, you
* can't use the role binding: {@value}.
*/
public static final String E_NO_SESSION_TOKENS_FOR_ROLE_BINDING
= "Cannot issue S3A Role Delegation Tokens without full AWS credentials";
/**
* The standard STS server.
*/
public static final String STS_STANDARD = "sts.amazonaws.com";
private DelegationConstants() {
}
}

View File

@ -0,0 +1,50 @@
/*
* 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.auth.delegation;
import java.io.IOException;
/**
* General IOException for Delegation Token issues.
* Includes recommended error strings, which can be used in tests when
* looking for specific errors.
*/
public class DelegationTokenIOException extends IOException {
private static final long serialVersionUID = 599813827985340023L;
/** Error: delegation token/token identifier class isn't the right one. */
public static final String TOKEN_WRONG_CLASS
= "Delegation token is wrong class";
/**
* The far end is expecting a different token kind than
* that which the client created.
*/
protected static final String TOKEN_MISMATCH = "Token mismatch";
public DelegationTokenIOException(final String message) {
super(message);
}
public DelegationTokenIOException(final String message,
final Throwable cause) {
super(message, cause);
}
}

View File

@ -0,0 +1,73 @@
/*
* 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.auth.delegation;
import java.util.Optional;
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
import com.amazonaws.services.s3.model.SSECustomerKey;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
/**
* These support operations on {@link EncryptionSecrets} which use the AWS SDK
* operations. Isolating them here ensures that that class is not required on
* the classpath.
*/
public final class EncryptionSecretOperations {
private EncryptionSecretOperations() {
}
/**
* Create SSE-C client side key encryption options on demand.
* @return an optional key to attach to a request.
* @param secrets source of the encryption secrets.
*/
public static Optional<SSECustomerKey> createSSECustomerKey(
final EncryptionSecrets secrets) {
if (secrets.hasEncryptionKey() &&
secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) {
return Optional.of(new SSECustomerKey(secrets.getEncryptionKey()));
} else {
return Optional.empty();
}
}
/**
* Create SSE-KMS options for a request, iff the encryption is SSE-KMS.
* @return an optional SSE-KMS param to attach to a request.
* @param secrets source of the encryption secrets.
*/
public static Optional<SSEAwsKeyManagementParams> createSSEAwsKeyManagementParams(
final EncryptionSecrets secrets) {
//Use specified key, otherwise default to default master aws/s3 key by AWS
if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS) {
if (secrets.hasEncryptionKey()) {
return Optional.of(new SSEAwsKeyManagementParams(
secrets.getEncryptionKey()));
} else {
return Optional.of(new SSEAwsKeyManagementParams());
}
} else {
return Optional.empty();
}
}
}

View File

@ -0,0 +1,221 @@
/*
* 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.auth.delegation;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.Serializable;
import java.util.Objects;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
/**
* Encryption options in a form which can serialized or marshalled as a hadoop
* Writeable.
*
* Maintainers: For security reasons, don't print any of this.
*
* Note this design marshalls/unmarshalls its serialVersionUID
* in its writable, which is used to compare versions.
*
* <i>Important.</i>
* If the wire format is ever changed incompatibly,
* update the serial version UID to ensure that older clients get safely
* rejected.
*
* <i>Important</i>
* Do not import any AWS SDK classes, directly or indirectly.
* This is to ensure that S3A Token identifiers can be unmarshalled even
* without that SDK.
*/
public class EncryptionSecrets implements Writable, Serializable {
public static final int MAX_SECRET_LENGTH = 2048;
private static final long serialVersionUID = 1208329045511296375L;
/**
* Encryption algorithm to use: must match one in
* {@link S3AEncryptionMethods}.
*/
private String encryptionAlgorithm = "";
/**
* Encryption key: possibly sensitive information.
*/
private String encryptionKey = "";
/**
* This field isn't serialized/marshalled; it is rebuilt from the
* encryptionAlgorithm field.
*/
private transient S3AEncryptionMethods encryptionMethod =
S3AEncryptionMethods.NONE;
/**
* Empty constructor, for use in marshalling.
*/
public EncryptionSecrets() {
}
/**
* Create a pair of secrets.
* @param encryptionAlgorithm algorithm enumeration.
* @param encryptionKey key/key reference.
* @throws IOException failure to initialize.
*/
public EncryptionSecrets(final S3AEncryptionMethods encryptionAlgorithm,
final String encryptionKey) throws IOException {
this(encryptionAlgorithm.getMethod(), encryptionKey);
}
/**
* Create a pair of secrets.
* @param encryptionAlgorithm algorithm name
* @param encryptionKey key/key reference.
* @throws IOException failure to initialize.
*/
public EncryptionSecrets(final String encryptionAlgorithm,
final String encryptionKey) throws IOException {
this.encryptionAlgorithm = encryptionAlgorithm;
this.encryptionKey = encryptionKey;
init();
}
/**
* Write out the encryption secrets.
* @param out {@code DataOutput} to serialize this object into.
* @throws IOException IO failure
*/
@Override
public void write(final DataOutput out) throws IOException {
new LongWritable(serialVersionUID).write(out);
Text.writeString(out, encryptionAlgorithm);
Text.writeString(out, encryptionKey);
}
/**
* Read in from the writable stream.
* After reading, call {@link #init()}.
* @param in {@code DataInput} to deserialize this object from.
* @throws IOException failure to read/validate data.
*/
@Override
public void readFields(final DataInput in) throws IOException {
final LongWritable version = new LongWritable();
version.readFields(in);
if (version.get() != serialVersionUID) {
throw new DelegationTokenIOException(
"Incompatible EncryptionSecrets version");
}
encryptionAlgorithm = Text.readString(in, MAX_SECRET_LENGTH);
encryptionKey = Text.readString(in, MAX_SECRET_LENGTH);
init();
}
/**
* For java serialization: read and then call {@link #init()}.
* @param in input
* @throws IOException IO problem
* @throws ClassNotFoundException problem loading inner class.
*/
private void readObject(ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
init();
}
/**
* Init all state, including after any read.
* @throws IOException error rebuilding state.
*/
private void init() throws IOException {
encryptionMethod = S3AEncryptionMethods.getMethod(
encryptionAlgorithm);
}
public String getEncryptionAlgorithm() {
return encryptionAlgorithm;
}
public String getEncryptionKey() {
return encryptionKey;
}
/**
* Does this instance have encryption options?
* That is: is the algorithm non-null.
* @return true if there's an encryption algorithm.
*/
public boolean hasEncryptionAlgorithm() {
return StringUtils.isNotEmpty(encryptionAlgorithm);
}
/**
* Does this instance have an encryption key?
* @return true if there's an encryption key.
*/
public boolean hasEncryptionKey() {
return StringUtils.isNotEmpty(encryptionKey);
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final EncryptionSecrets that = (EncryptionSecrets) o;
return Objects.equals(encryptionAlgorithm, that.encryptionAlgorithm)
&& Objects.equals(encryptionKey, that.encryptionKey);
}
@Override
public int hashCode() {
return Objects.hash(encryptionAlgorithm, encryptionKey);
}
/**
* Get the encryption method.
* @return the encryption method
*/
public S3AEncryptionMethods getEncryptionMethod() {
return encryptionMethod;
}
/**
* String function returns the encryption mode but not any other
* secrets.
* @return a string safe for logging.
*/
@Override
public String toString() {
return S3AEncryptionMethods.NONE.equals(encryptionMethod)
? "(no encryption)"
: encryptionMethod.getMethod();
}
}

View File

@ -0,0 +1,172 @@
/*
* 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.auth.delegation;
import java.io.IOException;
import java.net.URI;
import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.FULL_TOKEN_KIND;
/**
* Full credentials: they are simply passed as-is, rather than
* converted to a session.
* These aren't as secure; this class exists to (a) support deployments
* where there is not STS service and (b) validate the design of
* S3A DT support to support different managers.
*/
public class FullCredentialsTokenBinding extends
AbstractDelegationTokenBinding {
/**
* Wire name of this binding includes a version marker: {@value}.
*/
private static final String NAME = "FullCredentials/001";
public static final String FULL_TOKEN = "Full Delegation Token";
/**
* Long-lived AWS credentials.
*/
private MarshalledCredentials awsCredentials;
/**
* Origin of credentials.
*/
private String credentialOrigin;
/**
* Constructor, uses name of {@link #name} and token kind of
* {@link DelegationConstants#FULL_TOKEN_KIND}.
*
*/
public FullCredentialsTokenBinding() {
super(NAME, FULL_TOKEN_KIND);
}
@Override
protected void serviceStart() throws Exception {
super.serviceStart();
loadAWSCredentials();
}
/**
* Load the AWS credentials.
* @throws IOException failure
*/
private void loadAWSCredentials() throws IOException {
credentialOrigin = AbstractS3ATokenIdentifier.createDefaultOriginMessage();
Configuration conf = getConfig();
URI uri = getCanonicalUri();
// look for access keys to FS
S3xLoginHelper.Login secrets = S3AUtils.getAWSAccessKeys(uri, conf);
if (secrets.hasLogin()) {
awsCredentials = new MarshalledCredentials(
secrets.getUser(), secrets.getPassword(), "");
credentialOrigin += "; source = Hadoop configuration data";
} else {
// if there are none, look for the environment variables.
awsCredentials = MarshalledCredentialBinding.fromEnvironment(
System.getenv());
if (awsCredentials.isValid(
MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty)) {
// valid tokens, so mark as origin
credentialOrigin += "; source = Environment variables";
} else {
credentialOrigin = "no credentials in configuration or"
+ " environment variables";
}
}
awsCredentials.validate(credentialOrigin +": ",
MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty);
}
/**
* Serve up the credentials retrieved from configuration/environment in
* {@link #loadAWSCredentials()}.
* @return a credential provider for the unbonded instance.
* @throws IOException failure to load
*/
@Override
public AWSCredentialProviderList deployUnbonded() throws IOException {
requireServiceStarted();
return new AWSCredentialProviderList(
"Full Credentials Token Binding",
new MarshalledCredentialProvider(
FULL_TOKEN,
getFileSystem().getUri(),
getConfig(),
awsCredentials,
MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty));
}
/**
* Create a new delegation token.
*
* It's slightly inefficient to create a new one every time, but
* it avoids concurrency problems with managing any singleton.
* @param policy minimum policy to use, if known.
* @param encryptionSecrets encryption secrets.
* @return a DT identifier
* @throws IOException failure
*/
@Override
public AbstractS3ATokenIdentifier createTokenIdentifier(
final Optional<RoleModel.Policy> policy,
final EncryptionSecrets encryptionSecrets) throws IOException {
requireServiceStarted();
return new FullCredentialsTokenIdentifier(getCanonicalUri(),
getOwnerText(),
awsCredentials,
encryptionSecrets,
credentialOrigin);
}
@Override
public AWSCredentialProviderList bindToTokenIdentifier(
final AbstractS3ATokenIdentifier retrievedIdentifier)
throws IOException {
FullCredentialsTokenIdentifier tokenIdentifier =
convertTokenIdentifier(retrievedIdentifier,
FullCredentialsTokenIdentifier.class);
return new AWSCredentialProviderList(
"", new MarshalledCredentialProvider(
FULL_TOKEN,
getFileSystem().getUri(),
getConfig(),
tokenIdentifier.getMarshalledCredentials(),
MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty));
}
@Override
public AbstractS3ATokenIdentifier createEmptyIdentifier() {
return new FullCredentialsTokenIdentifier();
}
}

View File

@ -0,0 +1,50 @@
/*
* 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.auth.delegation;
import java.net.URI;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.io.Text;
/**
* The full credentials payload is the same of that for a session token, but
* a different token kind is used.
*
* Token kind is {@link DelegationConstants#FULL_TOKEN_KIND}.
*/
public class FullCredentialsTokenIdentifier extends SessionTokenIdentifier {
public FullCredentialsTokenIdentifier() {
super(DelegationConstants.FULL_TOKEN_KIND);
}
public FullCredentialsTokenIdentifier(final URI uri,
final Text owner,
final MarshalledCredentials marshalledCredentials,
final EncryptionSecrets encryptionSecrets,
String origin) {
super(DelegationConstants.FULL_TOKEN_KIND,
owner,
uri,
marshalledCredentials,
encryptionSecrets,
origin);
}
}

View File

@ -0,0 +1,176 @@
/*
* 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.auth.delegation;
import java.io.IOException;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import com.amazonaws.services.securitytoken.model.Credentials;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromSTSCredentials;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_ROLE_ARN;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.E_NO_SESSION_TOKENS_FOR_ROLE_BINDING;
/**
* Role Token support requests an explicit role and automatically restricts
* that role to the given policy of the binding.
* The session is locked down as much as possible.
*/
public class RoleTokenBinding extends SessionTokenBinding {
private static final Logger LOG = LoggerFactory.getLogger(
RoleTokenBinding.class);
private static final RoleModel MODEL = new RoleModel();
/**
* Wire name of this binding includes a version marker: {@value}.
*/
private static final String NAME = "RoleCredentials/001";
/**
* Error message when there is no Role ARN.
*/
@VisibleForTesting
public static final String E_NO_ARN =
"No role ARN defined in " + DELEGATION_TOKEN_ROLE_ARN;
public static final String COMPONENT = "Role Delegation Token";
/**
* Role ARN to use when requesting new tokens.
*/
private String roleArn;
/**
* Constructor.
* Name is {@link #name}; token kind is
* {@link DelegationConstants#ROLE_TOKEN_KIND}.
*/
public RoleTokenBinding() {
super(NAME, DelegationConstants.ROLE_TOKEN_KIND);
}
@Override
protected void serviceInit(final Configuration conf) throws Exception {
super.serviceInit(conf);
roleArn = getConfig().getTrimmed(DELEGATION_TOKEN_ROLE_ARN, "");
}
/**
* Returns a (wrapped) {@link MarshalledCredentialProvider} which
* requires the marshalled credentials to contain session secrets.
* @param retrievedIdentifier the incoming identifier.
* @return the provider chain.
* @throws IOException on failure
*/
@Override
public AWSCredentialProviderList bindToTokenIdentifier(
final AbstractS3ATokenIdentifier retrievedIdentifier)
throws IOException {
RoleTokenIdentifier tokenIdentifier =
convertTokenIdentifier(retrievedIdentifier,
RoleTokenIdentifier.class);
setTokenIdentifier(Optional.of(tokenIdentifier));
MarshalledCredentials marshalledCredentials
= tokenIdentifier.getMarshalledCredentials();
setExpirationDateTime(marshalledCredentials.getExpirationDateTime());
return new AWSCredentialProviderList(
"Role Token Binding",
new MarshalledCredentialProvider(
COMPONENT, getFileSystem().getUri(),
getConfig(),
marshalledCredentials,
MarshalledCredentials.CredentialTypeRequired.SessionOnly));
}
/**
* Create the Token Identifier.
* Looks for the option {@link DelegationConstants#DELEGATION_TOKEN_ROLE_ARN}
* in the config and fail if it is not set.
* @param policy the policy which will be used for the requested token.
* @param encryptionSecrets encryption secrets.
* @return the token.
* @throws IllegalArgumentException if there is no role defined.
* @throws IOException any problem acquiring the role.
*/
@Override
@Retries.RetryTranslated
public RoleTokenIdentifier createTokenIdentifier(
final Optional<RoleModel.Policy> policy,
final EncryptionSecrets encryptionSecrets) throws IOException {
requireServiceStarted();
Preconditions.checkState(!roleArn.isEmpty(), E_NO_ARN);
String policyJson = policy.isPresent() ?
MODEL.toJson(policy.get()) : "";
final STSClientFactory.STSClient client = prepareSTSClient()
.orElseThrow(() -> {
// we've come in on a parent binding, so fail fast
LOG.error("Cannot issue delegation tokens because the credential"
+ " providers listed in " + DELEGATION_TOKEN_CREDENTIALS_PROVIDER
+ " are returning session tokens");
return new DelegationTokenIOException(
E_NO_SESSION_TOKENS_FOR_ROLE_BINDING);
});
Credentials credentials = client
.requestRole(roleArn,
UUID.randomUUID().toString(),
policyJson,
getDuration(),
TimeUnit.SECONDS);
return new RoleTokenIdentifier(
getCanonicalUri(),
getOwnerText(),
fromSTSCredentials(credentials),
encryptionSecrets,
AbstractS3ATokenIdentifier.createDefaultOriginMessage()
+ " Role ARN=" + roleArn);
}
@Override
public RoleTokenIdentifier createEmptyIdentifier() {
return new RoleTokenIdentifier();
}
@Override
public String getDescription() {
return super.getDescription() + " Role ARN=" +
(roleArn.isEmpty() ? "(none)" : ('"' + roleArn +'"'));
}
@Override
protected String bindingName() {
return "Role";
}
}

View File

@ -16,30 +16,34 @@
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a;
package org.apache.hadoop.fs.s3a.auth.delegation;
import org.apache.hadoop.conf.Configuration;
import java.net.URI;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.io.Text;
/**
* Run the encryption tests against the Fast output stream.
* This verifies that both file writing paths can encrypt their data.
* Role token identifier.
* Token kind is {@link DelegationConstants#ROLE_TOKEN_KIND}
*/
public class RoleTokenIdentifier extends SessionTokenIdentifier {
public class ITestS3AEncryptionSSECBlockOutputStream
extends AbstractTestS3AEncryption {
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
conf.set(Constants.FAST_UPLOAD_BUFFER,
Constants.FAST_UPLOAD_BYTEBUFFER);
conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY,
"4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=");
return conf;
public RoleTokenIdentifier() {
super(DelegationConstants.ROLE_TOKEN_KIND);
}
@Override
protected S3AEncryptionMethods getSSEAlgorithm() {
return S3AEncryptionMethods.SSE_C;
public RoleTokenIdentifier(final URI uri,
final Text owner,
final MarshalledCredentials marshalledCredentials,
final EncryptionSecrets encryptionSecrets,
final String origin) {
super(DelegationConstants.ROLE_TOKEN_KIND,
owner,
uri,
marshalledCredentials,
encryptionSecrets,
origin);
}
}

View File

@ -0,0 +1,685 @@
/*
* 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.auth.delegation;
import java.io.IOException;
import java.net.URI;
import java.util.EnumSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.commit.DurationInfo;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.service.ServiceOperations;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DEFAULT_DELEGATION_TOKEN_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DURATION_LOG_AT_INFO;
/**
* Support for creating a DT from a filesystem.
*
* Isolated from S3A for control and testability.
*
* The S3A Delegation Tokens are special in that the tokens are not directly
* used to authenticate with the AWS services.
* Instead they can session/role credentials requested off AWS on demand.
*
* The design is extensible in that different back-end bindings can be used
* to switch to different session creation mechanisms, or indeed, to any
* other authentication mechanism supported by an S3 service, provided it
* ultimately accepts some form of AWS credentials for authentication through
* the AWS SDK. That is, if someone wants to wire this up to Kerberos, or
* OAuth2, this design should support them.
*
* URIs processed must be the canonical URIs for the service.
*/
@InterfaceAudience.Private
public class S3ADelegationTokens extends AbstractDTService {
private static final Logger LOG = LoggerFactory.getLogger(
S3ADelegationTokens.class);
@VisibleForTesting
static final String E_ALREADY_DEPLOYED
= "S3A Delegation tokens has already been bound/deployed";
public static final String E_DELEGATION_TOKENS_DISABLED
= "Delegation tokens are not enabled";
/**
* User who owns this FS; fixed at instantiation time, so that
* in calls to getDelegationToken() and similar, this user is the one whose
* credentials are involved.
*/
private final UserGroupInformation user;
/**
* Count of number of created tokens.
* For testing and diagnostics.
*/
private final AtomicInteger creationCount = new AtomicInteger(0);
/**
* Text value of this token service.
*/
private Text service;
/**
* Active Delegation token.
*/
private Optional<Token<AbstractS3ATokenIdentifier>> boundDT
= Optional.empty();
/**
* The DT decoded when this instance is created by bonding
* to an existing DT.
*/
private Optional<AbstractS3ATokenIdentifier> decodedIdentifier
= Optional.empty();
/**
* Dynamically loaded token binding; lifecycle matches this object.
*/
private AbstractDelegationTokenBinding tokenBinding;
/**
* List of cred providers; unset until {@link #bindToDelegationToken(Token)}.
*/
private Optional<AWSCredentialProviderList> credentialProviders
= Optional.empty();
/**
* The access policies we want for operations.
* There's no attempt to ask for "admin" permissions here, e.g.
* those to manipulate S3Guard tables.
*/
protected static final EnumSet<AWSPolicyProvider.AccessLevel> ACCESS_POLICY
= EnumSet.of(
AWSPolicyProvider.AccessLevel.READ,
AWSPolicyProvider.AccessLevel.WRITE);
/**
* Statistics for the owner FS.
*/
private S3AInstrumentation.DelegationTokenStatistics stats;
/**
* Name of the token binding as extracted from token kind; used for
* logging.
*/
private String tokenBindingName = "";
/**
* Instantiate.
*/
public S3ADelegationTokens() throws IOException {
super("S3ADelegationTokens");
user = UserGroupInformation.getCurrentUser();
}
@Override
public void bindToFileSystem(final URI uri, final S3AFileSystem fs)
throws IOException {
super.bindToFileSystem(uri, fs);
service = getTokenService(getCanonicalUri());
stats = fs.getInstrumentation().newDelegationTokenStatistics();
}
/**
* Init the service.
* This identifies the token binding class to use and creates, initializes
* and starts it.
* Will raise an exception if delegation tokens are not enabled.
* @param conf configuration
* @throws Exception any failure to start up
*/
@Override
protected void serviceInit(final Configuration conf) throws Exception {
super.serviceInit(conf);
checkState(hasDelegationTokenBinding(conf),
E_DELEGATION_TOKENS_DISABLED);
Class<? extends AbstractDelegationTokenBinding> binding = conf.getClass(
DelegationConstants.DELEGATION_TOKEN_BINDING,
SessionTokenBinding.class,
AbstractDelegationTokenBinding.class);
tokenBinding = binding.newInstance();
tokenBinding.bindToFileSystem(getCanonicalUri(), getFileSystem());
tokenBinding.init(conf);
tokenBindingName = tokenBinding.getKind().toString();
LOG.info("Filesystem {} is using delegation tokens of kind {}",
getCanonicalUri(), tokenBindingName);
}
/**
* Service startup includes binding to any delegation token, and
* deploying unbounded if there is none.
* It is after this that token operations can be used.
* @throws Exception any failure
*/
@Override
protected void serviceStart() throws Exception {
super.serviceStart();
tokenBinding.start();
bindToAnyDelegationToken();
LOG.info("S3A Delegation support token {} with {}",
identifierToString(),
tokenBinding.getDescription());
}
/**
* Get the identifier as a string, or "(none)".
* @return a string value for logs etc.
*/
private String identifierToString() {
return decodedIdentifier.map(Objects::toString)
.orElse("(none)");
}
/**
* Stop the token binding.
* @throws Exception on any failure
*/
@SuppressWarnings("ThrowableNotThrown")
@Override
protected void serviceStop() throws Exception {
LOG.debug("Stopping delegation tokens");
try {
super.serviceStop();
} finally {
ServiceOperations.stopQuietly(LOG, tokenBinding);
}
}
/**
* Perform the unbonded deployment operations.
* Create the AWS credential provider chain to use
* when talking to AWS when there is no delegation token to work with.
* authenticating this client with AWS services, and saves it
* to {@link #credentialProviders}
*
* @throws IOException any failure.
*/
private void deployUnbonded()
throws IOException {
requireServiceStarted();
checkState(!isBoundToDT(),
"Already Bound to a delegation token");
LOG.info("No delegation tokens present: using direct authentication");
credentialProviders = Optional.of(tokenBinding.deployUnbonded());
}
/**
* Attempt to bind to any existing DT, including unmarshalling its contents
* and creating the AWS credential provider used to authenticate
* the client.
*
* If successful:
* <ol>
* <li>{@link #boundDT} is set to the retrieved token.</li>
* <li>{@link #decodedIdentifier} is set to the extracted identifier.</li>
* <li>{@link #credentialProviders} is set to the credential
* provider(s) returned by the token binding.</li>
* </ol>
* If unsuccessful, {@link #deployUnbonded()} is called for the
* unbonded codepath instead, which will set
* {@link #credentialProviders} to its value.
*
* This means after this call (and only after) the token operations
* can be invoked.
*
* This method is called from {@link #serviceStart()}, so a check on
* the service state can be used to check things; the state model
* prevents re-entrant calls.
* @throws IOException selection/extraction/validation failure.
*/
private void bindToAnyDelegationToken() throws IOException {
checkState(!credentialProviders.isPresent(), E_ALREADY_DEPLOYED);
Token<AbstractS3ATokenIdentifier> token = selectTokenFromFSOwner();
if (token != null) {
bindToDelegationToken(token);
} else {
deployUnbonded();
}
if (credentialProviders.get().size() == 0) {
throw new DelegationTokenIOException("No AWS credential providers"
+ " created by Delegation Token Binding "
+ tokenBinding.getName());
}
}
/**
* This is a test-only back door which resets the state and binds to
* a token again.
* This allows an instance of this class to be bonded to a DT after being
* started, so avoids the need to have the token in the current user
* credentials. It is package scoped so as to only be usable in tests
* in the same package.
*
* Yes, this is ugly, but there is no obvious/easy way to test token
* binding without Kerberos getting involved.
* @param token token to decode and bind to.
* @throws IOException selection/extraction/validation failure.
*/
@VisibleForTesting
void resetTokenBindingToDT(final Token<AbstractS3ATokenIdentifier> token)
throws IOException{
credentialProviders = Optional.empty();
bindToDelegationToken(token);
}
/**
* Bind to a delegation token retrieved for this filesystem.
* Extract the secrets from the token and set internal fields
* to the values.
* <ol>
* <li>{@link #boundDT} is set to {@code token}.</li>
* <li>{@link #decodedIdentifier} is set to the extracted identifier.</li>
* <li>{@link #credentialProviders} is set to the credential
* provider(s) returned by the token binding.</li>
* </ol>
* @param token token to decode and bind to.
* @throws IOException selection/extraction/validation failure.
*/
@VisibleForTesting
public void bindToDelegationToken(
final Token<AbstractS3ATokenIdentifier> token)
throws IOException {
checkState(!credentialProviders.isPresent(), E_ALREADY_DEPLOYED);
boundDT = Optional.of(token);
AbstractS3ATokenIdentifier dti = extractIdentifier(token);
LOG.info("Using delegation token {}", dti);
decodedIdentifier = Optional.of(dti);
try (DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
"Creating Delegation Token")) {
// extract the credential providers.
credentialProviders = Optional.of(
tokenBinding.bindToTokenIdentifier(dti));
}
}
/**
* Predicate: is there a bound DT?
* @return true if there's a value in {@link #boundDT}.
*/
public boolean isBoundToDT() {
return boundDT.isPresent();
}
/**
* Get any bound DT.
* @return a delegation token if this instance was bound to it.
*/
public Optional<Token<AbstractS3ATokenIdentifier>> getBoundDT() {
return boundDT;
}
/**
* Predicate: will this binding issue a DT if requested
* in a call to {@link #getBoundOrNewDT(EncryptionSecrets)}?
* That is: should the filesystem declare that it is issuing
* delegation tokens?
* @return a declaration of what will happen when asked for a token.
*/
public TokenIssuingPolicy getTokenIssuingPolicy() {
return isBoundToDT()
? TokenIssuingPolicy.ReturnExistingToken
: tokenBinding.getTokenIssuingPolicy();
}
/**
* Get any bound DT or create a new one.
* @return a delegation token.
* @throws IOException if one cannot be created
* @param encryptionSecrets encryption secrets for any new token.
*/
@SuppressWarnings("OptionalGetWithoutIsPresent")
public Token<AbstractS3ATokenIdentifier> getBoundOrNewDT(
final EncryptionSecrets encryptionSecrets)
throws IOException {
LOG.debug("Delegation token requested");
if (isBoundToDT()) {
// the FS was created on startup with a token, so return it.
LOG.debug("Returning current token");
return getBoundDT().get();
} else {
// not bound to a token, so create a new one.
// issued DTs are not cached so that long-lived filesystems can
// reliably issue session/role tokens.
return createDelegationToken(encryptionSecrets);
}
}
/**
* How many delegation tokens have been issued?
* @return the number times {@link #createDelegationToken(EncryptionSecrets)}
* returned a token.
*/
public int getCreationCount() {
return creationCount.get();
}
/**
* Create a delegation token for the user.
* This will only be called if a new DT is needed, that is: the
* filesystem has been deployed unbonded.
* @param encryptionSecrets encryption secrets for the token.
* @return the token
* @throws IOException if one cannot be created
*/
@VisibleForTesting
public Token<AbstractS3ATokenIdentifier> createDelegationToken(
final EncryptionSecrets encryptionSecrets) throws IOException {
requireServiceStarted();
checkArgument(encryptionSecrets != null,
"Null encryption secrets");
// this isn't done in in advance as it needs S3Guard initialized in the
// filesystem before it can generate complete policies.
List<RoleModel.Statement> statements = getFileSystem()
.listAWSPolicyRules(ACCESS_POLICY);
Optional<RoleModel.Policy> rolePolicy =
statements.isEmpty() ?
Optional.empty() : Optional.of(new RoleModel.Policy(statements));
try(DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
"Creating New Delegation Token", tokenBinding.getKind())) {
Token<AbstractS3ATokenIdentifier> token
= tokenBinding.createDelegationToken(rolePolicy, encryptionSecrets);
if (token != null) {
token.setService(service);
noteTokenCreated(token);
}
return token;
}
}
/**
* Note that a token has been created; increment counters and statistics.
* @param token token created
*/
private void noteTokenCreated(final Token<AbstractS3ATokenIdentifier> token) {
LOG.info("Created S3A Delegation Token: {}", token);
creationCount.incrementAndGet();
stats.tokenIssued();
}
/**
* Get the AWS credential provider.
* @return the DT credential provider
* @throws IOException failure to parse the DT
* @throws IllegalStateException if this instance is not bound to a DT
*/
public AWSCredentialProviderList getCredentialProviders()
throws IOException {
return credentialProviders.orElseThrow(
() -> new DelegationTokenIOException("Not yet bonded"));
}
/**
* Get the encryption secrets of the DT.
* non-empty iff service is started and was bound to a DT.
* @return any encryption settings propagated with the DT.
*/
public Optional<EncryptionSecrets> getEncryptionSecrets() {
return decodedIdentifier.map(
AbstractS3ATokenIdentifier::getEncryptionSecrets);
}
/**
* Get any decoded identifier from the bound DT; empty if not bound.
* @return the decoded identifier.
*/
public Optional<AbstractS3ATokenIdentifier> getDecodedIdentifier() {
return decodedIdentifier;
}
/**
* Get the service identifier of the owning FS.
* @return a service identifier to use when registering tokens
*/
public Text getService() {
return service;
}
/**
* The canonical name of the service.
* This can be used as the canonical service name for the FS.
* @return the canonicalized FS URI.
*/
public String getCanonicalServiceName() {
return getCanonicalUri().toString();
}
/**
* Find a token for the FS user and canonical filesystem URI.
* @return the token, or null if one cannot be found.
* @throws IOException on a failure to unmarshall the token.
*/
@VisibleForTesting
public Token<AbstractS3ATokenIdentifier> selectTokenFromFSOwner()
throws IOException {
return lookupToken(user.getCredentials(),
service,
tokenBinding.getKind());
}
/**
* Get the service identifier of a filesystem.
* This must be unique for (S3A, the FS URI)
* @param fsURI filesystem URI
* @return identifier to use.
*/
private static Text getTokenService(final URI fsURI) {
return getTokenService(fsURI.toString());
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"S3ADelegationTokens{");
sb.append("canonicalServiceURI=").append(getCanonicalUri());
sb.append("; owner=").append(user.getShortUserName());
sb.append("; isBoundToDT=").append(isBoundToDT());
sb.append("; token creation count=").append(getCreationCount());
sb.append("; tokenManager=").append(tokenBinding);
sb.append("; token=").append(identifierToString());
sb.append('}');
return sb.toString();
}
/**
* Get the kind of the issued tokens.
* @return token kind.
*/
public Text getTokenKind() {
return tokenBinding.getKind();
}
/**
* Get the service identifier of a filesystem URI.
* This must be unique for (S3a, the FS URI)
* @param fsURI filesystem URI as a string
* @return identifier to use.
*/
@VisibleForTesting
static Text getTokenService(final String fsURI) {
return new Text(fsURI);
}
/**
* From a token, get the session token identifier.
* @param token token to process
* @return the session token identifier
* @throws IOException failure to validate/read data encoded in identifier.
* @throws IllegalArgumentException if the token isn't an S3A session token
*/
public AbstractS3ATokenIdentifier extractIdentifier(
final Token<? extends AbstractS3ATokenIdentifier> token)
throws IOException {
checkArgument(token != null, "null token");
AbstractS3ATokenIdentifier identifier;
// harden up decode beyond that Token does itself
try {
identifier = token.decodeIdentifier();
} catch (RuntimeException e) {
Throwable cause = e.getCause();
if (cause != null) {
// its a wrapping around class instantiation.
throw new DelegationTokenIOException("Decoding S3A token " + cause,
cause);
} else {
throw e;
}
}
if (identifier == null) {
throw new DelegationTokenIOException("Failed to unmarshall token for "
+ getCanonicalUri());
}
identifier.validate();
return identifier;
}
/**
* Return a string for use in building up the User-Agent field, so
* get into the S3 access logs. Useful for diagnostics.
* Delegates to {{@link AbstractDelegationTokenBinding#getUserAgentField()}}
* for the current binding.
* @return a string for the S3 logs or "" for "nothing to add"
*/
public String getUserAgentField() {
return tokenBinding.getUserAgentField();
}
/**
* Look up a token from the credentials, verify it is of the correct
* kind.
* @param credentials credentials to look up.
* @param service service name
* @param kind token kind to look for
* @return the token or null if no suitable token was found
* @throws DelegationTokenIOException wrong token kind found
*/
@VisibleForTesting
public static Token<AbstractS3ATokenIdentifier> lookupToken(
final Credentials credentials,
final Text service,
final Text kind)
throws DelegationTokenIOException {
LOG.debug("Looking for token for service {} in credentials", service);
Token<?> token = credentials.getToken(service);
if (token != null) {
Text tokenKind = token.getKind();
LOG.debug("Found token of kind {}", tokenKind);
if (kind.equals(tokenKind)) {
// the Oauth implementation catches and logs here; this one
// throws the failure up.
return (Token<AbstractS3ATokenIdentifier>) token;
} else {
// there's a token for this URI, but its not the right DT kind
throw new DelegationTokenIOException(
DelegationTokenIOException.TOKEN_MISMATCH + ": expected token"
+ " for " + service
+ " of type " + kind
+ " but got a token of type " + tokenKind);
}
}
// A token for the service was not found
LOG.debug("No token for {} found", service);
return null;
}
/**
* Look up any token from the service; cast it to one of ours.
* @param credentials credentials
* @param service service to look up
* @return any token found or null if none was
* @throws ClassCastException if the token is of a wrong type.
*/
public static Token<AbstractS3ATokenIdentifier> lookupToken(
final Credentials credentials,
final Text service) {
return (Token<AbstractS3ATokenIdentifier>) credentials.getToken(service);
}
/**
* Look for any S3A token for the given FS service.
* @param credentials credentials to scan.
* @param uri the URI of the FS to look for
* @return the token or null if none was found
*/
public static Token<AbstractS3ATokenIdentifier> lookupS3ADelegationToken(
final Credentials credentials,
final URI uri) {
return lookupToken(credentials, getTokenService(uri.toString()));
}
/**
* Predicate: does this configuration enable delegation tokens?
* That is: is there any text in the option
* {@link DelegationConstants#DELEGATION_TOKEN_BINDING} ?
* @param conf configuration to examine
* @return true iff the trimmed configuration option is not empty.
*/
public static boolean hasDelegationTokenBinding(Configuration conf) {
return StringUtils.isNotEmpty(
conf.getTrimmed(DELEGATION_TOKEN_BINDING,
DEFAULT_DELEGATION_TOKEN_BINDING));
}
/**
* How will tokens be issued on request?
*
* The {@link #RequestNewToken} policy does not guarantee that a tokens
* can be created, only that an attempt will be made to request one.
* It may fail (wrong credential types, wrong role, etc).
*/
public enum TokenIssuingPolicy {
/** The existing token will be returned. */
ReturnExistingToken,
/** No tokens will be issued. */
NoTokensAvailable,
/** An attempt will be made to request a new DT. */
RequestNewToken
}
}

View File

@ -0,0 +1,80 @@
/**
* 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.auth.delegation;
import java.net.URI;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.DtFetcher;
import org.apache.hadoop.security.token.Token;
/**
* A DT fetcher for S3A.
* This is a copy-and-paste of
* {@code org.apache.hadoop.hdfs.HdfsDtFetcher}.
*
* It is only needed for the `hadoop dtutil` command.
*/
public class S3ADtFetcher implements DtFetcher {
private static final String SERVICE_NAME = Constants.FS_S3A;
private static final String FETCH_FAILED =
"Filesystem not generating Delegation Tokens";
/**
* Returns the service name for HDFS, which is also a valid URL prefix.
*/
public Text getServiceName() {
return new Text(SERVICE_NAME);
}
public boolean isTokenRequired() {
return UserGroupInformation.isSecurityEnabled();
}
/**
* Returns Token object via FileSystem, null if bad argument.
* @param conf - a Configuration object used with FileSystem.get()
* @param creds - a Credentials object to which token(s) will be added
* @param renewer - the renewer to send with the token request
* @param url - the URL to which the request is sent
* @return a Token, or null if fetch fails.
*/
public Token<?> addDelegationTokens(Configuration conf,
Credentials creds,
String renewer,
String url) throws Exception {
if (!url.startsWith(getServiceName().toString())) {
url = getServiceName().toString() + "://" + url;
}
FileSystem fs = FileSystem.get(URI.create(url), conf);
Token<?> token = fs.getDelegationToken(renewer);
if (token == null) {
throw new DelegationTokenIOException(FETCH_FAILED + ": " + url);
}
creds.addToken(token.getService(), token);
return token;
}
}

View File

@ -0,0 +1,421 @@
/*
* 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.auth.delegation;
import java.io.IOException;
import java.net.URI;
import java.time.OffsetDateTime;
import java.util.HashSet;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSSessionCredentials;
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Invoker.once;
import static org.apache.hadoop.fs.s3a.S3AUtils.STANDARD_AWS_PROVIDERS;
import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromAWSCredentials;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromSTSCredentials;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
/**
* The session token DT binding: creates an AWS session token
* for the DT, extracts and serves it up afterwards.
*/
public class SessionTokenBinding extends AbstractDelegationTokenBinding {
private static final Logger LOG = LoggerFactory.getLogger(
SessionTokenBinding.class);
/**
* Wire name of this binding: {@value}.
*/
private static final String NAME = "SessionTokens/001";
/**
* A message added to the standard origin string when the DT is
* built from session credentials passed in.
*/
@VisibleForTesting
public static final String CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN
= "Existing session credentials converted to Delegation Token";
public static final String SESSION_TOKEN
= "Session Delegation Token";
/** Invoker for STS calls. */
private Invoker invoker;
/**
* Has an attempt to initialize STS been attempted?
*/
private final AtomicBoolean stsInitAttempted = new AtomicBoolean(false);
/** The STS client; created in startup if the parental credentials permit. */
@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
private Optional<STSClientFactory.STSClient> stsClient = Optional.empty();
/**
* Duration of session in seconds.
*/
private long duration;
/**
* Flag to indicate that the auth chain provides session credentials.
* If true it means that STS cannot be used (and stsClient is null).
*/
private boolean hasSessionCreds;
/**
* The auth chain for the parent options.
*/
private AWSCredentialProviderList parentAuthChain;
/**
* Has a log message about forwarding credentials been printed yet?
*/
private final AtomicBoolean forwardMessageLogged = new AtomicBoolean(false);
/** STS endpoint. */
private String endpoint;
/** STS region. */
private String region;
/**
* Expiration date time as passed in from source.
* If unset, either we are unbound, or the token which came in does not
* know its expiry.
*/
private Optional<OffsetDateTime> expirationDateTime;
/**
* Token identifier bound to.
*/
private Optional<SessionTokenIdentifier> tokenIdentifier = Optional.empty();
/** Constructor for reflection. */
public SessionTokenBinding() {
this(NAME, SESSION_TOKEN_KIND);
}
/**
* Constructor for subclasses.
* @param name binding name.
* @param kind token kind.
*/
protected SessionTokenBinding(final String name,
final Text kind) {
super(name, kind);
}
/**
* Service start will read in all configuration options
* then build that client.
*/
@Override
protected void serviceStart() throws Exception {
super.serviceStart();
Configuration conf = getConfig();
duration = conf.getTimeDuration(DELEGATION_TOKEN_DURATION,
DEFAULT_DELEGATION_TOKEN_DURATION,
TimeUnit.SECONDS);
endpoint = conf.getTrimmed(DELEGATION_TOKEN_ENDPOINT,
DEFAULT_DELEGATION_TOKEN_ENDPOINT);
region = conf.getTrimmed(DELEGATION_TOKEN_REGION,
DEFAULT_DELEGATION_TOKEN_REGION);
// create the provider set for session credentials.
parentAuthChain = buildAWSProviderList(
getCanonicalUri(),
conf,
AWS_CREDENTIALS_PROVIDER,
STANDARD_AWS_PROVIDERS,
new HashSet<>());
}
@Override
protected void serviceStop() throws Exception {
super.serviceStop();
// this is here to keep findbugs quiet, even though nothing
// can safely invoke stsClient as we are shut down.
synchronized (this) {
this.stsClient.ifPresent(IOUtils::closeStream);
this.stsClient = Optional.empty();
}
}
/**
* Return an unbonded provider chain.
* @return the auth chain built from the assumed role credentials
* @throws IOException any failure.
*/
@Override
public AWSCredentialProviderList deployUnbonded()
throws IOException {
requireServiceStarted();
return parentAuthChain;
}
/**
* Get the invoker for STS calls.
* @return the invoker
*/
protected Invoker getInvoker() {
return invoker;
}
/**
* Sets the field {@link #tokenIdentifier} to the extracted/cast
* session token identifier, and {@link #expirationDateTime} to
* any expiration passed in.
* @param retrievedIdentifier the unmarshalled data
* @return the provider list
* @throws IOException failure
*/
@Override
public AWSCredentialProviderList bindToTokenIdentifier(
final AbstractS3ATokenIdentifier retrievedIdentifier)
throws IOException {
final SessionTokenIdentifier identifier = convertTokenIdentifier(
retrievedIdentifier,
SessionTokenIdentifier.class);
setTokenIdentifier(Optional.of(identifier));
MarshalledCredentials marshalledCredentials
= identifier.getMarshalledCredentials();
setExpirationDateTime(marshalledCredentials.getExpirationDateTime());
return new AWSCredentialProviderList(
"Session Token Binding",
new MarshalledCredentialProvider(
SESSION_TOKEN,
getFileSystem().getUri(),
getConfig(),
marshalledCredentials,
MarshalledCredentials.CredentialTypeRequired.SessionOnly));
}
@Override
public String getDescription() {
return String.format(
"%s token binding for user %s, " +
"with STS endpoint \"%s\", region \"%s\""
+ " and token duration %d:%02d",
bindingName(), getOwner().getShortUserName(), endpoint, region,
TimeUnit.SECONDS.toMinutes(duration),
duration % 60);
}
/**
* Get the role of this token; subclasses should override this
* for better logging.
* @return the role of this token
*/
protected String bindingName() {
return "Session";
}
/**
* UA field contains the UUID of the token if present.
* @return a string for the S3 logs.
*/
public String getUserAgentField() {
if (tokenIdentifier.isPresent()) {
return "; session ID " + tokenIdentifier.get().getUuid();
} else {
return "";
}
}
/**
* Attempt to init the STS connection, only does it once.
* If the AWS credential list to this service return session credentials
* then this method will return {@code empty()}; no attempt is
* made to connect to STS.
* Otherwise, the STS binding info will be looked up and an attempt
* made to connect to STS.
* Only one attempt will be made.
* @return any STS client created.
* @throws IOException any failure to bind to STS.
*/
private synchronized Optional<STSClientFactory.STSClient> maybeInitSTS()
throws IOException {
if (stsInitAttempted.getAndSet(true)) {
// whether or not it succeeded, the state of the STS client is what
// callers get after the first attempt.
return stsClient;
}
Configuration conf = getConfig();
URI uri = getCanonicalUri();
// Ask the owner for any session credentials which it already has
// so that it can just propagate them.
// this call may fail if there are no credentials on the auth
// chain.
// As no codepath (session propagation, STS creation) will work,
// throw this.
final AWSCredentials parentCredentials = once("get credentials",
"",
() -> parentAuthChain.getCredentials());
hasSessionCreds = parentCredentials instanceof AWSSessionCredentials;
if (!hasSessionCreds) {
LOG.info("Creating STS client for {}", getDescription());
invoker = new Invoker(new S3ARetryPolicy(conf), LOG_EVENT);
ClientConfiguration awsConf =
S3AUtils.createAwsConf(conf, uri.getHost());
AWSSecurityTokenService tokenService =
STSClientFactory.builder(parentAuthChain,
awsConf,
endpoint,
region)
.build();
stsClient = Optional.of(
STSClientFactory.createClientConnection(tokenService, invoker));
} else {
LOG.debug("Parent-provided session credentials will be propagated");
stsClient = Optional.empty();
}
return stsClient;
}
/**
* Log retries at debug.
*/
public static final Invoker.Retried LOG_EVENT =
(text, exception, retries, idempotent) -> {
LOG.info("{}: " + exception, text);
if (retries == 1) {
// stack on first attempt, to keep noise down
LOG.debug("{}: " + exception, text, exception);
}
};
/**
* Get the client to AWS STS.
* @return the STS client, when successfully inited.
*/
protected Optional<STSClientFactory.STSClient> prepareSTSClient()
throws IOException {
return maybeInitSTS();
}
/**
* Duration of sessions.
* @return duration in seconds.
*/
public long getDuration() {
return duration;
}
@Override
@Retries.RetryTranslated
public SessionTokenIdentifier createTokenIdentifier(
final Optional<RoleModel.Policy> policy,
final EncryptionSecrets encryptionSecrets) throws IOException {
requireServiceStarted();
final MarshalledCredentials marshalledCredentials;
String origin = AbstractS3ATokenIdentifier.createDefaultOriginMessage();
final Optional<STSClientFactory.STSClient> client = prepareSTSClient();
if (client.isPresent()) {
// this is the normal route: ask for a new STS token
marshalledCredentials = fromSTSCredentials(
client.get()
.requestSessionCredentials(duration, TimeUnit.SECONDS));
} else {
// get a new set of parental session credentials (pick up IAM refresh)
if (!forwardMessageLogged.getAndSet(true)) {
// warn caller on the first -and only the first- use.
LOG.warn("Forwarding existing session credentials to {}"
+ " -duration unknown", getCanonicalUri());
}
origin += " " + CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN;
final AWSCredentials awsCredentials
= parentAuthChain.getCredentials();
if (awsCredentials instanceof AWSSessionCredentials) {
marshalledCredentials = fromAWSCredentials(
(AWSSessionCredentials) awsCredentials);
} else {
throw new DelegationTokenIOException(
"AWS Authentication chain is no longer supplying session secrets");
}
}
return new SessionTokenIdentifier(getKind(),
getOwnerText(),
getCanonicalUri(),
marshalledCredentials,
encryptionSecrets,
origin);
}
@Override
public SessionTokenIdentifier createEmptyIdentifier() {
return new SessionTokenIdentifier();
}
/**
* Expiration date time as passed in from source.
* If unset, either we are unbound, or the token which came in does not
* know its expiry.
*/
protected Optional<OffsetDateTime> getExpirationDateTime() {
return expirationDateTime;
}
protected void setExpirationDateTime(
Optional<OffsetDateTime> expirationDateTime) {
this.expirationDateTime = expirationDateTime;
}
/**
* Token identifier bound to.
*/
protected Optional<SessionTokenIdentifier> getTokenIdentifier() {
return tokenIdentifier;
}
protected void setTokenIdentifier(Optional<SessionTokenIdentifier>
tokenIdentifier) {
this.tokenIdentifier = tokenIdentifier;
}
}

View File

@ -0,0 +1,146 @@
/*
* 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.auth.delegation;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.net.URI;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.io.Text;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.SESSION_TOKEN_KIND;
/**
* A token identifier which contains a set of AWS session credentials,
* credentials which will be valid until they expire.
*
* <b>Note 1:</b>
* There's a risk here that the reference to {@link MarshalledCredentials}
* may trigger a transitive load of AWS classes, a load which will
* fail if the aws SDK isn't on the classpath.
*
* <b>Note 2:</b>
* This class does support subclassing, but every subclass MUST declare itself
* to be of a different token kind.
* Otherwise the process for decoding tokens breaks.
*/
public class SessionTokenIdentifier extends
AbstractS3ATokenIdentifier {
/**
* Session credentials: initially empty but non-null.
*/
private MarshalledCredentials marshalledCredentials
= new MarshalledCredentials();
/**
* Constructor for service loader use.
* Created with the kind {@link DelegationConstants#SESSION_TOKEN_KIND}.
* Subclasses MUST NOT subclass this; they must provide their own
* token kind.
*/
public SessionTokenIdentifier() {
super(SESSION_TOKEN_KIND);
}
/**
* Constructor for subclasses.
* @param kind kind of token identifier, for storage in the
* token kind to implementation map.
*/
protected SessionTokenIdentifier(final Text kind) {
super(kind);
}
/**
* Constructor.
* @param kind token kind.
* @param owner token owner
* @param uri filesystem URI.
* @param marshalledCredentials credentials to marshall
* @param encryptionSecrets encryption secrets
* @param origin origin text for diagnostics.
*/
public SessionTokenIdentifier(
final Text kind,
final Text owner,
final URI uri,
final MarshalledCredentials marshalledCredentials,
final EncryptionSecrets encryptionSecrets,
final String origin) {
super(kind, uri, owner, origin, encryptionSecrets);
this.marshalledCredentials = marshalledCredentials;
}
/**
* Constructor.
* @param kind token kind.
* @param owner token owner
* @param uri filesystem URI.
*/
public SessionTokenIdentifier(final Text kind,
final Text owner,
final Text renewer,
final Text realUser,
final URI uri) {
super(kind, owner, renewer, realUser, uri);
}
@Override
public void write(final DataOutput out) throws IOException {
super.write(out);
marshalledCredentials.write(out);
}
@Override
public void readFields(final DataInput in)
throws IOException {
super.readFields(in);
marshalledCredentials.readFields(in);
}
/**
* Return the expiry time in seconds since 1970-01-01.
* @return the time when the AWS credentials expire.
*/
@Override
public long getExpiryTime() {
return marshalledCredentials.getExpiration();
}
/**
* Get the marshalled credentials.
* @return marshalled AWS credentials.
*/
public MarshalledCredentials getMarshalledCredentials() {
return marshalledCredentials;
}
/**
* Add the (sanitized) marshalled credentials to the string value.
* @return a string value for test assertions and debugging.
*/
@Override
public String toString() {
return super.toString()
+ "; " + marshalledCredentials.toString();
}
}

View File

@ -0,0 +1,34 @@
/*
* 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.
*/
/**
* Extensible delegation token support for the S3A connector.
*
* Goal: support multiple back end token issue/renewal services, from
* "pure client side" session tokens to full "Kerberos auth".
*
* It is intended for internal use only; any external implementation
* of {@link org.apache.hadoop.fs.s3a.auth.delegation.AbstractDelegationTokenBinding}
* must consider this API unstable and track changes as they happen.
*/
@InterfaceAudience.LimitedPrivate("authorization-subsystems")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.auth.delegation;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -18,8 +18,12 @@
/**
* Authentication and permissions support.
*
* Some of the classes in here are expected to be referred to in configuration
* files, so must not change their name. These will be explicitly identified.
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate("Authentication services")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.auth;

View File

@ -23,7 +23,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
/**
* A duration with logging of final state at info in the {@code close()} call.
* A duration with logging of final state at info or debug
* in the {@code close()} call.
* This allows it to be used in a try-with-resources clause, and have the
* duration automatically logged.
*/
@ -35,15 +36,41 @@ public class DurationInfo extends Duration
private final Logger log;
/**
* Create the duration text from a {@code String.format()} code call.
* Should the log be at INFO rather than DEBUG.
*/
private final boolean logAtInfo;
/**
* Create the duration text from a {@code String.format()} code call;
* log output at info level.
* @param log log to write to
* @param format format string
* @param args list of arguments
*/
public DurationInfo(Logger log, String format, Object... args) {
this(log, true, format, args);
}
/**
* Create the duration text from a {@code String.format()} code call
* and log either at info or debug.
* @param log log to write to
* @param logAtInfo should the log be at info, rather than debug
* @param format format string
* @param args list of arguments
*/
public DurationInfo(Logger log,
boolean logAtInfo,
String format,
Object... args) {
this.text = String.format(format, args);
this.log = log;
this.logAtInfo = logAtInfo;
if (logAtInfo) {
log.info("Starting: {}", text);
} else {
log.debug("Starting: {}", text);
}
}
@Override
@ -54,6 +81,10 @@ public String toString() {
@Override
public void close() {
finished();
log.info(this.toString());
if (logAtInfo) {
log.info("{}", this);
} else {
log.debug("{}", this);
}
}
}

View File

@ -273,8 +273,8 @@ private void checkChildPath(Path childPath) {
// If this dir's path has host (and thus scheme), so must its children
URI parentUri = path.toUri();
if (parentUri.getHost() != null) {
URI childUri = childPath.toUri();
if (parentUri.getHost() != null) {
Preconditions.checkNotNull(childUri.getHost(), "Expected non-null URI " +
"host: %s", childUri);
Preconditions.checkArgument(
@ -286,7 +286,8 @@ private void checkChildPath(Path childPath) {
}
Preconditions.checkArgument(!childPath.isRoot(),
"childPath cannot be the root path: %s", childPath);
Preconditions.checkArgument(childPath.getParent().equals(path),
Preconditions.checkArgument(parentUri.getPath().equals(
childPath.getParent().toUri().getPath()),
"childPath %s must be a child of %s", childPath, path);
}

View File

@ -26,6 +26,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
@ -70,6 +71,7 @@
import com.amazonaws.waiters.WaiterTimedOutException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -90,7 +92,9 @@
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.auth.RolePolicies;
import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.security.UserGroupInformation;
@ -98,6 +102,8 @@
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowAllDynamoDBOperations;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3GuardClientOperations;
import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
@ -185,7 +191,8 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DynamoDBMetadataStore implements MetadataStore {
public class DynamoDBMetadataStore implements MetadataStore,
AWSPolicyProvider {
public static final Logger LOG = LoggerFactory.getLogger(
DynamoDBMetadataStore.class);
@ -231,6 +238,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
private String region;
private Table table;
private String tableName;
private String tableArn;
private Configuration conf;
private String username;
@ -403,6 +411,8 @@ public void initialize(Configuration config) throws IOException {
region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
Preconditions.checkArgument(!StringUtils.isEmpty(region),
"No DynamoDB region configured");
// there's no URI here, which complicates life: you cannot
// create AWS providers here which require one.
credentials = createAWSCredentialProviderSet(null, conf);
dynamoDB = createDynamoDB(conf, region, null, credentials);
@ -1122,9 +1132,33 @@ public String toString() {
return getClass().getSimpleName() + '{'
+ "region=" + region
+ ", tableName=" + tableName
+ ", tableArn=" + tableArn
+ '}';
}
/**
* The administrative policy includes all DDB table operations;
* application access is restricted to those operations S3Guard operations
* require when working with data in a guarded bucket.
* @param access access level desired.
* @return a possibly empty list of statements.
*/
@Override
public List<RoleModel.Statement> listAWSPolicyRules(
final Set<AccessLevel> access) {
Preconditions.checkState(tableArn != null, "TableARN not known");
if (access.isEmpty()) {
return Collections.emptyList();
}
RoleModel.Statement stat;
if (access.contains(AccessLevel.ADMIN)) {
stat = allowAllDynamoDBOperations(tableArn);
} else {
stat = allowS3GuardClientOperations(tableArn);
}
return Lists.newArrayList(stat);
}
/**
* Create a table if it does not exist and wait for it to become active.
*
@ -1151,6 +1185,7 @@ void initTable() throws IOException {
LOG.debug("Binding to table {}", tableName);
TableDescription description = table.describe();
LOG.debug("Table state: {}", description);
tableArn = description.getTableArn();
final String status = description.getTableStatus();
switch (status) {
case "CREATING":

View File

@ -51,8 +51,10 @@
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.shell.CommandFormat;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
@ -1168,6 +1170,23 @@ public int run(String[] args, PrintStream out)
"none");
printOption(out, "\tInput seek policy", INPUT_FADVISE, INPUT_FADV_NORMAL);
// look at delegation token support
if (fs.getDelegationTokens().isPresent()) {
// DT is enabled
S3ADelegationTokens dtIntegration = fs.getDelegationTokens().get();
println(out, "Delegation Support enabled: token kind = %s",
dtIntegration.getTokenKind());
UserGroupInformation.AuthenticationMethod authenticationMethod
= UserGroupInformation.getCurrentUser().getAuthenticationMethod();
println(out, "Hadoop security mode: %s", authenticationMethod);
if (UserGroupInformation.isSecurityEnabled()) {
println(out,
"Warning: security is disabled; tokens will not be collected");
}
} else {
println(out, "Delegation token support is disabled");
}
if (usingS3Guard) {
if (commands.getOpt(UNGUARDED_FLAG)) {
throw badState("S3Guard is enabled for %s", fsUri);

View File

@ -136,8 +136,6 @@ static Login extractLoginDetails(URI name) {
/**
* Canonicalize the given URI.
*
* This strips out login information.
*
* @param uri the URI to canonicalize
* @param defaultPort default port to use in canonicalized URI if the input
* URI has no port and this value is greater than 0

View File

@ -0,0 +1,18 @@
#
# 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.
#
org.apache.hadoop.fs.s3a.auth.delegation.S3ADtFetcher

View File

@ -0,0 +1,20 @@
#
# 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.
#
org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenIdentifier
org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenIdentifier
org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenIdentifier

View File

@ -178,12 +178,14 @@ Here are the full set of configuration options.
<property>
<name>fs.s3a.assumed.role.credentials.provider</name>
<value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
<value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
com.amazonaws.auth.EnvironmentVariableCredentialsProvider
</value>
<description>
List of credential providers to authenticate with the STS endpoint and
retrieve short-lived role credentials.
Only used if AssumedRoleCredentialProvider is the AWS credential provider.
If unset, uses "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider".
Used by AssumedRoleCredentialProvider and the S3A Session Delegation Token
and S3A Role Delegation Token bindings.
</description>
</property>
```
@ -468,17 +470,69 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
```
### <a name="invalid_duration"></a> "Assume Role session duration should be in the range of 15min - 1Hr"
### <a name="invalid_duration"></a> `Member must have value greater than or equal to 900`
The value of `fs.s3a.assumed.role.session.duration` is out of range.
The value of `fs.s3a.assumed.role.session.duration` is too low.
```
java.lang.IllegalArgumentException: Assume Role session duration should be in the range of 15min
- 1Hr
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$Builder.withRoleSessionDurationSeconds(STSAssumeRoleSessionCredentialsProvider.java:437)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:86)
org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
1 validation error detected: Value '20' at 'durationSeconds' failed to satisfy constraint:
Member must have value greater than or equal to 900 (Service: AWSSecurityTokenService;
Status Code: 400; Error Code: ValidationError;
Request ID: b9a82403-d0a7-11e8-98ef-596679ee890d)
```
Fix: increase.
### <a name="duration_too_high"></a> Error "The requested DurationSeconds exceeds the MaxSessionDuration set for this role"
The value of `fs.s3a.assumed.role.session.duration` is too high.
```
org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
The requested DurationSeconds exceeds the MaxSessionDuration set for this role.
(Service: AWSSecurityTokenService; Status Code: 400;
Error Code: ValidationError; Request ID: 17875165-d0a7-11e8-b85f-d15a599a7f6d)
```
There are two solutions to this
* Decrease the duration value.
* Increase the duration of a role in the [AWS IAM Console](https://console.aws.amazon.com/iam/home#/roles).
### "Value '345600' at 'durationSeconds' failed to satisfy constraint: Member must have value less than or equal to 43200"
Irrespective of the maximum duration of a role, the AWS role API only permits callers to request
any role for up to 12h; attempting to use a larger number will fail.
```
Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
1 validation error detected:
Value '345600' at 'durationSeconds' failed to satisfy constraint:
Member must have value less than or equal to 43200
(Service: AWSSecurityTokenService;
Status Code: 400; Error Code:
ValidationError;
Request ID: dec1ca6b-d0aa-11e8-ac8c-4119b3ea9f7f)
```
For full sessions, the duration limit is 129600 seconds: 36h.
```
org.apache.hadoop.fs.s3a.AWSBadRequestException: request session credentials:
com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
1 validation error detected: Value '345600' at 'durationSeconds' failed to satisfy constraint:
Member must have value less than or equal to 129600
(Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
Request ID: a6e73d44-d0aa-11e8-95ed-c5bba29f0635)
```
For both these errors, the sole fix is to request a shorter duration
in `fs.s3a.assumed.role.session.duration`.
### <a name="malformed_policy"></a> `MalformedPolicyDocumentException` "The policy is not in the valid JSON format"
@ -487,7 +541,7 @@ The policy set in `fs.s3a.assumed.role.policy` is not valid according to the
AWS specification of Role Policies.
```
rg.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException:
The policy is not in the valid JSON format. (Service: AWSSecurityTokenService; Status Code: 400;
Error Code: MalformedPolicyDocument; Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c):
@ -508,36 +562,9 @@ Caused by: com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentExc
Error Code: MalformedPolicyDocument; Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:583)
... 19 more
```
### <a name="malformed_policy"></a> `MalformedPolicyDocumentException` "Syntax errors in policy"
### <a name="policy_syntax_error"></a> `MalformedPolicyDocumentException` "Syntax errors in policy"
The policy set in `fs.s3a.assumed.role.policy` is not valid JSON.
@ -564,31 +591,6 @@ Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:583)
... 19 more
```
@ -646,34 +648,6 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date,
(Service: AWSSecurityTokenService; Status Code: 400; Error Code: IncompleteSignature;
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:583)
... 25 more
```
### <a name="invalid_token"></a> `AccessDeniedException/InvalidClientTokenId`: "The security token included in the request is invalid"
@ -702,31 +676,6 @@ The security token included in the request is invalid.
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:583)
... 25 more
```
@ -740,7 +689,8 @@ match these constraints.
If set explicitly, it must be valid.
```
org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on
org.apache.hadoop.fs.s3a.AWSBadRequestException:
Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on
com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]*
@ -765,33 +715,6 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
(Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:135)
at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:124)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:583)
... 26 more
```
@ -818,24 +741,6 @@ Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied
S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4229)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4176)
at com.amazonaws.services.s3.AmazonS3Client.deleteObject(AmazonS3Client.java:2066)
at com.amazonaws.services.s3.AmazonS3Client.deleteObject(AmazonS3Client.java:2052)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$deleteObject$7(S3AFileSystem.java:1338)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:314)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:280)
at org.apache.hadoop.fs.s3a.S3AFileSystem.deleteObject(S3AFileSystem.java:1334)
at org.apache.hadoop.fs.s3a.S3AFileSystem.removeKeys(S3AFileSystem.java:1657)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerRename(S3AFileSystem.java:1046)
at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:851)
```
This is the policy restriction behaving as intended: the caller is trying to
@ -882,3 +787,63 @@ or just that this specific permission has been omitted.
If the role policy requested for the assumed role didn't ask for any DynamoDB
permissions, this is where all attempts to work with a S3Guarded bucket will
fail. Check the value of `fs.s3a.assumed.role.policy`
### Error `Unable to execute HTTP request`
This is a low-level networking error. Possible causes include:
* The endpoint set in `fs.s3a.assumed.role.sts.endpoint` is invalid.
* There are underlying network problems.
```
org.apache.hadoop.fs.s3a.AWSClientIOException: request session credentials:
com.amazonaws.SdkClientException:
Unable to execute HTTP request: null: Unable to execute HTTP request: null
at com.amazonaws.thirdparty.apache.http.impl.conn.DefaultRoutePlanner.determineRoute(DefaultRoutePlanner.java:88)
at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.determineRoute(InternalHttpClient.java:124)
at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:183)
at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
```
### <a name="credential_scope"></a> Error "Credential should be scoped to a valid region"
This is based on conflict between the values of `fs.s3a.assumed.role.sts.endpoint`
and `fs.s3a.assumed.role.sts.endpoint.region`
Two variants, "not '''"
Variant 1: `Credential should be scoped to a valid region, not 'us-west-1'` (or other string)
```
java.nio.file.AccessDeniedException: : request session credentials:
com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
Credential should be scoped to a valid region, not 'us-west-1'.
(Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch; Request ID: d9065cc4-e2b9-11e8-8b7b-f35cb8d7aea4):SignatureDoesNotMatch
```
One of:
* the value of `fs.s3a.assumed.role.sts.endpoint.region` is not a valid region
* the value of `fs.s3a.assumed.role.sts.endpoint.region` is not the signing
region of the endpoint set in `fs.s3a.assumed.role.sts.endpoint`
Variant 2: `Credential should be scoped to a valid region, not ''`
```
java.nio.file.AccessDeniedException: : request session credentials:
com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
Credential should be scoped to a valid region, not ''. (
Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch;
Request ID: bd3e5121-e2ac-11e8-a566-c1a4d66b6a16):SignatureDoesNotMatch
```
This should be intercepted earlier: an endpoint has been specified but
not a region.
There's special handling for the central `sts.amazonaws.com` region; when
that is declared as the value of `fs.s3a.assumed.role.sts.endpoint.region` then
there is no need to declare a region: whatever value it has is ignored.

View File

@ -0,0 +1,466 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# S3A Delegation Token Architecture
This is an architecture document to accompany
[Working with Delegation Tokens](delegation_tokens.html)
## Background: Delegation Tokens
Delegation Tokens, "DTs" are a common feature of Hadoop Services.
They are opaque byte arrays which can be issued by services like
HDFS, HBase, YARN, and which can be used to authenticate a request with
that service.
### Tokens are Issued
In a Kerberized cluster, they are issued by the service after the caller
has authenticated, and so that principal is trusted to be who they say they are.
The issued DT can therefore attest that whoever is including that token
on a request is authorized to act on behalf of that principal —for the
specific set of operations which the DT grants.
As an example, an HDFS DT can be requested by a user, included in the
launch context of a YARN application -say DistCp, and that launched application
can then talk to HDFS as if they were that user.
### Tokens are marshalled
Tokens are opaque byte arrays. They are contained within a `Token<T extends TokenIdentifier>`
class which includes an expiry time, the service identifier, and some other details.
`Token<>` instances can be serialized as a Hadoop Writable, or converted saved to/from a protobuf
format. This is how they are included in YARN application and container requests,
and elsewhere. They can even be saved to files through the `hadoop dt` command.
### Tokens can be unmarshalled
At the far end, tokens can be unmarshalled and converted into instances of
the java classes. This assumes that all the dependent classes are on the
classpath, obviously.
### Tokens can be used to authenticate callers
The Hadoop RPC layer and the web SPNEGO layer support tokens.
### Tokens can be renewed
DTs can be renewed by the specific principal declared at creation time as
"the renewer". In the example above, the YARN Resource Manager's principal
can be declared as the reviewer. Then, even while a token is attached
to a queued launch request in the RM, the RM can regularly request of HDFS
that the token is renewed.
There's an ultimate limit on how long tokens can be renewed for, but its
generally 72h or similar, so that medium-life jobs can access services
and data on behalf of a user.
### Tokens can be Revoked
When tokens are no longer needed, the service can be told to revoke a token.
Continuing the YARN example, after an application finishes the YARN RM
can revoke every token marshalled into the application launch request.
At which point there's no risk associated with that token being
compromised.
*This is all how "real" Hadoop tokens work*
The S3A Delegation Tokens are subtly different.
The S3A DTs actually include the AWS credentials within the token
data marshalled and shared across the cluster. The credentials can be one
of:
* The Full AWS (`fs.s3a.access.key`, `fs.s3a.secret.key`) login.
* A set of AWS session credentials
(`fs.s3a.access.key`, `fs.s3a.secret.key`, `fs.s3a.session.token`).
These credentials are obtained from the AWS Secure Token Service (STS) when the the token is issued.
* A set of AWS session credentials binding the user to a specific AWS IAM Role,
further restricted to only access the S3 bucket and matching S3Guard DynamoDB table.
Again, these credentials are requested when the token is issued.
*Tokens can be issued*
When an S3A Filesystem instance is asked to issue a token it can simply package
up the login secrets (The "Full" tokens), or talk to the AWS STS service
to get a set of session/assumed role credentials. These are marshalled within
the overall token, and then onwards to applications.
*Tokens can be marshalled*
The AWS secrets are held in a subclass of `org.apache.hadoop.security.token.TokenIdentifier`.
This class gets serialized to a byte array when the whole token is marshalled, and deserialized
when the token is loaded.
*Tokens can be used to authenticate callers*
The S3A FS does not hand the token to AWS services to authenticate the caller.
Instead it takes the AWS credentials included in the token identifier
and uses them to sign the requests.
*Tokens cannot be renewed*
The tokens contain the credentials; you cant use them to ask AWS for more.
For full credentials that is moot, but for the session and role credentials,
they will expire. At which point the application will be unable to
talk to the AWS infrastructure.
*Tokens cannot be revoked*
The AWS STS APIs don't let you revoke a single set of session credentials.
## Background: How Tokens are collected in MapReduce jobs
### `org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal()`
1. Calls `org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes()`
for the job submission dir on the cluster FS (i.e. `fs.defaultFS`).
1. Reads in the property `mapreduce.job.hdfs-servers` and extracts DTs from them,
1. Tells the `FileInputFormat` and `FileOutputFormat` subclasses of the job
to collect their source and dest FS tokens.
All token collection is via `TokenCache.obtainTokensForNamenodes()`
### `TokenCache.obtainTokensForNamenodes(Credentials, Path[], Configuration) `
1. Returns immediately if security is off.
1. Retrieves all the filesystems in the list of paths.
1. Retrieves a token from each unless it is in the list of filesystems in `mapreduce.job.hdfs-servers.token-renewal.exclude`
1. Merges in any DTs stored in the file referenced under: `mapreduce.job.credentials.binary`
1. Calls `FileSystem.collectDelegationTokens()`, which, if there isn't any token already in the credential list, issues and adds a new token. *There is no check to see if that existing credential has expired*.
### `FileInputFormat.listStatus(JobConf job): FileStatus[]`
Enumerates source paths in (`mapreduce.input.fileinputformat.inputdir`) ; uses `TokenCache.obtainTokensForNamenodes()`
to collate a token for all of these paths.
This operation is called by the public interface method `FileInputFormat.getSplits()`.
### `FileOutputFormat.checkOutputSpecs()`
Calls `getOutputPath(job)` and asks for the DTs of that output path FS.
## Architecture of the S3A Delegation Token Support
1. The S3A FS client has the ability to be configured with a delegation
token binding, the "DT Binding", a class declared in the option `fs.s3a.delegation.token.binding`.
1. If set, when a filesystem is instantiated it asks the DT binding for its list of AWS credential providers.
(the list in `fs.s3a.aws.credentials.provider` are only used if the DT binding wishes to).
1. The DT binding scans for the current principal (`UGI.getCurrentUser()`/"the Owner") to see if they
have any token in their credential cache whose service name matches the URI of the filesystem.
1. If one is found, it is unmarshalled and then used to authenticate the caller via
some AWS Credential provider returned to the S3A FileSystem instance.
1. If none is found, the Filesystem is considered to have been deployed "Unbonded".
The DT binding has to return a list of the AWS credential providers to use.
When requests are made of AWS services, the created credential provider(s) are
used to sign requests.
When the filesystem is asked for a delegation token, the
DT binding will generate a token identifier containing the marshalled tokens.
If the Filesystem was deployed with a DT, that is, it was deployed "bonded", that existing
DT is returned.
If it was deployed unbonded, the DT Binding is asked to create a new DT.
It is up to the binding what it includes in the token identifier, and how it obtains them.
This new token identifier is included in a token which has a "canonical service name" of
the URI of the filesystem (e.g "s3a://landsat-pds").
The issued/reissued token identifier can be marshalled and reused.
### class `org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens`
This joins up the S3A Filesystem with the pluggable DT binding classes.
One is instantiated in the S3A Filesystem instance if a DT Binding class
has been instantiated. If so, it is invoked for
* Building up the authentication chain during filesystem initialization.
* Determining if the FS should declare that it has a canonical name
(in `getCanonicalServiceName()`).
* When asked for a DT (in `getDelegationToken(String renewer)`).
The `S3ADelegationTokens` has the task of instantiating the actual DT binding,
which must be a subclass of `AbstractDelegationTokenBinding`.
All the DT bindings, and `S3ADelegationTokens` itself are subclasses of
`org.apache.hadoop.service.AbstractService`; they follow the YARN service lifecycle
of: create -> init -> start -> stop. This means that a DT binding, may, if it chooses,
start worker threads when the service is started (`serviceStart()`); it must
then stop them in the `serviceStop` method. (Anyone doing this must be aware
that the owner FS is not fully initialized in serviceStart: they must not
call into the Filesystem).
The actions of this class are
* Lookup of DTs associated with this S3A FS (scanning credentials, unmarshalling).
* initiating the DT binding in bound/unbound state.
* issuing DTs, either serving up the existing one, or requesting the DT Binding for
a new instance of `AbstractS3ATokenIdentifier` and then wrapping a hadoop token
around it.
* General logging, debugging, and metrics. Delegation token metrics are
collected in (`S3AInstrumentation.DelegationTokenStatistics`)
### class `org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier`
All tokens returned are a subclass of `AbstractS3ATokenIdentifier`.
This class contains the following fields:
```java
/** Canonical URI of the bucket. */
private URI uri;
/**
* Encryption secrets to also marshall with any credentials.
* Set during creation to ensure it is never null.
*/
private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
/**
* Timestamp of creation.
* This is set to the current time; it will be overridden when
* deserializing data.
*/
private long created = System.currentTimeMillis();
/**
* An origin string for diagnostics.
*/
private String origin = "";
/**
* This marshalled UUID can be used in testing to verify transmission,
* and reuse; as it is printed you can see what is happending too.
*/
private String uuid = UUID.randomUUID().toString();
```
The `uuid` field is used for equality tests and debugging; the `origin` and
`created` fields are also for diagnostics.
The `encryptionSecrets` structure enumerates the AWS encryption mechanism
of the filesystem instance, and any declared key. This allows
the client-side secret for SSE-C encryption to be passed to the filesystem,
or the key name for SSE-KMS.
*The encryption settings and secrets of the S3A filesystem on the client
are included in the DT, so can be used to encrypt/decrypt data in the cluster.*
### class `SessionTokenIdentifier` extends `AbstractS3ATokenIdentifier`
This holds session tokens, and it also gets used as a superclass of
the other token identifiers.
It adds a set of `MarshalledCredentials` containing the session secrets.
Every token/token identifier must have a unique *Kind*; this is how token
identifier deserializers are looked up. For Session Credentials, it is
`S3ADelegationToken/Session`. Subclasses *must* have a different token kind,
else the unmarshalling and binding mechanism will fail.
### classes `RoleTokenIdentifier` and `FullCredentialsTokenIdentifier`
These are subclasses of `SessionTokenIdentifier` with different token kinds,
needed for that token unmarshalling.
Their kinds are `S3ADelegationToken/Role` and `S3ADelegationToken/Full`
respectively.
Having different possible token bindings raises the risk that a job is submitted
with one binding and yet the cluster is expecting another binding.
Provided the configuration option `fs.s3a.delegation.token.binding` is not
marked as final in the `core-site.xml` file, the value of that binding
set in the job should propagate with the binding: the choice of provider
is automatic. A cluster can even mix bindings across jobs.
However if a core-site XML file declares a specific binding for a single bucket and
the job only had the generic `fs.s3a.delegation.token.binding`` binding,
then there will be a mismatch.
Each binding must be rigorous about checking the Kind of any found delegation
token and failing meaningfully here.
### class `MarshalledCredentials`
Can marshall a set of AWS credentials (access key, secret key, session token)
as a Hadoop Writable.
These can be given to an instance of class `MarshalledCredentialProvider`
and used to sign AWS RPC/REST API calls.
## DT Binding: `AbstractDelegationTokenBinding`
The plugin point for this design is the DT binding, which must be a subclass
of `org.apache.hadoop.fs.s3a.auth.delegation.AbstractDelegationTokenBinding`.
This class
* Returns the *Kind* of these tokens.
* declares whether tokens will actually be issued or not (the TokenIssuingPolicy).
* can issue a DT in
```java
public abstract AWSCredentialProviderList deployUnbonded()
throws IOException;
```
The S3A FS has been brought up with DTs enabled, but none have been found
for its service name. The DT binding is tasked with coming up with the
fallback list of AWS credential providers.
```java
public abstract AWSCredentialProviderList bindToTokenIdentifier(
AbstractS3ATokenIdentifier retrievedIdentifier)
throws IOException;
```
A DT for this FS instance been found. Bind to it and extract enough information
to authenticate with AWS. Return the list of providers to use.
```java
public abstract AbstractS3ATokenIdentifier createEmptyIdentifier();
```
Return an empty identifier.
```java
public abstract AbstractS3ATokenIdentifier createTokenIdentifier(
Optional<RoleModel.Policy> policy,
EncryptionSecrets encryptionSecrets)
```
This is the big one: creatw a new Token Identifier for this filesystem, one
which must include the encryption secrets, and which may make use of
the role policy.
## Token issuing
### How Full Delegation Tokens are issued.
If the client is only logged in with session credentials: fail.
Else: take the AWS access/secret key, store them in the MarshalledCredentials
in a new `FullCredentialsTokenIdentifier`, and return.
### How Session Delegation Tokens are issued.
If the client is only logged in with session credentials: return these.
This is taken from the Yahoo! patch: if a user is logged
in with a set of session credentials (including those from some 2FA login),
they just get wrapped up and passed in.
There's no clue as to how long they will last, so there's a warning printed.
If there is a full set of credentials, then an `SessionTokenBinding.maybeInitSTS()`
creates an STS client set up to communicate with the (configured) STS endpoint,
retrying with the same retry policy as the filesystem.
This client is then used to request a set of session credentials.
### How Role Delegation Tokens are issued.
If the client is only logged in with session credentials: fail.
We don't know whether this is a full user session or some role session,
and rather than pass in some potentially more powerful secrets with the job,
just fail.
Else: as with session delegation tokens, an STS client is created. This time
`assumeRole()` is invoked with the ARN of the role and an extra AWS role policy
set to restrict access to:
* CRUD access the specific bucket a token is being requested for
* CRUD access to the contents of any S3Guard DDB used (not admin rights though).
* access to all KMS keys (assumption: AWS KMS is where restrictions are set up)
*Example Generated Role Policy*
```json
{
"Version" : "2012-10-17",
"Statement" : [ {
"Sid" : "7",
"Effect" : "Allow",
"Action" : [ "s3:GetBucketLocation", "s3:ListBucket*" ],
"Resource" : "arn:aws:s3:::example-bucket"
}, {
"Sid" : "8",
"Effect" : "Allow",
"Action" : [ "s3:Get*", "s3:PutObject", "s3:DeleteObject", "s3:AbortMultipartUpload" ],
"Resource" : "arn:aws:s3:::example-bucket/*"
}, {
"Sid" : "1",
"Effect" : "Allow",
"Action" : [ "kms:Decrypt", "kms:GenerateDataKey" ],
"Resource" : "arn:aws:kms:*"
}, {
"Sid" : "9",
"Effect" : "Allow",
"Action" : [ "dynamodb:BatchGetItem", "dynamodb:BatchWriteItem", "dynamodb:DeleteItem", "dynamodb:DescribeTable", "dynamodb:GetItem", "dynamodb:PutItem", "dynamodb:Query", "dynamodb:UpdateItem" ],
"Resource" : "arn:aws:dynamodb:eu-west-1:980678866fff:table/example-bucket"
} ]
}
```
These permissions are sufficient for all operations the S3A client currently
performs on a bucket. If those requirements are expanded, these policies
may change.
## Testing.
Look in `org.apache.hadoop.fs.s3a.auth.delegation`
It's proven impossible to generate a full end-to-end test in an MR job.
1. MapReduce only collects DTs when kerberos is enabled in the cluster.
1. A Kerberized MiniYARN cluster refuses to start on a local file:// fs without the
native libraries, so it can set directory permissions.
1. A Kerberized MiniHDFS cluster and MiniYARN cluster refuse to talk to each
other reliably, at least in the week or so of trying.
The `ITestDelegatedMRJob` test works around this by using Mockito to mock
the actual YARN job submit operation in `org.apache.hadoop.mapreduce.protocol.ClientProtocol`.
The MR code does all the work of collecting tokens and attaching them to
the launch context, "submits" the job, which then immediately succeeds.
The job context is examined to verify that the source and destination filesystem
DTs were extracted.
To test beyond this requires a real Kerberized cluster, or someone able to fix
up Mini* clusters to run kerberized.

View File

@ -0,0 +1,870 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# Working with Delegation Tokens
<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
## <a name="introduction"></a> Introducing S3A Delegation Tokens.
The S3A filesystem client supports `Hadoop Delegation Tokens`.
This allows YARN application like MapReduce, Distcp, Apache Flink and Apache Spark to
obtain credentials to access S3 buckets and pass them pass these credentials to
jobs/queries, so granting them access to the service with the same access
permissions as the user.
Three different token types are offered.
*Full Delegation Tokens:* include the full login values of `fs.s3a.access.key`
and `fs.s3a.secret.key` in the token, so the recipient has access to
the data as the submitting user, with unlimited duration.
These tokens do not involve communication with the AWS STS service, so
can be used with other S3 installations.
*Session Delegation Tokens:* These contain an "STS Session Token" requested by
the S3A client from the AWS STS service. They have a limited duration
so restrict how long an application can access AWS on behalf of a user.
Clients with this token have the full permissions of the user.
*Role Delegation Tokens:* These contain an "STS Session Token" requested by by the
STS "Assume Role" API, so grant the caller to interact with S3 as specific AWS
role, *with permissions restricted to purely accessing the S3 bucket
and associated S3Guard data*.
Role Delegation Tokens are the most powerful. By restricting the access rights
of the granted STS token, no process receiving the token may perform
any operations in the AWS infrastructure other than those for the S3 bucket,
and that restricted by the rights of the requested role ARN.
All three tokens also marshall the encryption settings: The encryption mechanism
to use and the KMS key ID or SSE-C client secret. This allows encryption
policy and secrets to be uploaded from the client to the services.
This document covers how to use these tokens. For details on the implementation
see [S3A Delegation Token Architecture](delegation_token_architecture.html).
## <a name="background"></a> Background: Hadoop Delegation Tokens.
A Hadoop Delegation Token are is a byte array of data which is submitted to
a Hadoop services as proof that the caller has the permissions to perform
the operation which it is requesting —
and which can be passed between applications to *delegate* those permission.
Tokens are opaque to clients, clients who simply get a byte array
of data which they must to provide to a service when required.
This normally contains encrypted data for use by the service.
The service, which holds the password to encrypt/decrypt this data,
can decrypt the byte array and read the contents,
knowing that it has not been tampered with, then
use the presence of a valid token as evidence the caller has
at least temporary permissions to perform the requested operation.
Tokens have a limited lifespan.
They may be renewed, with the client making an IPC/HTTP request of a renewer service.
This renewal service can also be executed on behalf of the caller by
some other Hadoop cluster services, such as the YARN Resource Manager.
After use, tokens may be revoked: this relies on services holding tables of
valid tokens, either in memory or, for any HA service, in Apache Zookeeper or
similar. Revoking tokens is used to clean up after jobs complete.
Delegation support is tightly integrated with YARN: requests to launch
containers and applications can include a list of delegation tokens to
pass along. These tokens are serialized with the request, saved to a file
on the node launching the container, and then loaded in to the credentials
of the active user. Normally the HDFS cluster is one of the tokens used here,
added to the credentials through a call to `FileSystem.getDelegationToken()`
(usually via `FileSystem.addDelegationTokens()`).
Delegation Tokens are also supported with applications such as Hive: a query
issued to a shared (long-lived) Hive cluster can include the delegation
tokens required to access specific filesystems *with the rights of the user
submitting the query*.
All these applications normally only retrieve delegation tokens when security
is enabled. This is why the cluster configuration needs to enable Kerberos.
Production Hadoop clusters need Kerberos for security anyway.
## <a name="s3a-delegation-tokens"></a> S3A Delegation Tokens.
S3A now supports delegation tokens, so allowing a caller to acquire tokens
from a local S3A Filesystem connector instance and pass them on to
applications to grant them equivalent or restricted access.
These S3A Delegation Tokens are special in that they do not contain
password-protected data opaque to clients; they contain the secrets needed
to access the relevant S3 buckets and associated services.
They are obtained by requesting a delegation token from the S3A filesystem client.
Issued token mey be included in job submissions, passed to running applications,
etc. This token is specific to an individual bucket; all buckets which a client
wishes to work with must have a separate delegation token issued.
S3A implements Delegation Tokens in its `org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens`
class, which then supports multiple "bindings" behind it, so supporting
different variants of S3A Delegation Tokens.
Because applications only collect Delegation Tokens in secure clusters,
It does mean that to be able to submit delegation tokens in transient
cloud-hosted Hadoop clusters, _these clusters must also have Kerberos enabled_.
### <a name="session-tokens"></a> S3A Session Delegation Tokens
A Session Delegation Token is created by asking the AWS
[Security Token Service](http://docs.aws.amazon.com/STS/latest/APIReference/Welcome.html)
to issue an AWS session password and identifier for a limited duration.
These AWS session credentials are valid until the end of that time period.
They are marshalled into the S3A Delegation Token.
Other S3A connectors can extract these credentials and use them to
talk to S3 and related services.
Issued tokens cannot be renewed or revoked.
See [GetSessionToken](http://docs.aws.amazon.com/STS/latest/APIReference/API_GetSessionToken.html)
for specifics details on the (current) token lifespan.
### <a name="role-tokens"></a> S3A Role Delegation Tokens
A Role Delegation Tokens is created by asking the AWS
[Security Token Service](http://docs.aws.amazon.com/STS/latest/APIReference/Welcome.html)
for set of "Assumed Role" credentials, with a AWS account specific role for a limited duration..
This role is restricted to only grant access the S3 bucket, the S3Guard table
and all KMS keys,
They are marshalled into the S3A Delegation Token.
Other S3A connectors can extract these credentials and use them to
talk to S3 and related services.
They may only work with the explicit AWS resources identified when the token was generated.
Issued tokens cannot be renewed or revoked.
### <a name="full-credentials"></a> S3A Full-Credential Delegation Tokens
Full Credential Delegation Tokens tokens contain the full AWS login details
(access key and secret key) needed to access a bucket.
They never expire, so are the equivalent of storing the AWS account credentials
in a Hadoop, Hive, Spark configuration or similar.
They differences are:
1. They are automatically passed from the client/user to the application.
A remote application can use them to access data on behalf of the user.
1. When a remote application destroys the filesystem connector instances and
tokens of a user, the secrets are destroyed too.
1. Secrets in the `AWS_` environment variables on the client will be picked up
and automatically propagated.
1. They do not use the AWS STS service, so may work against third-party implementations
of the S3 protocol.
## <a name="enabling "></a> Using S3A Delegation Tokens
A prerequisite to using S3A filesystem delegation tokens is to run with
Hadoop security enabled —which inevitably means with Kerberos.
Even though S3A delegation tokens do not use Kerberos, the code in
applications which fetch DTs is normally only executed when the cluster is
running in secure mode; somewhere where the `core-site.xml` configuration
sets `hadoop.security.authentication` to to `kerberos` or another valid
authentication mechanism.
* Without enabling security at this level, delegation tokens will not
be collected.*
Once Kerberos enabled, the process for acquiring tokens is as follows:
1. Enable Delegation token support by setting `fs.s3a.delegation.token.binding`
to the classname of the token binding to use.
to use.
1. Add any other binding-specific settings (STS endpoint, IAM role, etc.)
1. Make sure the settings are the same in the service as well as the client.
1. In the client, switch to using a [Hadoop Credential Provider](hadoop-project-dist/hadoop-common/CredentialProviderAPI.html)
for storing your local credentials, *with a local filesystem store
(`localjceks:` or `jcecks://file`), so as to keep the full secrets out of any
job configurations.
1. Execute the client from a Kerberos-authenticated account
application configured with the login credentials for an AWS account able to issue session tokens.
### Configuration Parameters
| **Key** | **Meaning** | **Default** |
| --- | --- | --- |
| `fs.s3a.delegation.token.binding` | delegation token binding class | `` |
### Warnings
##### Use Hadoop Credential Providers to keep secrets out of job configurations.
Hadoop MapReduce jobs copy their client-side configurations with the job.
If your AWS login secrets are set in an XML file then they are picked up
and passed in with the job, _even if delegation tokens are used to propagate
session or role secrets.
Spark-submit will take any credentials in the `spark-defaults.conf`file
and again, spread them across the cluster.
It wil also pick up any `AWS_` environment variables and convert them into
`fs.s3a.access.key`, `fs.s3a.secret.key` and `fs.s3a.session.key` configuration
options.
To guarantee that the secrets are not passed in, keep your secrets in
a [hadoop credential provider file on the local filesystem](index.html#hadoop_credential_providers").
Secrets stored here will not be propagated -the delegation tokens collected
during job submission will be the sole AWS secrets passed in.
##### Token Life
* S3A Delegation tokens cannot be renewed.
* S3A Delegation tokens cannot be revoked. It is possible for an administrator
to terminate *all AWS sessions using a specific role*
[from the AWS IAM console](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp_control-access_disable-perms.html),
if desired.
* The lifespan of Session Delegation Tokens are limited to those of AWS sessions,
maximum of 36 hours.
* The lifespan of a Role Delegation Token is limited to 1 hour by default;
a longer duration of up to 12 hours can be enabled in the AWS console for
the specific role being used.
* The lifespan of Full Delegation tokens is unlimited: the secret needs
to be reset in the AWS Admin console to revoke it.
##### Service Load on the AWS Secure Token Service
All delegation tokens are issued on a bucket-by-bucket basis: clients
must request a delegation token from every S3A filesystem to which it desires
access.
For Session and Role Delegation Tokens, this places load on the AWS STS service,
which may trigger throttling amongst all users within the same AWS account using
the same STS endpoint.
* In experiments, a few hundred requests per second are needed to trigger throttling,
so this is very unlikely to surface in production systems.
* The S3A filesystem connector retries all throttled requests to AWS services, including STS.
* Other S3 clients with use the AWS SDK will, if configured, also retry throttled requests.
Overall, the risk of triggering STS throttling appears low, and most applications
will recover from what is generally an intermittently used AWS service.
### <a name="enabling-session-tokens"></a> Enabling Session Delegation Tokens
For session tokens, set `fs.s3a.delegation.token.binding`
to `org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding`
| **Key** | **Value** |
| --- | --- |
| `fs.s3a.delegation.token.binding` | `org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding` |
There some further configuration options.
| **Key** | **Meaning** | **Default** |
| --- | --- | --- |
| `fs.s3a.assumed.role.session.duration` | Duration of delegation tokens | `1h` |
| `fs.s3a.assumed.role.sts.endpoint` | URL to service issuing tokens | (undefined) |
| `fs.s3a.assumed.role.sts.endpoint.region` | region for issued tokens | (undefined) |
The XML settings needed to enable session tokens are:
```xml
<property>
<name>fs.s3a.delegation.token.binding</name>
<value>org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding</value>
</property>
<property>
<name>fs.s3a.assumed.role.session.duration</name>
<value>1h</value>
</property>
```
1. If the application requesting a token has full AWS credentials for the
relevant bucket, then a new session token will be issued.
1. If the application requesting a token is itself authenticating with
a session delegation token, then the existing token will be forwarded.
The life of the token will not be extended.
1. If the application requesting a token does not have either of these,
the the tokens cannot be issued: the operation will fail with an error.
The endpoint for STS requests are set by the same configuration
property as for the `AssumedRole` credential provider and for Role Delegation
tokens.
```xml
<!-- Optional -->
<property>
<name>fs.s3a.assumed.role.sts.endpoint</name>
<value>sts.amazonaws.com</value>
</property>
<property>
<name>fs.s3a.assumed.role.sts.endpoint.region</name>
<value>us-west-1</value>
</property>
```
If the `fs.s3a.assumed.role.sts.endpoint` option is set, or set to something
other than the central `sts.amazonaws.com` endpoint, then the region property
*must* be set.
Both the Session and the Role Delegation Token bindings use the option
`fs.s3a.aws.credentials.provider` to define the credential providers
to authenticate to the AWS STS with.
Here is the effective list of providers if none are declared:
```xml
<property>
<name>fs.s3a.aws.credentials.provider</name>
<value>
org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
</value>
</property>
```
Not all these authentication mechanisms provide the full set of credentials
STS needs. The session token provider will simply forward any session credentials
it is authenticated with; the role token binding will fail.
#### Forwarding of existing AWS Session credentials.
When the AWS credentials supplied to the Session Delegation Token binding
through `fs.s3a.aws.credentials.provider` are themselves a set of
session credentials, generated delegation tokens with simply contain these
existing session credentials, a new set of credentials obtained from STS.
This is because the STS service does not let
callers authenticated with session/role credentials from requesting new sessions.
This feature is useful when generating tokens from an EC2 VM instance in one IAM
role and forwarding them over to VMs which are running in a different IAM role.
The tokens will grant the permissions of the original VM's IAM role.
The duration of the forwarded tokens will be exactly that of the current set of
tokens, which may be very limited in lifespan. A warning will appear
in the logs declaring this.
Note: Role Delegation tokens do not support this forwarding of session credentials,
because there's no way to explicitly change roles in the process.
### <a name="enabling-role-tokens"></a> Enabling Role Delegation Tokens
For role delegation tokens, set `fs.s3a.delegation.token.binding`
to `org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding`
| **Key** | **Value** |
| --- | --- |
| `fs.s3a.delegation.token.binding` | `org.apache.hadoop.fs.s3a.auth.delegation.SessionToRoleTokenBinding` |
There are some further configuration options:
| **Key** | **Meaning** | **Default** |
| --- | --- | --- |
| `fs.s3a.assumed.role.session.duration"` | Duration of delegation tokens | `1h` |
| `fs.s3a.assumed.role.arn` | ARN for role to request | (undefined) |
| `fs.s3a.assumed.role.sts.endpoint.region` | region for issued tokens | (undefined) |
The option `fs.s3a.assumed.role.arn` must be set to a role which the
user can assume. It must have permissions to access the bucket, any
associated S3Guard table and any KMS encryption keys. The actual
requested role will be this role, explicitly restricted to the specific
bucket and S3Guard table.
The XML settings needed to enable session tokens are:
```xml
<property>
<name>fs.s3a.delegation.token.binding</name>
<value>org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding</value>
</property>
<property>
<name>fs.s3a.assumed.role.arn</name>
<value>ARN of role to request</value>
<value>REQUIRED ARN</value>
</property>
<property>
<name>fs.s3a.assumed.role.session.duration</name>
<value>1h</value>
</property>
```
A JSON role policy for the role/session will automatically be generated which will
consist of
1. Full access to the S3 bucket for all operations used by the S3A client
(read, write, list, multipart operations, get bucket location, etc).
1. Full user access to any S3Guard DynamoDB table used by the bucket.
1. Full user access to KMS keys. This is to be able to decrypt any data
in the bucket encrypted with SSE-KMS, as well as encrypt new data if that
is the encryption policy.
If the client doesn't have S3Guard enabled, but the remote application does,
the issued role tokens will not have permission to access the S3Guard table.
### <a name="enabling-full-tokens"></a> Enabling Full Delegation Tokens
This passes the full credentials in, falling back to any session credentials
which were used to configure the S3A FileSystem instance.
For Full Credential Delegation tokens, set `fs.s3a.delegation.token.binding`
to `org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding`
| **Key** | **Value** |
| --- | --- |
| `fs.s3a.delegation.token.binding` | `org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding` |
There are no other configuration options.
```xml
<property>
<name>fs.s3a.delegation.token.binding</name>
<value>org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding</value>
</property>
```
Key points:
1. If the application requesting a token has full AWS credentials for the
relevant bucket, then a full credential token will be issued.
1. If the application requesting a token is itself authenticating with
a session delegation token, then the existing token will be forwarded.
The life of the token will not be extended.
1. If the application requesting a token does not have either of these,
the the tokens cannot be issued: the operation will fail with an error.
## <a name="managing_token_duration"></a> Managing the Delegation Tokens Duration
Full Credentials have an unlimited lifespan.
Session and role credentials have a lifespan defined by the duration
property `fs.s3a.assumed.role.session.duration`.
This can have a maximum value of "36h" for session delegation tokens.
For Role Delegation Tokens, the maximum duration of a token is
that of the role itself: 1h by default, though this can be changed to
12h [In the IAM Console](https://console.aws.amazon.com/iam/home#/roles),
or from the AWS CLI.
*Without increasing the duration of role, one hour is the maximum value;
the error message `The requested DurationSeconds exceeds the MaxSessionDuration set for this role`
is returned if the requested duration of a Role Delegation Token is greater
than that available for the role.
## <a name="testing"></a> Testing Delegation Token Support
The easiest way to test that delegation support is configured is to use
the `hdfs fetchdt` command, which can fetch tokens from S3A, Azure ABFS
and any other filesystem which can issue tokens, as well as HDFS itself.
This will fetch the token and save it to the named file (here, `tokens.bin`),
even if Kerberos is disabled.
```bash
# Fetch a token for the AWS landsat-pds bucket and save it to tokens.bin
$ hdfs fetchdt --webservice s3a://landsat-pds/ tokens.bin
```
If the command fails with `ERROR: Failed to fetch token` it means the
filesystem does not have delegation tokens enabled.
If it fails for other reasons, the likely causes are configuration and
possibly connectivity to the AWS STS Server.
Once collected, the token can be printed. This will show
the type of token, details about encryption and expiry, and the
host on which it was created.
```bash
$ bin/hdfs fetchdt --print tokens.bin
Token (S3ATokenIdentifier{S3ADelegationToken/Session; uri=s3a://landsat-pds;
timestamp=1541683947569; encryption=EncryptionSecrets{encryptionMethod=SSE_S3};
Created on vm1.local/192.168.99.1 at time 2018-11-08T13:32:26.381Z.};
Session credentials for user AAABWL expires Thu Nov 08 14:02:27 GMT 2018; (valid))
for s3a://landsat-pds
```
The "(valid)" annotation means that the AWS credentials are considered "valid":
there is both a username and a secret.
You can use the `s3guard bucket-info` command to see what the delegation
support for a specific bucket is.
If delegation support is enabled, it also prints the current
hadoop security level.
```bash
$ hadoop s3guard bucket-info s3a://landsat-pds/
Filesystem s3a://landsat-pds
Location: us-west-2
Filesystem s3a://landsat-pds is not using S3Guard
The "magic" committer is supported
S3A Client
Endpoint: fs.s3a.endpoint=s3.amazonaws.com
Encryption: fs.s3a.server-side-encryption-algorithm=none
Input seek policy: fs.s3a.experimental.input.fadvise=normal
Delegation Support enabled: token kind = S3ADelegationToken/Session
Hadoop security mode: SIMPLE
```
Although the S3A delegation tokens do not depend upon Kerberos,
MapReduce and other applications only request tokens from filesystems when
security is enabled in Hadoop.
## <a name="troubleshooting"></a> Troubleshooting S3A Delegation Tokens
The `hadoop s3guard bucket-info` command will print information about
the delegation state of a bucket.
Consult [troubleshooting Assumed Roles](assumed_roles.html#troubleshooting)
for details on AWS error messages related to AWS IAM roles.
The [cloudstore](https://github.com/steveloughran/cloudstore) module's StoreDiag
utility can also be used to explore delegation token support
### Submitted job cannot authenticate
There are many causes for this; delegation tokens add some more.
### Tokens are not issued
* This user is not `kinit`-ed in to Kerberos. Use `klist` and
`hadoop kdiag` to see the Kerberos authentication state of the logged in user.
* The filesystem instance on the client has not had a token binding set in
`fs.s3a.delegation.token.binding`, so does not attempt to issue any.
* The job submission is not aware that access to the specific S3 buckets
are required. Review the application's submission mechanism to determine
how to list source and destination paths. For example, for MapReduce,
tokens for the cluster filesystem (`fs.defaultFS`) and all filesystems
referenced as input and output paths will be queried for
delegation tokens.
For Apache Spark, the cluster filesystem and any filesystems listed in the
property `spark.yarn.access.hadoopFileSystems` are queried for delegation
tokens in secure clusters.
See [Running on Yarn](https://spark.apache.org/docs/latest/running-on-yarn.html).
### Error `No AWS login credentials`
The client does not have any valid credentials to request a token
from the Amazon STS service.
### Tokens Expire before job completes
The default duration of session and role tokens as set in
`fs.s3a.assumed.role.session.duration` is one hour, "1h".
For session tokens, this can be increased to any time up to 36 hours.
For role tokens, it can be increased up to 12 hours, *but only if
the role is configured in the AWS IAM Console to have a longer lifespan*.
### Error `DelegationTokenIOException: Token mismatch`
```
org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenIOException:
Token mismatch: expected token for s3a://example-bucket
of type S3ADelegationToken/Session but got a token of type S3ADelegationToken/Full
at org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupToken(S3ADelegationTokens.java:379)
at org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.selectTokenFromActiveUser(S3ADelegationTokens.java:300)
at org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.bindToExistingDT(S3ADelegationTokens.java:160)
at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:423)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:265)
```
The value of `fs.s3a.delegation.token.binding` is different in the remote
service than in the local client. As a result, the remote service
cannot use the token supplied by the client to authenticate.
Fix: reference the same token binding class at both ends.
### Warning `Forwarding existing session credentials`
This message is printed when an S3A filesystem instance has been asked
for a Session Delegation Token, and it is itself only authenticated with
a set of AWS session credentials (such as those issued by the IAM metadata
service).
The created token will contain these existing credentials, credentials which
can be used until the existing session expires.
The duration of this existing session is unknown: the message is warning
you that it may expire without warning.
### Error `Cannot issue S3A Role Delegation Tokens without full AWS credentials`
An S3A filesystem instance has been asked for a Role Delegation Token,
but the instance is only authenticated with session tokens.
This means that a set of role tokens cannot be requested.
Note: no attempt is made to convert the existing set of session tokens into
a delegation token, unlike the Session Delegation Tokens. This is because
the role of the current session (if any) is unknown.
## <a name="implementation"></a> Implementation Details
### <a name="architecture"></a> Architecture
Concepts:
1. The S3A FileSystem can create delegation tokens when requested.
1. These can be marshalled as per other Hadoop Delegation Tokens.
1. At the far end, they can be retrieved, unmarshalled and used to authenticate callers.
1. DT binding plugins can then use these directly, or, somehow,
manage authentication and token issue through other services
(for example: Kerberos)
1. Token Renewal and Revocation are not supported.
There's support for different back-end token bindings through the
`org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokenManager`
Every implementation of this must return a subclass of
`org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier`
when asked to create a delegation token; this subclass must be registered
in `META-INF/services/org.apache.hadoop.security.token.TokenIdentifier`
for unmarshalling.
This identifier must contain all information needed at the far end to
authenticate the caller with AWS services used by the S3A client: AWS S3 and
potentially AWS KMS (for SSE-KMS) and AWS DynamoDB (for S3Guard).
It must have its own unique *Token Kind*, to ensure that it can be distinguished
from the other token identifiers when tokens are being unmarshalled.
| Kind | Token class |
|------|--------------|
| `S3ADelegationToken/Full` | `org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenIdentifier` |
| `S3ADelegationToken/Session` | `org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenIdentifier`|
| `S3ADelegationToken/Role` | `org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenIdentifier` |
If implementing an external binding:
1. Follow the security requirements below.
1. Define a new token identifier; there is no requirement for the `S3ADelegationToken/`
prefix —but it is useful for debugging.
1. Token Renewal and Revocation is not integrated with the binding mechanism;
if the operations are supported, implementation is left as an exercise.
1. Be aware of the stability guarantees of the module "LimitedPrivate/Unstable".
### <a name="security"></a> Security
S3A DTs contain secrets valuable for a limited period (session secrets) or
long-lived secrets with no explicit time limit.
* The `toString()` operations on token identifiers MUST NOT print secrets; this
is needed to keep them out of logs.
* Secrets MUST NOT be logged, even at debug level.
* Prefer short-lived session secrets over long-term secrets.
* Try to restrict the permissions to what a client with the delegated token
may perform to those needed to access data in the S3 bucket. This potentially
includes a DynamoDB table, KMS access, etc.
* Implementations need to be resistant to attacks which pass in invalid data as
their token identifier: validate the types of the unmarshalled data; set limits
on the size of all strings and other arrays to read in, etc.
### <a name="resilience"></a> Resilience
Implementations need to handle transient failures of any remote authentication
service, and the risk of a large-cluster startup overloading it.
* All get/renew/cancel operations should be considered idempotent.
* And clients to repeat with backoff & jitter on recoverable connectivity failures.
* While failing fast on the unrecoverable failures (DNS, authentication).
### <a name="scalability"></a> Scalability limits of AWS STS service
There is currently no documented rate limit for token requests against the AWS
STS service.
We have two tests which attempt to generate enough requests for
delegation tokens that the AWS STS service will throttle requests for
tokens by that AWS account for that specific STS endpoint
(`ILoadTestRoleCredentials` and `ILoadTestSessionCredentials`).
In the initial results of these tests:
* A few hundred requests a second can be made before STS block the caller.
* The throttling does not last very long (seconds)
* Tt does not appear to affect any other STS endpoints.
If developers wish to experiment with these tests and provide more detailed
analysis, we would welcome this. Do bear in mind that all users of the
same AWS account in that region will be throttled. Your colleagues may
notice, especially if the applications they are running do not retry on
throttle responses from STS (it's not a common occurrence after all...).
## Implementing your own Delegation Token Binding
The DT binding mechanism is designed to be extensible: if you have an alternate
authentication mechanism, such as an S3-compatible object store with
Kerberos support —S3A Delegation tokens should support it.
*if it can't: that's a bug in the implementation which needs to be corrected*.
### Steps
1. Come up with a token "Kind"; a unique name for the delegation token identifier.
1. Implement a subclass of `AbstractS3ATokenIdentifier` which adds all information which
is marshalled from client to remote services. This must subclass the `Writable` methods to read
and write the data to a data stream: these subclasses must call the superclass methods first.
1. Add a resource `META-INF/services/org.apache.hadoop.security.token.TokenIdentifier`
1. And list in it, the classname of your new identifier.
1. Implement a subclass of `AbstractDelegationTokenBinding`
### Implementing `AbstractS3ATokenIdentifier`
Look at the other examples to see what to do; `SessionTokenIdentifier` does
most of the work.
Having a `toString()` method which is informative is ideal for the `hdfs creds`
command as well as debugging: *but do not print secrets*
*Important*: Add no references to any AWS SDK class, to
ensure it can be safely deserialized whenever the relevant token
identifier is examined. Best practise is: avoid any references to
classes which may not be on the classpath of core Hadoop services,
especially the YARN Resource Manager and Node Managers.
### `AWSCredentialProviderList deployUnbonded()`
1. Perform all initialization needed on an "unbonded" deployment to authenticate with the store.
1. Return a list of AWS Credential providers which can be used to authenticate the caller.
**Tip**: consider *not* doing all the checks to verify that DTs can be issued.
That can be postponed until a DT is issued -as in any deployments where a DT is not actually
needed, failing at this point is overkill. As an example, `RoleTokenBinding` cannot issue
DTs if it only has a set of session credentials, but it will deploy without them, so allowing
`hadoop fs` commands to work on an EC2 VM with IAM role credentials.
**Tip**: The class `org.apache.hadoop.fs.s3a.auth.MarshalledCredentials` holds a set of
marshalled credentials and so can be used within your own Token Identifier if you want
to include a set of full/session AWS credentials in your token identifier.
### `AWSCredentialProviderList bindToTokenIdentifier(AbstractS3ATokenIdentifier id)`
The identifier passed in will be the one for the current filesystem URI and of your token kind.
1. Use `convertTokenIdentifier` to cast it to your DT type, or fail with a meaningful `IOException`.
1. Extract the secrets needed to authenticate with the object store (or whatever service issues
object store credentials).
1. Return a list of AWS Credential providers which can be used to authenticate the caller with
the extracted secrets.
### `AbstractS3ATokenIdentifier createEmptyIdentifier()`
Return an empty instance of your token identifier.
### `AbstractS3ATokenIdentifier createTokenIdentifier(Optional<RoleModel.Policy> policy, EncryptionSecrets secrets)`
Create the delegation token.
If non-empty, the `policy` argument contains an AWS policy model to grant access to:
* The target S3 bucket.
* Any S3Guard DDB table it is bonded to.
* KMS key `"kms:GenerateDataKey` and `kms:Decrypt`permissions for all KMS keys.
This can be converted to a string and passed to the AWS `assumeRole` operation.
The `secrets` argument contains encryption policy and secrets:
this should be passed to the superclass constructor as is; it is retrieved and used
to set the encryption policy on the newly created filesystem.
*Tip*: Use `AbstractS3ATokenIdentifier.createDefaultOriginMessage()` to create an initial
message for the origin of the token —this is useful for diagnostics.
#### Token Renewal
There's no support in the design for token renewal; it would be very complex
to make it pluggable, and as all the bundled mechanisms don't support renewal,
untestable and unjustifiable.
Any token binding which wants to add renewal support will have to implement
it directly.
### Testing
Use the tests `org.apache.hadoop.fs.s3a.auth.delegation` as examples. You'll have to
copy and paste some of the test base classes over; `hadoop-common`'s test JAR is published
to Maven Central, but not the S3A one (a fear of leaking AWS credentials).
#### Unit Test `TestS3ADelegationTokenSupport`
This tests marshalling and unmarshalling of tokens identifiers.
*Test that every field is preserved.*
#### Integration Test `ITestSessionDelegationTokens`
Tests the lifecycle of session tokens.
#### Integration Test `ITestSessionDelegationInFileystem`.
This collects DTs from one filesystem, and uses that to create a new FS instance and
then perform filesystem operations. A miniKDC is instantiated
* Take care to remove all login secrets from the environment, so as to make sure that
the second instance is picking up the DT information.
* `UserGroupInformation.reset()` can be used to reset user secrets after every test
case (e.g. teardown), so that issued DTs from one test case do not contaminate the next.
* its subclass, `ITestRoleDelegationInFileystem` adds a check that the current credentials
in the DT cannot be used to access data on other buckets —that is, the active
session really is restricted to the target bucket.
#### Integration Test `ITestDelegatedMRJob`
It's not easy to bring up a YARN cluster with a secure HDFS and miniKDC controller in
test cases —this test, the closest there is to an end-to-end test,
uses mocking to mock the RPC calls to the YARN AM, and then verifies that the tokens
have been collected in the job context,
#### Load Test `ILoadTestSessionCredentials`
This attempts to collect many, many delegation tokens simultaneously and sees
what happens.
Worth doing if you have a new authentication service provider, or
implementing custom DT support.
Consider also something for going from DT to
AWS credentials if this is also implemented by your own service.
This is left as an exercise for the developer.
**Tip**: don't go overboard here, especially against AWS itself.

View File

@ -31,6 +31,8 @@ See also:
* [Committing work to S3 with the "S3A Committers"](./committers.html)
* [S3A Committers Architecture](./committer_architecture.html)
* [Working with IAM Assumed Roles](./assumed_roles.html)
* [S3A Delegation Token Support](./delegation_tokens.html)
* [S3A Delegation Token Architecture](delegation_token_architecture.html).
* [Testing](./testing.html)
## <a name="overview"></a> Overview
@ -338,15 +340,20 @@ on the hosts/processes where the work is executed.
### <a name="auth_providers"></a> Changing Authentication Providers
The standard way to authenticate is with an access key and secret key using the
properties in the configuration file.
The standard way to authenticate is with an access key and secret key set in
the Hadoop configuration files.
The S3A client follows the following authentication chain:
By default, the S3A client follows the following authentication chain:
1. The options `fs.s3a.access.key`, `fs.s3a.secret.key` and `fs.s3a.sesson.key
are looked for in the Hadoop XML configuration/Hadoop credential providers,
returning a set of session credentials if all three are defined.
1. The `fs.s3a.access.key` and `fs.s3a.secret.key` are looked for in the Hadoop
XML configuration.
XML configuration//Hadoop credential providers, returning a set of long-lived
credentials if they are defined.
1. The [AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment),
are then looked for.
are then looked for: these will return session or full credentials depending
on which values are set.
1. An attempt is made to query the Amazon EC2 Instance Metadata Service to
retrieve credentials published to EC2 VMs.
@ -362,13 +369,19 @@ 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:
All Hadoop `fs.s3a.` options used to store login details can all be secured
in [Hadoop credential providers](../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html);
this is advised as a more secure way to store valuable secrets.
There are a number of 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 |
| `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider<`| [Assumed Role credentials](assumed_roles.html) |
There are also many in the Amazon SDKs, in particular two which are automatically
set up in the authentication chain:
@ -483,10 +496,52 @@ 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.TemporaryAWSCredentialsProvider,
org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
com.amazonaws.auth.InstanceProfileCredentialsProvider
org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
</value>
<description>
Comma-separated class names of credential provider classes which implement
com.amazonaws.auth.AWSCredentialsProvider.
When S3A delegation tokens are not enabled, this list will be used
to directly authenticate with S3 and DynamoDB services.
When S3A Delegation tokens are enabled, depending upon the delegation
token binding it may be used
to communicate wih the STS endpoint to request session/role
credentials.
These are loaded and queried in sequence for a valid set of credentials.
Each listed class must implement one of the following means of
construction, which are attempted in order:
* a public constructor accepting java.net.URI and
org.apache.hadoop.conf.Configuration,
* a public constructor accepting org.apache.hadoop.conf.Configuration,
* a public static method named getInstance that accepts no
arguments and returns an instance of
com.amazonaws.auth.AWSCredentialsProvider, or
* a public 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
security and therefore is unsuitable for most use cases. It can be useful
for accessing public data sets without requiring AWS credentials.
If unspecified, then the default list of credential provider classes,
queried in sequence, is:
* org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider: looks
for session login secrets in the Hadoop configuration.
* org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
Uses the values of fs.s3a.access.key and fs.s3a.secret.key.
* com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
configuration of AWS access key ID and secret access key in
environment variables named AWS_ACCESS_KEY_ID and
AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
* com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
of instance profile credentials if running in an EC2 VM.
</description>
</property>
```
@ -501,9 +556,6 @@ and significantly damage your organisation.
1. Never commit your secrets into an SCM repository.
The [git secrets](https://github.com/awslabs/git-secrets) can help here.
1. Avoid using s3a URLs which have key and secret in the URL. This
is dangerous as the secrets leak into the logs.
1. Never include AWS credentials in bug reports, files attached to them,
or similar.
@ -524,20 +576,23 @@ The command line of any launched program is visible to all users on a Unix syste
management: a specific S3A connection can be made with a different assumed role
and permissions from the primary user account.
1. Consider a workflow in which usera and applications are issued with short-lived
1. Consider a workflow in which users and applications are issued with short-lived
session credentials, configuring S3A to use these through
the `TemporaryAWSCredentialsProvider`.
1. Have a secure process in place for cancelling and re-issuing credentials for
users and applications. Test it regularly by using it to refresh credentials.
1. In installations where Kerberos is enabled, [S3A Delegation Tokens](delegation_tokens.html)
can be used to acquire short-lived session/role credentials and then pass them
into the shared application. This can ensure that the long-lived secrets stay
on the local system.
When running in EC2, the IAM EC2 instance credential provider will automatically
obtain the credentials needed to access AWS services in the role the EC2 VM
was deployed as.
This credential provider is enabled in S3A by default.
This AWS credential provider is enabled in S3A by default.
The safest way to keep the AWS login keys a secret within Hadoop is to use
Hadoop Credentials.
## <a name="hadoop_credential_providers"></a>Storing secrets with Hadoop Credential Providers

View File

@ -481,10 +481,11 @@ value as "do not override the default").
### Testing Session Credentials
The test `TestS3ATemporaryCredentials` requests a set of temporary
credentials from the STS service, then uses them to authenticate with S3.
Some tests requests a session credentials and assumed role credentials from the
AWS Secure Token Service, then use them to authenticate with S3 either directly
or via delegation tokens.
If an S3 implementation does not support STS, then the functional test
If an S3 implementation does not support STS, then these functional test
cases must be disabled:
```xml
@ -492,18 +493,30 @@ cases must be disabled:
<name>test.fs.s3a.sts.enabled</name>
<value>false</value>
</property>
```
These tests request a temporary set of credentials from the STS service endpoint.
An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
An alternate endpoint may be defined in `fs.s3a.assumed.role.sts.endpoint`.
If this is set, a delegation token region must also be defined:
in `fs.s3a.assumed.role.sts.endpoint.region`.
This is useful not just for testing alternative infrastructures,
but to reduce latency on tests executed away from the central
service.
```xml
<property>
<name>test.fs.s3a.sts.endpoint</name>
<value>https://sts.example.org/</value>
<name>fs.s3a.delegation.token.endpoint</name>
<value>fs.s3a.assumed.role.sts.endpoint</value>
</property>
<property>
<name>fs.s3a.assumed.role.sts.endpoint.region</name>
<value>eu-west-2</value>
</property>
```
The default is ""; meaning "use the amazon default value".
The default is ""; meaning "use the amazon default endpoint" (`sts.amazonaws.com`).
Consult the [AWS documentation](https://docs.aws.amazon.com/general/latest/gr/rande.html#sts_region)
for the full list of locations.
## <a name="debugging"></a> Debugging Test failures
@ -1134,16 +1147,25 @@ This is not for use in production.
Tests for the AWS Assumed Role credential provider require an assumed
role to request.
If this role is not set, the tests which require it will be skipped.
If this role is not declared in `fs.s3a.assumed.role.arn`,
the tests which require it will be skipped.
To run the tests in `ITestAssumeRole`, you need:
The specific tests an Assumed Role ARN is required for are
- `ITestAssumeRole`.
- `ITestRoleDelegationTokens`.
- One of the parameterized test cases in `ITestDelegatedMRJob`.
To run these tests you need:
1. A role in your AWS account will full read and write access rights to
the S3 bucket used in the tests, and ideally DynamoDB, for S3Guard.
the S3 bucket used in the tests, DynamoDB, for S3Guard, and KMS for any
SSE-KMS tests.
If your bucket is set up by default to use S3Guard, the role must have access
to that service.
1. Your IAM User to have the permissions to adopt that role.
1. Your IAM User to have the permissions to "assume" that role.
1. The role ARN must be set in `fs.s3a.assumed.role.arn`.

View File

@ -235,7 +235,23 @@ As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
</property>
```
## <a name="access_denied"></a> `AccessDeniedException` "Access Denied"
## <a name="access_denied"></a> "The security token included in the request is invalid"
You are trying to use session/temporary credentials and the session token
supplied is considered invalid.
```
org.apache.hadoop.fs.s3a.AWSBadRequestException: initTable on bucket:
com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException:
The security token included in the request is invalid
(Service: AmazonDynamoDBv2; Status Code: 400; Error Code: UnrecognizedClientException)
```
This can surface if your configuration is setting the `fs.s3a.secret.key`,
`fs.s3a.access.key` and `fs.s3a.session.key` correctly, but the
AWS credential provider list set in `AWS_CREDENTIALS_PROVIDER` does not include
`org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider`.
### <a name="access_denied_unknown-ID"></a> AccessDeniedException "The AWS Access Key Id you provided does not exist in our records."

View File

@ -133,10 +133,10 @@ protected void writeThenReadFile(Path path, int len) throws IOException {
* Assert that an exception failed with a specific status code.
* @param e exception
* @param code expected status code
* @throws AWSS3IOException rethrown if the status code does not match.
* @throws AWSServiceIOException rethrown if the status code does not match.
*/
protected void assertStatusCode(AWSS3IOException e, int code)
throws AWSS3IOException {
protected void assertStatusCode(AWSServiceIOException e, int code)
throws AWSServiceIOException {
if (e.getStatusCode() != code) {
throw e;
}

View File

@ -1,50 +0,0 @@
/*
* 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 static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
/**
* Run the encryption tests against the Fast output stream.
* This verifies that both file writing paths can encrypt their data. This
* requires the SERVER_SIDE_ENCRYPTION_KEY to be set in auth-keys.xml for it
* to run.
*/
public class ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream
extends AbstractTestS3AEncryption {
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
if(StringUtils.isBlank(conf.get(Constants.SERVER_SIDE_ENCRYPTION_KEY))){
skip(Constants.SERVER_SIDE_ENCRYPTION_KEY+ " is not set for " +
S3AEncryptionMethods.SSE_KMS.getMethod());
}
conf.set(Constants.FAST_UPLOAD_BUFFER, Constants.FAST_UPLOAD_BYTEBUFFER);
return conf;
}
@Override
protected S3AEncryptionMethods getSSEAlgorithm() {
return S3AEncryptionMethods.SSE_KMS;
}
}

View File

@ -19,49 +19,89 @@
package org.apache.hadoop.fs.s3a;
import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.file.AccessDeniedException;
import java.time.Duration;
import java.time.OffsetDateTime;
import java.util.concurrent.TimeUnit;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
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.s3a.auth.STSClientFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.test.LambdaTestUtils;
import org.hamcrest.Matchers;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenIdentifier;
import org.apache.hadoop.fs.s3a.commit.DurationInfo;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.LambdaTestUtils;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeSessionTestsEnabled;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.requestSessionCredentials;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.unsetHadoopCredentialProviders;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromSTSCredentials;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.toAWSCredentials;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.assertCredentialsEqual;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
import static org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding.CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.hamcrest.Matchers.containsString;
/**
* 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.
*
* The property {@link Constants#ASSUMED_ROLE_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.
*/
public class ITestS3ATemporaryCredentials extends AbstractS3ATestBase {
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3ATemporaryCredentials.class);
private static final String PROVIDER_CLASS
private static final String TEMPORARY_AWS_CREDENTIALS
= TemporaryAWSCredentialsProvider.NAME;
private static final long TEST_FILE_SIZE = 1024;
public static final String STS_LONDON = "sts.eu-west-2.amazonaws.com";
public static final String EU_IRELAND = "eu-west-1";
private AWSCredentialProviderList credentials;
@Override
public void setup() throws Exception {
super.setup();
assumeSessionTestsEnabled(getConfiguration());
}
@Override
public void teardown() throws Exception {
S3AUtils.closeAutocloseables(LOG, credentials);
super.teardown();
}
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
conf.set(DELEGATION_TOKEN_BINDING,
DELEGATION_TOKEN_SESSION_BINDING);
return conf;
}
/**
* Test use of STS for requesting temporary credentials.
*
@ -75,9 +115,6 @@ public void teardown() throws Exception {
@Test
public void testSTS() throws IOException {
Configuration conf = getContract().getConf();
if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
skip("STS functional tests disabled");
}
S3AFileSystem testFS = getFileSystem();
credentials = testFS.shareCredentials("testSTS");
@ -86,18 +123,15 @@ public void testSTS() throws IOException {
conf,
bucket,
credentials,
conf.getTrimmed(TEST_STS_ENDPOINT, ""), "");
AWSSecurityTokenService stsClient = builder.build();
if (!conf.getTrimmed(TEST_STS_ENDPOINT, "").isEmpty()) {
LOG.debug("STS Endpoint ={}", conf.getTrimmed(TEST_STS_ENDPOINT, ""));
stsClient.setEndpoint(conf.getTrimmed(TEST_STS_ENDPOINT, ""));
}
GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
sessionTokenRequest.setDurationSeconds(900);
GetSessionTokenResult sessionTokenResult;
sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
Credentials sessionCreds = sessionTokenResult.getCredentials();
getStsEndpoint(conf),
getStsRegion(conf));
STSClientFactory.STSClient clientConnection =
STSClientFactory.createClientConnection(
builder.build(),
new Invoker(new S3ARetryPolicy(conf), Invoker.LOG_EVENT));
Credentials sessionCreds = clientConnection
.requestSessionCredentials(TEST_SESSION_TOKEN_DURATION_SECONDS,
TimeUnit.SECONDS);
// clone configuration so changes here do not affect the base FS.
Configuration conf2 = new Configuration(conf);
@ -106,11 +140,10 @@ public void testSTS() throws IOException {
S3AUtils.clearBucketOption(conf2, bucket, SECRET_KEY);
S3AUtils.clearBucketOption(conf2, bucket, SESSION_TOKEN);
conf2.set(ACCESS_KEY, sessionCreds.getAccessKeyId());
conf2.set(SECRET_KEY, sessionCreds.getSecretAccessKey());
conf2.set(SESSION_TOKEN, sessionCreds.getSessionToken());
MarshalledCredentials mc = fromSTSCredentials(sessionCreds);
updateConfigWithSessionCreds(conf2, mc);
conf2.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
conf2.set(AWS_CREDENTIALS_PROVIDER, TEMPORARY_AWS_CREDENTIALS);
// with valid credentials, we can set properties.
try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf2)) {
@ -130,6 +163,16 @@ public void testSTS() throws IOException {
}
}
protected String getStsEndpoint(final Configuration conf) {
return conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT,
DEFAULT_ASSUMED_ROLE_STS_ENDPOINT);
}
protected String getStsRegion(final Configuration conf) {
return conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT_REGION,
ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT);
}
@Test
public void testTemporaryCredentialValidation() throws Throwable {
Configuration conf = new Configuration();
@ -139,4 +182,265 @@ public void testTemporaryCredentialValidation() throws Throwable {
LambdaTestUtils.intercept(CredentialInitializationException.class,
() -> new TemporaryAWSCredentialsProvider(conf).getCredentials());
}
/**
* Test that session tokens are propagated, with the origin string
* declaring this.
*/
@Test
public void testSessionTokenPropagation() throws Exception {
Configuration conf = new Configuration(getContract().getConf());
MarshalledCredentials sc = requestSessionCredentials(conf,
getFileSystem().getBucket());
updateConfigWithSessionCreds(conf, sc);
conf.set(AWS_CREDENTIALS_PROVIDER, TEMPORARY_AWS_CREDENTIALS);
try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
SessionTokenIdentifier identifier
= (SessionTokenIdentifier) fs.getDelegationToken("")
.decodeIdentifier();
String ids = identifier.toString();
assertThat("origin in " + ids,
identifier.getOrigin(),
containsString(CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN));
// and validate the AWS bits to make sure everything has come across.
assertCredentialsEqual("Reissued credentials in " + ids,
sc,
identifier.getMarshalledCredentials());
}
}
/**
* Examine the returned expiry time and validate it against expectations.
* Allows for some flexibility in local clock, but not much.
*/
@Test
public void testSessionTokenExpiry() throws Exception {
Configuration conf = new Configuration(getContract().getConf());
MarshalledCredentials sc = requestSessionCredentials(conf,
getFileSystem().getBucket());
long permittedExpiryOffset = 60;
OffsetDateTime expirationTimestamp = sc.getExpirationDateTime().get();
OffsetDateTime localTimestamp = OffsetDateTime.now();
assertTrue("local time of " + localTimestamp
+ " is after expiry time of " + expirationTimestamp,
localTimestamp.isBefore(expirationTimestamp));
// what is the interval
Duration actualDuration = Duration.between(localTimestamp,
expirationTimestamp);
Duration offset = actualDuration.minus(TEST_SESSION_TOKEN_DURATION);
assertThat(
"Duration of session " + actualDuration
+ " out of expected range of with " + offset
+ " this host's clock may be wrong.",
offset.getSeconds(),
Matchers.lessThanOrEqualTo(permittedExpiryOffset));
}
protected void updateConfigWithSessionCreds(final Configuration conf,
final MarshalledCredentials sc) {
unsetHadoopCredentialProviders(conf);
sc.setSecretsInConfiguration(conf);
}
/**
* Create an invalid session token and verify that it is rejected.
*/
@Test
public void testInvalidSTSBinding() throws Exception {
Configuration conf = new Configuration(getContract().getConf());
MarshalledCredentials sc = requestSessionCredentials(conf,
getFileSystem().getBucket());
toAWSCredentials(sc,
MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty, "");
updateConfigWithSessionCreds(conf, sc);
conf.set(AWS_CREDENTIALS_PROVIDER, TEMPORARY_AWS_CREDENTIALS);
conf.set(SESSION_TOKEN, "invalid-" + sc.getSessionToken());
S3AFileSystem fs = null;
try {
// this may throw an exception, which is an acceptable outcome.
// it must be in the try/catch clause.
fs = S3ATestUtils.createTestFileSystem(conf);
Path path = path("testSTSInvalidToken");
createAndVerifyFile(fs,
path,
TEST_FILE_SIZE);
// this is a failure path, so fail with a meaningful error
fail("request to create a file should have failed");
} catch (AWSBadRequestException expected){
// likely at two points in the operation, depending on
// S3Guard state
} finally {
IOUtils.closeStream(fs);
}
}
@Test
public void testSessionCredentialsBadRegion() throws Throwable {
describe("Create a session with a bad region and expect failure");
expectedSessionRequestFailure(
IllegalArgumentException.class,
DEFAULT_DELEGATION_TOKEN_ENDPOINT,
"us-west-12",
"");
}
@Test
public void testSessionCredentialsWrongRegion() throws Throwable {
describe("Create a session with the wrong region and expect failure");
expectedSessionRequestFailure(
AccessDeniedException.class,
STS_LONDON,
EU_IRELAND,
"");
}
@Test
public void testSessionCredentialsWrongCentralRegion() throws Throwable {
describe("Create a session sts.amazonaws.com; region='us-west-1'");
expectedSessionRequestFailure(
IllegalArgumentException.class,
"sts.amazonaws.com",
"us-west-1",
"");
}
@Test
public void testSessionCredentialsRegionNoEndpoint() throws Throwable {
describe("Create a session with a bad region and expect fast failure");
expectedSessionRequestFailure(
IllegalArgumentException.class,
"",
EU_IRELAND,
EU_IRELAND);
}
@Test
public void testSessionCredentialsRegionBadEndpoint() throws Throwable {
describe("Create a session with a bad region and expect fast failure");
IllegalArgumentException ex
= expectedSessionRequestFailure(
IllegalArgumentException.class,
" ",
EU_IRELAND,
"");
LOG.info("Outcome: ", ex);
if (!(ex.getCause() instanceof URISyntaxException)) {
throw ex;
}
}
@Test
public void testSessionCredentialsEndpointNoRegion() throws Throwable {
expectedSessionRequestFailure(
IllegalArgumentException.class,
STS_LONDON,
"",
STS_LONDON);
}
/**
* Expect an attempt to create a session or request credentials to fail
* with a specific exception class, optionally text.
* @param clazz exact class of exception.
* @param endpoint value for the sts endpoint option.
* @param region signing region.
* @param exceptionText text or "" in the exception.
* @param <E> type of exception.
* @return the caught exception.
* @throws Exception any unexpected exception.
*/
public <E extends Exception> E expectedSessionRequestFailure(
final Class<E> clazz,
final String endpoint,
final String region,
final String exceptionText) throws Exception {
try(AWSCredentialProviderList parentCreds =
getFileSystem().shareCredentials("test");
DurationInfo ignored = new DurationInfo(LOG, "requesting credentials")) {
Configuration conf = new Configuration(getContract().getConf());
ClientConfiguration awsConf =
S3AUtils.createAwsConf(conf, null);
return intercept(clazz, exceptionText,
() -> {
AWSSecurityTokenService tokenService =
STSClientFactory.builder(parentCreds,
awsConf,
endpoint,
region)
.build();
Invoker invoker = new Invoker(new S3ARetryPolicy(conf),
LOG_AT_ERROR);
STSClientFactory.STSClient stsClient
= STSClientFactory.createClientConnection(tokenService,
invoker);
return stsClient.requestSessionCredentials(30, TimeUnit.MINUTES);
});
}
}
/**
* Log retries at debug.
*/
public static final Invoker.Retried LOG_AT_ERROR =
(text, exception, retries, idempotent) -> {
LOG.error("{}", text, exception);
};
@Test
public void testTemporaryCredentialValidationOnLoad() throws Throwable {
Configuration conf = new Configuration();
unsetHadoopCredentialProviders(conf);
conf.set(ACCESS_KEY, "aaa");
conf.set(SECRET_KEY, "bbb");
conf.set(SESSION_TOKEN, "");
final MarshalledCredentials sc = MarshalledCredentialBinding.fromFileSystem(
null, conf);
intercept(IOException.class,
MarshalledCredentials.INVALID_CREDENTIALS,
() -> {
sc.validate("",
MarshalledCredentials.CredentialTypeRequired.SessionOnly);
return sc.toString();
});
}
@Test
public void testEmptyTemporaryCredentialValidation() throws Throwable {
Configuration conf = new Configuration();
unsetHadoopCredentialProviders(conf);
conf.set(ACCESS_KEY, "");
conf.set(SECRET_KEY, "");
conf.set(SESSION_TOKEN, "");
final MarshalledCredentials sc = MarshalledCredentialBinding.fromFileSystem(
null, conf);
intercept(IOException.class,
MarshalledCredentialBinding.NO_AWS_CREDENTIALS,
() -> {
sc.validate("",
MarshalledCredentials.CredentialTypeRequired.SessionOnly);
return sc.toString();
});
}
/**
* Verify that the request mechanism is translating exceptions.
* @throws Exception on a failure
*/
@Test
public void testSessionRequestExceptionTranslation() throws Exception {
intercept(IOException.class,
() -> requestSessionCredentials(getConfiguration(),
getFileSystem().getBucket(), 10));
}
}

View File

@ -37,6 +37,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
import org.apache.hadoop.util.Progressable;
@ -71,6 +72,8 @@ public class MockS3AFileSystem extends S3AFileSystem {
/** Log the entire stack of where operations are called: {@value}. */
public static final int LOG_STACK = 2;
private final Path root;
/**
* This can be edited to set the log level of events through the
* mock FS.
@ -85,8 +88,10 @@ public MockS3AFileSystem(S3AFileSystem mock,
Pair<StagingTestBase.ClientResults, StagingTestBase.ClientErrors> outcome) {
this.mock = mock;
this.outcome = outcome;
setUri(FS_URI);
setUri(FS_URI, false);
setBucket(BUCKET);
setEncryptionSecrets(new EncryptionSecrets());
root = new Path(FS_URI.toString());
}
public Pair<StagingTestBase.ClientResults, StagingTestBase.ClientErrors>
@ -118,9 +123,19 @@ private void event(String format, Object... args) {
}
}
@Override
public URI getUri() {
return FS_URI;
}
@Override
public Path getWorkingDirectory() {
return new Path("s3a://" + BUCKET + "/work");
return new Path(root, "work");
}
@Override
public Path qualify(final Path path) {
return path.makeQualified(FS_URI, getWorkingDirectory());
}
@Override

View File

@ -37,7 +37,8 @@ public class MockS3ClientFactory implements S3ClientFactory {
@Override
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentialSet) {
final AWSCredentialsProvider credentialSet,
final String userAgentSuffix) {
AmazonS3 s3 = mock(AmazonS3.class);
when(s3.doesBucketExist(bucket)).thenReturn(true);
// this listing is used in startup if purging is enabled, so

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.fs.s3a;
import java.time.Duration;
/**
* Constants for S3A Testing.
*/
@ -137,6 +139,12 @@ public interface S3ATestConstants {
*/
String TEST_UNIQUE_FORK_ID = "test.unique.fork.id";
String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
/**
* Endpoint for STS testing.
* @deprecated : Use {@link Constants#ASSUMED_ROLE_STS_ENDPOIN}
*/
@Deprecated
String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
/**
@ -173,4 +181,16 @@ public interface S3ATestConstants {
*/
String FS_S3A_IMPL_DISABLE_CACHE
= "fs.s3a.impl.disable.cache";
/**
* Duration in seconds for role/session token requests: {@value}.
*/
int TEST_SESSION_TOKEN_DURATION_SECONDS = 900;
/**
* Test session duration as a java 8 Duration.
*/
Duration TEST_SESSION_TOKEN_DURATION = Duration.ofSeconds(
TEST_SESSION_TOKEN_DURATION_SECONDS);
}

View File

@ -29,10 +29,21 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.service.Service;
import org.apache.hadoop.service.ServiceOperations;
import org.apache.hadoop.util.ReflectionUtils;
import com.amazonaws.auth.AWSCredentialsProvider;
import org.hamcrest.core.Is;
import org.junit.Assert;
import org.junit.Assume;
@ -51,6 +62,7 @@
import java.util.Map;
import java.util.concurrent.Callable;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH;
import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
import static org.apache.hadoop.fs.s3a.FailureInjectionPolicy.*;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
@ -76,6 +88,27 @@ public final class S3ATestUtils {
public static final String UNSET_PROPERTY = "unset";
public static final int PURGE_DELAY_SECONDS = 60 * 60;
/** Add any deprecated keys. */
@SuppressWarnings("deprecation")
private static void addDeprecatedKeys() {
// STS endpoint configuration option
Configuration.DeprecationDelta[] deltas = {
// STS endpoint configuration option
new Configuration.DeprecationDelta(
S3ATestConstants.TEST_STS_ENDPOINT,
ASSUMED_ROLE_STS_ENDPOINT)
};
if (deltas.length > 0) {
Configuration.addDeprecations(deltas);
Configuration.reloadExistingConfigurations();
}
}
static {
addDeprecatedKeys();
}
/**
* Get S3A FS name.
* @param conf configuration.
@ -514,6 +547,212 @@ public static Configuration prepareTestConfiguration(final Configuration conf) {
return conf;
}
/**
* Clear any Hadoop credential provider path.
* This is needed if people's test setups switch to credential providers,
* and the test case is altering FS login details: changes made in the
* config will not be picked up.
* @param conf configuration to update
*/
public static void unsetHadoopCredentialProviders(final Configuration conf) {
conf.unset(HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH);
}
/**
* Build AWS credentials to talk to the STS. Also where checks for the
* session tests being disabled are implemented.
* @return a set of credentials
* @throws IOException on a failure
*/
public static AWSCredentialsProvider buildAwsCredentialsProvider(
final Configuration conf)
throws IOException {
assumeSessionTestsEnabled(conf);
S3xLoginHelper.Login login = S3AUtils.getAWSAccessKeys(
URI.create("s3a://foobar"), conf);
if (!login.hasLogin()) {
skip("testSTS disabled because AWS credentials not configured");
}
return new SimpleAWSCredentialsProvider(login);
}
/**
* Skip the current test if STS tess are not enabled.
* @param conf configuration to examine
*/
public static void assumeSessionTestsEnabled(final Configuration conf) {
if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
skip("STS functional tests disabled");
}
}
/**
* Request session credentials for the default time (900s).
* @param conf configuration to use for login
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @return the credentials
* @throws IOException on a failure
*/
public static MarshalledCredentials requestSessionCredentials(
final Configuration conf,
final String bucket)
throws IOException {
return requestSessionCredentials(conf, bucket,
TEST_SESSION_TOKEN_DURATION_SECONDS);
}
/**
* Request session credentials.
* @param conf The Hadoop configuration
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @param duration duration in seconds.
* @return the credentials
* @throws IOException on a failure
*/
public static MarshalledCredentials requestSessionCredentials(
final Configuration conf,
final String bucket,
final int duration)
throws IOException {
assumeSessionTestsEnabled(conf);
MarshalledCredentials sc = MarshalledCredentialBinding
.requestSessionCredentials(
buildAwsCredentialsProvider(conf),
S3AUtils.createAwsConf(conf, bucket),
conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT,
DEFAULT_ASSUMED_ROLE_STS_ENDPOINT),
conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT_REGION,
ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT),
duration,
new Invoker(new S3ARetryPolicy(conf), Invoker.LOG_EVENT));
sc.validate("requested session credentials: ",
MarshalledCredentials.CredentialTypeRequired.SessionOnly);
return sc;
}
/**
* Round trip a writable to a new instance.
* @param source source object
* @param conf configuration
* @param <T> type
* @return an unmarshalled instance of the type
* @throws Exception on any failure.
*/
@SuppressWarnings("unchecked")
public static <T extends Writable> T roundTrip(
final T source,
final Configuration conf)
throws Exception {
DataOutputBuffer dob = new DataOutputBuffer();
source.write(dob);
DataInputBuffer dib = new DataInputBuffer();
dib.reset(dob.getData(), dob.getLength());
T after = ReflectionUtils.newInstance((Class<T>) source.getClass(), conf);
after.readFields(dib);
return after;
}
/**
* Remove any values from a bucket.
* @param bucket bucket whose overrides are to be removed. Can be null/empty
* @param conf config
* @param options list of fs.s3a options to remove
*/
public static void removeBucketOverrides(final String bucket,
final Configuration conf,
final String... options) {
if (StringUtils.isEmpty(bucket)) {
return;
}
final String bucketPrefix = FS_S3A_BUCKET_PREFIX + bucket + '.';
for (String option : options) {
final String stripped = option.substring("fs.s3a.".length());
String target = bucketPrefix + stripped;
if (conf.get(target) != null) {
LOG.debug("Removing option {}", target);
conf.unset(target);
}
}
}
/**
* Remove any values from a bucket and the base values too.
* @param bucket bucket whose overrides are to be removed. Can be null/empty.
* @param conf config
* @param options list of fs.s3a options to remove
*/
public static void removeBaseAndBucketOverrides(final String bucket,
final Configuration conf,
final String... options) {
for (String option : options) {
conf.unset(option);
}
removeBucketOverrides(bucket, conf, options);
}
/**
* Call a function; any exception raised is logged at info.
* This is for test teardowns.
* @param log log to use.
* @param operation operation to invoke
* @param <T> type of operation.
*/
public static <T> void callQuietly(final Logger log,
final Invoker.Operation<T> operation) {
try {
operation.execute();
} catch (Exception e) {
log.info(e.toString(), e);
}
}
/**
* Call a void operation; any exception raised is logged at info.
* This is for test teardowns.
* @param log log to use.
* @param operation operation to invoke
*/
public static void callQuietly(final Logger log,
final Invoker.VoidOperation operation) {
try {
operation.execute();
} catch (Exception e) {
log.info(e.toString(), e);
}
}
/**
* Deploy a hadoop service: init and start it.
* @param conf configuration to use
* @param service service to configure
* @param <T> type of service
* @return the started service
*/
public static <T extends Service> T deployService(
final Configuration conf,
final T service) {
service.init(conf);
service.start();
return service;
}
/**
* Terminate a service, returning {@code null} cast at compile-time
* to the type of the service, for ease of setting fields to null.
* @param service service.
* @param <T> type of the service
* @return null, always
*/
@SuppressWarnings("ThrowableNotThrown")
public static <T extends Service> T terminateService(final T service) {
ServiceOperations.stopQuietly(LOG, service);
return null;
}
/**
* Helper class to do diffs of metrics.
*/

View File

@ -23,12 +23,14 @@
import java.net.URI;
import java.nio.file.AccessDeniedException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
import com.google.common.collect.Sets;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
@ -51,18 +53,24 @@
*/
public class TestS3AAWSCredentialsProvider {
/**
* URI of the landsat images.
*/
private static final URI TESTFILE_URI = new Path(
DEFAULT_CSVTEST_FILE).toUri();
@Rule
public ExpectedException exception = ExpectedException.none();
@Test
public void testProviderWrongClass() throws Exception {
expectProviderInstantiationFailure(this.getClass().getName(),
expectProviderInstantiationFailure(this.getClass(),
NOT_AWS_PROVIDER);
}
@Test
public void testProviderAbstractClass() throws Exception {
expectProviderInstantiationFailure(AbstractProvider.class.getName(),
expectProviderInstantiationFailure(AbstractProvider.class,
ABSTRACT_PROVIDER);
}
@ -75,20 +83,20 @@ public void testProviderNotAClass() throws Exception {
@Test
public void testProviderConstructorError() throws Exception {
expectProviderInstantiationFailure(
ConstructorSignatureErrorProvider.class.getName(),
ConstructorSignatureErrorProvider.class,
CONSTRUCTOR_EXCEPTION);
}
@Test
public void testProviderFailureError() throws Exception {
expectProviderInstantiationFailure(
ConstructorFailureProvider.class.getName(),
ConstructorFailureProvider.class,
INSTANTIATION_EXCEPTION);
}
@Test
public void testInstantiationChain() throws Throwable {
Configuration conf = new Configuration();
Configuration conf = new Configuration(false);
conf.set(AWS_CREDENTIALS_PROVIDER,
TemporaryAWSCredentialsProvider.NAME
+ ", \t" + SimpleAWSCredentialsProvider.NAME
@ -96,10 +104,9 @@ public void testInstantiationChain() throws Throwable {
Path testFile = new Path(
conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
URI uri = testFile.toUri();
AWSCredentialProviderList list = S3AUtils.createAWSCredentialProviderSet(
uri, conf);
List<Class<? extends AWSCredentialsProvider>> expectedClasses =
AWSCredentialProviderList list = createAWSCredentialProviderSet(
testFile.toUri(), conf);
List<Class<?>> expectedClasses =
Arrays.asList(
TemporaryAWSCredentialsProvider.class,
SimpleAWSCredentialsProvider.class,
@ -110,60 +117,73 @@ public void testInstantiationChain() throws Throwable {
@Test
public void testDefaultChain() throws Exception {
URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
Configuration conf = new Configuration();
Configuration conf = new Configuration(false);
// use the default credential provider chain
conf.unset(AWS_CREDENTIALS_PROVIDER);
AWSCredentialProviderList list1 = S3AUtils.createAWSCredentialProviderSet(
AWSCredentialProviderList list1 = createAWSCredentialProviderSet(
uri1, conf);
AWSCredentialProviderList list2 = S3AUtils.createAWSCredentialProviderSet(
AWSCredentialProviderList list2 = createAWSCredentialProviderSet(
uri2, conf);
List<Class<? extends AWSCredentialsProvider>> expectedClasses =
Arrays.asList(
SimpleAWSCredentialsProvider.class,
EnvironmentVariableCredentialsProvider.class,
InstanceProfileCredentialsProvider.class);
List<Class<?>> expectedClasses = STANDARD_AWS_PROVIDERS;
assertCredentialProviders(expectedClasses, list1);
assertCredentialProviders(expectedClasses, list2);
assertSameInstanceProfileCredentialsProvider(list1.getProviders().get(2),
list2.getProviders().get(2));
}
@Test
public void testDefaultChainNoURI() throws Exception {
Configuration conf = new Configuration(false);
// use the default credential provider chain
conf.unset(AWS_CREDENTIALS_PROVIDER);
assertCredentialProviders(STANDARD_AWS_PROVIDERS,
createAWSCredentialProviderSet(null, conf));
}
@Test
public void testConfiguredChain() throws Exception {
URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
Configuration conf = new Configuration();
List<Class<? extends AWSCredentialsProvider>> expectedClasses =
List<Class<?>> expectedClasses =
Arrays.asList(
EnvironmentVariableCredentialsProvider.class,
InstanceProfileCredentialsProvider.class,
AnonymousAWSCredentialsProvider.class);
conf.set(AWS_CREDENTIALS_PROVIDER, buildClassListString(expectedClasses));
AWSCredentialProviderList list1 = S3AUtils.createAWSCredentialProviderSet(
Configuration conf =
createProviderConfiguration(buildClassListString(expectedClasses));
AWSCredentialProviderList list1 = createAWSCredentialProviderSet(
uri1, conf);
AWSCredentialProviderList list2 = S3AUtils.createAWSCredentialProviderSet(
AWSCredentialProviderList list2 = createAWSCredentialProviderSet(
uri2, conf);
assertCredentialProviders(expectedClasses, list1);
assertCredentialProviders(expectedClasses, list2);
assertSameInstanceProfileCredentialsProvider(list1.getProviders().get(1),
list2.getProviders().get(1));
}
@Test
public void testConfiguredChainUsesSharedInstanceProfile() throws Exception {
URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
Configuration conf = new Configuration();
List<Class<? extends AWSCredentialsProvider>> expectedClasses =
Arrays.<Class<? extends AWSCredentialsProvider>>asList(
Configuration conf = new Configuration(false);
List<Class<?>> expectedClasses =
Arrays.asList(
InstanceProfileCredentialsProvider.class);
conf.set(AWS_CREDENTIALS_PROVIDER, buildClassListString(expectedClasses));
AWSCredentialProviderList list1 = S3AUtils.createAWSCredentialProviderSet(
AWSCredentialProviderList list1 = createAWSCredentialProviderSet(
uri1, conf);
AWSCredentialProviderList list2 = S3AUtils.createAWSCredentialProviderSet(
AWSCredentialProviderList list2 = createAWSCredentialProviderSet(
uri2, conf);
assertCredentialProviders(expectedClasses, list1);
assertCredentialProviders(expectedClasses, list2);
assertSameInstanceProfileCredentialsProvider(list1.getProviders().get(0),
list2.getProviders().get(0));
}
@Test
public void testFallbackToDefaults() throws Throwable {
// build up the base provider
final AWSCredentialProviderList credentials = buildAWSProviderList(
new URI("s3a://bucket1"),
createProviderConfiguration(" "),
ASSUMED_ROLE_CREDENTIALS_PROVIDER,
Arrays.asList(
EnvironmentVariableCredentialsProvider.class),
Sets.newHashSet());
assertTrue("empty credentials", credentials.size() > 0);
}
/**
@ -213,14 +233,110 @@ public void refresh() {
}
}
@Test
public void testAWSExceptionTranslation() throws Throwable {
IOException ex = expectProviderInstantiationFailure(
AWSExceptionRaisingFactory.class,
AWSExceptionRaisingFactory.NO_AUTH);
if (!(ex instanceof AccessDeniedException)) {
throw ex;
}
}
static class AWSExceptionRaisingFactory implements AWSCredentialsProvider {
public static final String NO_AUTH = "No auth";
public static AWSCredentialsProvider getInstance() {
throw new NoAuthWithAWSException(NO_AUTH);
}
@Override
public AWSCredentials getCredentials() {
return null;
}
@Override
public void refresh() {
}
}
@Test
public void testFactoryWrongType() throws Throwable {
expectProviderInstantiationFailure(
FactoryOfWrongType.class,
CONSTRUCTOR_EXCEPTION);
}
static class FactoryOfWrongType implements AWSCredentialsProvider {
public static final String NO_AUTH = "No auth";
public static String getInstance() {
return "oops";
}
@Override
public AWSCredentials getCredentials() {
return null;
}
@Override
public void refresh() {
}
}
/**
* Expect a provider to raise an exception on failure.
* @param option aws provider option string.
* @param expectedErrorText error text to expect
* @return the exception raised
* @throws Exception any unexpected exception thrown.
*/
private IOException expectProviderInstantiationFailure(String option,
String expectedErrorText) throws Exception {
Configuration conf = new Configuration();
conf.set(AWS_CREDENTIALS_PROVIDER, option);
Path testFile = new Path(
conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
return intercept(IOException.class, expectedErrorText,
() -> S3AUtils.createAWSCredentialProviderSet(testFile.toUri(), conf));
() -> createAWSCredentialProviderSet(
TESTFILE_URI,
createProviderConfiguration(option)));
}
/**
* Expect a provider to raise an exception on failure.
* @param aClass class to use
* @param expectedErrorText error text to expect
* @return the exception raised
* @throws Exception any unexpected exception thrown.
*/
private IOException expectProviderInstantiationFailure(Class aClass,
String expectedErrorText) throws Exception {
return expectProviderInstantiationFailure(
buildClassListString(Collections.singletonList(aClass)),
expectedErrorText);
}
/**
* Create a configuration with a specific provider.
* @param providerOption option for the aws credential provider option.
* @return a configuration to use in test cases
*/
private Configuration createProviderConfiguration(
final String providerOption) {
Configuration conf = new Configuration(false);
conf.set(AWS_CREDENTIALS_PROVIDER, providerOption);
return conf;
}
/**
* Create a configuration with a specific class.
* @param aClass class to use
* @return a configuration to use in test cases
*/
public Configuration createProviderConfiguration(final Class<?> aClass) {
return createProviderConfiguration(buildClassListString(
Collections.singletonList(aClass)));
}
/**
@ -229,13 +345,13 @@ private IOException expectProviderInstantiationFailure(String option,
* @param list providers to check
*/
private static void assertCredentialProviders(
List<Class<? extends AWSCredentialsProvider>> expectedClasses,
List<Class<?>> expectedClasses,
AWSCredentialProviderList list) {
assertNotNull(list);
List<AWSCredentialsProvider> providers = list.getProviders();
assertEquals(expectedClasses.size(), providers.size());
for (int i = 0; i < expectedClasses.size(); ++i) {
Class<? extends AWSCredentialsProvider> expectedClass =
Class<?> expectedClass =
expectedClasses.get(i);
AWSCredentialsProvider provider = providers.get(i);
assertNotNull(
@ -248,23 +364,6 @@ private static void assertCredentialProviders(
}
}
/**
* Asserts that two different references point to the same shared instance of
* InstanceProfileCredentialsProvider using a descriptive assertion message.
* @param provider1 provider to check
* @param provider2 provider to check
*/
private static void assertSameInstanceProfileCredentialsProvider(
AWSCredentialsProvider provider1, AWSCredentialsProvider provider2) {
assertNotNull(provider1);
assertInstanceOf(InstanceProfileCredentialsProvider.class, provider1);
assertNotNull(provider2);
assertInstanceOf(InstanceProfileCredentialsProvider.class, provider2);
assertSame("Expected all usage of InstanceProfileCredentialsProvider to "
+ "share a singleton instance, but found unique instances.",
provider1, provider2);
}
/**
* This is here to check up on the S3ATestUtils probes themselves.
* @see S3ATestUtils#authenticationContains(Configuration, String).
@ -291,7 +390,7 @@ public void testExceptionLogic() throws Throwable {
// but that it closes safely
providers.close();
S3ARetryPolicy retryPolicy = new S3ARetryPolicy(new Configuration());
S3ARetryPolicy retryPolicy = new S3ARetryPolicy(new Configuration(false));
assertEquals("Expected no retry on auth failure",
RetryPolicy.RetryAction.FAIL.action,
retryPolicy.shouldRetry(noAuth, 0, 0, true).action);
@ -356,6 +455,9 @@ public void testIOEInConstructorPropagation() throws Throwable {
}
}
/**
* Credential provider which raises an IOE when constructed.
*/
private static class IOERaisingProvider implements AWSCredentialsProvider {
public IOERaisingProvider(URI uri, Configuration conf)

View File

@ -280,4 +280,29 @@ public void testGetBucketPasswordFromProviderShort() throws Throwable {
assertSecretKeyEquals(conf, bucketURI.getHost(), "overidden", "overidden");
}
@Test
public void testUnknownEncryptionMethod() throws Throwable {
intercept(IOException.class, UNKNOWN_ALGORITHM,
() -> S3AEncryptionMethods.getMethod("SSE-ROT13"));
}
@Test
public void testClientEncryptionMethod() throws Throwable {
S3AEncryptionMethods method = getMethod("CSE-KMS");
assertEquals(CSE_KMS, method);
assertFalse("shouldn't be server side " + method, method.isServerSide());
}
@Test
public void testCSEKMSEncryptionMethod() throws Throwable {
S3AEncryptionMethods method = getMethod("CSE-CUSTOM");
assertEquals(CSE_CUSTOM, method);
assertFalse("shouldn't be server side " + method, method.isServerSide());
}
@Test
public void testNoEncryptionMethod() throws Throwable {
assertEquals(NONE, getMethod(" "));
}
}

View File

@ -30,6 +30,7 @@
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -61,6 +62,7 @@
import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
import static org.apache.hadoop.test.LambdaTestUtils.*;
@ -76,6 +78,9 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
private static final Path ROOT = new Path("/");
private static final Statement STATEMENT_ALL_BUCKET_READ_ACCESS
= statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS);
/**
* test URI, built in setup.
*/
@ -135,6 +140,34 @@ private <E extends Throwable> E expectFileSystemCreateFailure(
public void testCreateCredentialProvider() throws IOException {
describe("Create the credential provider");
Configuration conf = createValidRoleConf();
try (AssumedRoleCredentialProvider provider
= new AssumedRoleCredentialProvider(uri, conf)) {
LOG.info("Provider is {}", provider);
AWSCredentials credentials = provider.getCredentials();
assertNotNull("Null credentials from " + provider, credentials);
}
}
@Test
public void testCreateCredentialProviderNoURI() throws IOException {
describe("Create the credential provider");
Configuration conf = createValidRoleConf();
try (AssumedRoleCredentialProvider provider
= new AssumedRoleCredentialProvider(null, conf)) {
LOG.info("Provider is {}", provider);
AWSCredentials credentials = provider.getCredentials();
assertNotNull("Null credentials from " + provider, credentials);
}
}
/**
* Create a valid role configuration.
* @return a configuration set to use to the role ARN.
* @throws JsonProcessingException problems working with JSON policies.
*/
protected Configuration createValidRoleConf() throws JsonProcessingException {
String roleARN = getAssumedRoleARN();
Configuration conf = new Configuration(getContract().getConf());
@ -143,12 +176,7 @@ public void testCreateCredentialProvider() throws IOException {
conf.set(ASSUMED_ROLE_SESSION_NAME, "valid");
conf.set(ASSUMED_ROLE_SESSION_DURATION, "45m");
bindRolePolicy(conf, RESTRICTED_POLICY);
try (AssumedRoleCredentialProvider provider
= new AssumedRoleCredentialProvider(uri, conf)) {
LOG.info("Provider is {}", provider);
AWSCredentials credentials = provider.getCredentials();
assertNotNull("Null credentials from " + provider, credentials);
}
return conf;
}
@Test
@ -205,11 +233,12 @@ public void testAssumeRoleCannotAuthAssumedRole() throws Exception {
describe("Assert that you can't use assumed roles to auth assumed roles");
Configuration conf = createAssumedRoleConfig();
unsetHadoopCredentialProviders(conf);
conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
AssumedRoleCredentialProvider.NAME);
expectFileSystemCreateFailure(conf,
IOException.class,
AssumedRoleCredentialProvider.E_FORBIDDEN_PROVIDER);
E_FORBIDDEN_AWS_PROVIDER);
}
@Test
@ -217,6 +246,7 @@ public void testAssumeRoleBadInnerAuth() throws Exception {
describe("Try to authenticate with a keypair with spaces");
Configuration conf = createAssumedRoleConfig();
unsetHadoopCredentialProviders(conf);
conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
SimpleAWSCredentialsProvider.NAME);
conf.set(ACCESS_KEY, "not valid");
@ -232,6 +262,7 @@ public void testAssumeRoleBadInnerAuth2() throws Exception {
describe("Try to authenticate with an invalid keypair");
Configuration conf = createAssumedRoleConfig();
unsetHadoopCredentialProviders(conf);
conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
SimpleAWSCredentialsProvider.NAME);
conf.set(ACCESS_KEY, "notvalid");
@ -461,7 +492,7 @@ public void testRestrictedWriteSubdir() throws Throwable {
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
STATEMENT_ALL_BUCKET_READ_ACCESS,
STATEMENT_ALLOW_SSE_KMS_RW,
new Statement(Effects.Allow)
.addActions(S3_ALL_OPERATIONS)
@ -525,7 +556,7 @@ public void executeRestrictedRename(final Configuration conf)
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALLOW_SSE_KMS_RW,
statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
STATEMENT_ALL_BUCKET_READ_ACCESS,
new Statement(Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(restrictedDir))
@ -617,7 +648,7 @@ public void executeRenameReadOnlyData(final Configuration conf)
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
STATEMENT_ALL_BUCKET_READ_ACCESS,
new Statement(Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(destDir))
@ -698,7 +729,7 @@ public void testRestrictedCommitActions() throws Throwable {
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALLOW_SSE_KMS_RW,
statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
STATEMENT_ALL_BUCKET_READ_ACCESS,
new Statement(Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(writeableDir))

View File

@ -74,7 +74,7 @@ public void setup() throws Exception {
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALLOW_SSE_KMS_RW,
statement(true, S3_ALL_BUCKETS, S3_ROOT_READ_OPERATIONS),
statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS),
new RoleModel.Statement(RoleModel.Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(restrictedDir))

View File

@ -22,6 +22,7 @@
import java.util.concurrent.Callable;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.junit.Assume;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -30,7 +31,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.s3a.Constants.*;
@ -38,6 +39,8 @@
import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Helper class for testing roles.
@ -150,6 +153,7 @@ public static Configuration newAssumedRoleConfig(
conf.set(ASSUMED_ROLE_ARN, roleARN);
conf.set(ASSUMED_ROLE_SESSION_NAME, "test");
conf.set(ASSUMED_ROLE_SESSION_DURATION, "15m");
conf.unset(DelegationConstants.DELEGATION_TOKEN_BINDING);
disableFilesystemCaching(conf);
return conf;
}
@ -170,4 +174,39 @@ public static <T> AccessDeniedException forbidden(
contained, eval);
}
/**
* Get the Assumed role referenced by ASSUMED_ROLE_ARN;
* skip the test if it is unset.
* @param conf config
* @return the string
*/
public static String probeForAssumedRoleARN(Configuration conf) {
String arn = conf.getTrimmed(ASSUMED_ROLE_ARN, "");
Assume.assumeTrue("No ARN defined in " + ASSUMED_ROLE_ARN,
!arn.isEmpty());
return arn;
}
/**
* Assert that credentials are equal without printing secrets.
* Different assertions will have different message details.
* @param message message to use as base of error.
* @param expected expected credentials
* @param actual actual credentials.
*/
public static void assertCredentialsEqual(final String message,
final MarshalledCredentials expected,
final MarshalledCredentials actual) {
// DO NOT use assertEquals() here, as that could print a secret to
// the test report.
assertEquals(message + ": access key",
expected.getAccessKey(),
actual.getAccessKey());
assertTrue(message + ": secret key",
expected.getSecretKey().equals(actual.getSecretKey()));
assertEquals(message + ": session token",
expected.getSessionToken(),
actual.getSessionToken());
}
}

View File

@ -0,0 +1,138 @@
/*
* 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.auth;
import java.net.URI;
import java.net.URISyntaxException;
import com.amazonaws.auth.AWSCredentials;
import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import org.apache.hadoop.test.HadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Unit test of marshalled credential support.
*/
public class TestMarshalledCredentials extends HadoopTestBase {
private MarshalledCredentials credentials;
private int expiration;
private URI bucketURI;
@Before
public void createSessionToken() throws URISyntaxException {
bucketURI = new URI("s3a://bucket1");
credentials = new MarshalledCredentials("accessKey",
"secretKey", "sessionToken");
credentials.setRoleARN("roleARN");
expiration = 1970;
credentials.setExpiration(expiration);
}
@Test
public void testRoundTrip() throws Throwable {
MarshalledCredentials c2 = S3ATestUtils.roundTrip(this.credentials,
new Configuration());
assertEquals(credentials, c2);
assertEquals("accessKey", c2.getAccessKey());
assertEquals("secretKey", c2.getSecretKey());
assertEquals("sessionToken", c2.getSessionToken());
assertEquals(expiration, c2.getExpiration());
assertEquals(credentials, c2);
}
@Test
public void testRoundTripNoSessionData() throws Throwable {
MarshalledCredentials c = new MarshalledCredentials();
c.setAccessKey("A");
c.setSecretKey("K");
MarshalledCredentials c2 = S3ATestUtils.roundTrip(c,
new Configuration());
assertEquals(c, c2);
}
@Test
public void testRoundTripEncryptionData() throws Throwable {
EncryptionSecrets secrets = new EncryptionSecrets(
S3AEncryptionMethods.SSE_KMS,
"key");
EncryptionSecrets result = S3ATestUtils.roundTrip(secrets,
new Configuration());
assertEquals("round trip", secrets, result);
}
@Test
public void testMarshalledCredentialProviderSession() throws Throwable {
MarshalledCredentialProvider provider
= new MarshalledCredentialProvider("test",
bucketURI,
new Configuration(false),
credentials,
MarshalledCredentials.CredentialTypeRequired.SessionOnly);
AWSCredentials aws = provider.getCredentials();
assertEquals(credentials.toString(),
credentials.getAccessKey(),
aws.getAWSAccessKeyId());
assertEquals(credentials.toString(),
credentials.getSecretKey(),
aws.getAWSSecretKey());
// because the credentials are set to full only, creation will fail
}
/**
* Create with a mismatch of type and supplied credentials.
* Verify that the operation fails, but only when credentials
* are actually requested.
*/
@Test
public void testCredentialTypeMismatch() throws Throwable {
MarshalledCredentialProvider provider
= new MarshalledCredentialProvider("test",
bucketURI,
new Configuration(false),
credentials,
MarshalledCredentials.CredentialTypeRequired.FullOnly);
// because the credentials are set to full only, creation will fail
intercept(NoAuthWithAWSException.class, "test",
() -> provider.getCredentials());
}
/**
* This provider fails fast if there's no URL.
*/
@Test
public void testCredentialProviderNullURI() throws Throwable {
intercept(NullPointerException.class, "",
() ->
new MarshalledCredentialProvider("test",
null,
new Configuration(false),
credentials,
MarshalledCredentials.CredentialTypeRequired.FullOnly));
}
}

View File

@ -0,0 +1,207 @@
/*
* 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.auth.delegation;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.net.URI;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken;
/**
* superclass class for DT tests.
*/
public abstract class AbstractDelegationIT extends AbstractS3ATestBase {
protected static final String YARN_RM = "yarn-rm@EXAMPLE";
private static final Logger LOG =
LoggerFactory.getLogger(AbstractDelegationIT.class);
/**
* Look up a token from the submitted credentials.
* @param submittedCredentials credentials
* @param uri URI of the FS
* @param kind required kind of the token (which is asserted on)
* @return the token
* @throws IOException IO failure
*/
public static AbstractS3ATokenIdentifier lookupToken(
Credentials submittedCredentials,
URI uri,
Text kind) throws IOException {
final Token<AbstractS3ATokenIdentifier> token =
requireNonNull(
lookupS3ADelegationToken(submittedCredentials, uri),
"No Token for " + uri);
assertEquals("Kind of token " + token,
kind,
token.getKind());
return token.decodeIdentifier();
}
/**
* Create credentials with the DTs of the given FS.
* @param fs filesystem
* @return a non-empty set of credentials.
* @throws IOException failure to create.
*/
protected static Credentials mkTokens(final S3AFileSystem fs)
throws IOException {
Credentials cred = new Credentials();
fs.addDelegationTokens(AbstractDelegationIT.YARN_RM, cred);
return cred;
}
/**
* Create and Init an FS instance.
* @param uri URI
* @param conf config to use
* @return the instance
* @throws IOException failure to create/init
*/
protected static S3AFileSystem newS3AInstance(final URI uri,
final Configuration conf)
throws IOException {
S3AFileSystem fs = new S3AFileSystem();
fs.initialize(uri, conf);
return fs;
}
/**
* Assert that a filesystem is bound to a DT; that is: it is a delegate FS.
* @param fs filesystem
* @param tokenKind the kind of the token to require
*/
protected static void assertBoundToDT(final S3AFileSystem fs,
final Text tokenKind) {
final S3ADelegationTokens dtSupport = fs.getDelegationTokens().get();
assertTrue("Expected bound to a delegation token: " + dtSupport,
dtSupport.isBoundToDT());
assertEquals("Wrong token kind",
tokenKind, dtSupport.getBoundDT().get().getKind());
}
/**
* Assert that the number of tokens created by an FS matches the
* expected value.
* @param fs filesystem
* @param expected expected creation count.
*/
protected static void assertTokenCreationCount(final S3AFileSystem fs,
final int expected) {
assertEquals("DT creation count from " + fs.getDelegationTokens().get(),
expected,
getTokenCreationCount(fs));
}
/**
* Get the token creation count of a filesystem.
* @param fs FS
* @return creation count
*/
private static int getTokenCreationCount(final S3AFileSystem fs) {
return fs.getDelegationTokens()
.map(S3ADelegationTokens::getCreationCount)
.get();
}
/**
* Patch the current config with the DT binding.
* @param conf configuration to patch
* @param binding binding to use
*/
protected void enableDelegationTokens(Configuration conf, String binding) {
LOG.info("Enabling delegation token support for {}", binding);
conf.set(DELEGATION_TOKEN_BINDING, binding);
}
/**
* Reset UGI info.
*/
protected void resetUGI() {
UserGroupInformation.reset();
}
/**
* Bind the provider list to the args supplied.
* At least one must be provided, to stop the default list being
* picked up.
* @param config configuration to patch.
* @param bucket bucket to clear.
* @param providerClassnames providers
*/
protected void bindProviderList(String bucket,
Configuration config,
String... providerClassnames) {
removeBaseAndBucketOverrides(bucket, config, AWS_CREDENTIALS_PROVIDER);
assertTrue("No providers to bind to", providerClassnames.length > 0);
config.setStrings(AWS_CREDENTIALS_PROVIDER, providerClassnames);
}
/**
* Save a DT to a file.
* @param tokenFile destination file
* @param token token to save
* @throws IOException failure
*/
protected void saveDT(final File tokenFile, final Token<?> token)
throws IOException {
requireNonNull(token, "Null token");
Credentials cred = new Credentials();
cred.addToken(token.getService(), token);
try(DataOutputStream out = new DataOutputStream(
new FileOutputStream(tokenFile))) {
cred.writeTokenStorageToStream(out);
}
}
/**
* Create and init an S3a DT instance, but don't start it.
* @param conf conf to use
* @return a new instance
* @throws IOException IOE
*/
public S3ADelegationTokens instantiateDTSupport(Configuration conf)
throws IOException {
S3AFileSystem fs = getFileSystem();
S3ADelegationTokens tokens = new S3ADelegationTokens();
tokens.bindToFileSystem(fs.getCanonicalUri(), fs);
tokens.init(conf);
return tokens;
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.auth.delegation;
import java.util.concurrent.atomic.AtomicLong;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
/**
* Simple AWS credential provider which counts how often it is invoked.
*/
public class CountInvocationsProvider
implements AWSCredentialsProvider {
public static final String NAME = CountInvocationsProvider.class.getName();
public static final AtomicLong COUNTER = new AtomicLong(0);
@Override
public AWSCredentials getCredentials() {
COUNTER.incrementAndGet();
throw new CredentialInitializationException("no credentials");
}
@Override
public void refresh() {
}
public static long getInvocationCount() {
return COUNTER.get();
}
}

View File

@ -0,0 +1,103 @@
/*
* 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.auth.delegation;
import java.io.Closeable;
import java.io.IOException;
import java.io.Writer;
/**
* This is a small utility class to write out rows to a CSV/TSV file.
* It does not do any escaping of written text, so don't write entries
* containing separators.
* Quoting must be done external to this class.
*/
public final class Csvout implements Closeable {
private final Writer out;
private final String separator;
private final String eol;
private boolean isStartOfLine = true;
/**
* Instantiate.
* @param out output stream.
* @param separator field separator.
* @param eol end of line sequence
*/
public Csvout(final Writer out,
final String separator,
final String eol) {
this.out = out;
this.separator = separator;
this.eol = eol;
}
/**
* Close the output stream.
* @throws IOException IO failure.
*/
@Override
public void close() throws IOException {
out.close();
}
/**
* Write a single object's string value.
* @param o object to write.
* @return this instance
* @throws IOException IO failure.
*/
public Csvout write(Object o) throws IOException {
if (isStartOfLine) {
isStartOfLine = false;
} else {
out.write(separator);
}
out.write(o.toString());
return this;
}
/**
* Write a newline.
* @return this instance
* @throws IOException IO failure.
*/
public Csvout newline() throws IOException {
out.write(eol);
isStartOfLine = true;
return this;
}
/**
* Write a collection of objects.
* @param objects varags list of objects to write
* @return this instance.
* @throws IOException IO failure.
*/
public Csvout write(Object... objects) throws IOException {
for (Object object : objects) {
write(object);
}
return this;
}
}

View File

@ -16,29 +16,23 @@
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a;
package org.apache.hadoop.fs.s3a.auth.delegation;
import org.apache.hadoop.conf.Configuration;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_ROLE_BINDING;
/**
* Run the encryption tests against the block output stream.
* This looks at the cost of assume role, to see if it is more expensive
* than creating simple session credentials.
*/
public class ITestS3AEncryptionSSES3BlockOutputStream
extends AbstractTestS3AEncryption {
public class ILoadTestRoleCredentials extends ILoadTestSessionCredentials {
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
conf.set(Constants.FAST_UPLOAD_BUFFER,
Constants.FAST_UPLOAD_BYTEBUFFER);
//must specify encryption key as empty because SSE-S3 does not allow it,
//nor can it be null.
conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, "");
return conf;
protected String getDelegationBinding() {
return DELEGATION_TOKEN_ROLE_BINDING;
}
@Override
protected S3AEncryptionMethods getSSEAlgorithm() {
return S3AEncryptionMethods.SSE_S3;
protected String getFilePrefix() {
return "role";
}
}

View File

@ -0,0 +1,295 @@
/*
* 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.auth.delegation;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.scale.NanoTimerStats;
import org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeSessionTestsEnabled;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_SESSION_BINDING;
/**
* This test has a unique name as it is designed to do something special:
* generate enough load on the AWS STS service to get some
* statistics on its throttling.
* This isn't documented anywhere, and for DT support it's
* important to know how much effort it takes to overload the service.
*
* <b>Important</b>
*
* If this test does trigger STS throttling, then all users in the same
* AWS account will experience throttling. This may be observable,
* in delays and, if the applications in use are not resilient to
* throttling events in STS, from application failures.
*
* Use with caution.
* <ol>
* <li>Don't run it on an AWS endpoint which other users in a
* shared AWS account are actively using. </li>
* <li>Don't run it on the same AWS account which is being used for
* any production service.</li>
* <li>And choose a time (weekend, etc) where the account is under-used.</li>
* <li>Warn your fellow users.</li>
* </ol>
*
* In experiments, the throttling recovers fast and appears restricted
* to the single STS service which the test overloads.
*
* @see <a href="https://github.com/steveloughran/datasets/releases/tag/tag_2018-09-17-aws">
* AWS STS login throttling statistics</a>
*/
public class ILoadTestSessionCredentials extends S3AScaleTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(ILoadTestSessionCredentials.class);
protected static final int THREADS = 100;
private final ExecutorService executor =
HadoopExecutors.newFixedThreadPool(
THREADS,
new ThreadFactoryBuilder()
.setNameFormat("DelegationTokenFetcher #%d")
.build());
private final CompletionService<Outcome>
completionService =
new ExecutorCompletionService<>(executor);
private File dataDir;
@Override
protected Configuration createScaleConfiguration() {
Configuration conf = super.createScaleConfiguration();
conf.set(DELEGATION_TOKEN_BINDING,
getDelegationBinding());
conf.setInt(Constants.MAXIMUM_CONNECTIONS,
Math.max(THREADS, Constants.DEFAULT_MAXIMUM_CONNECTIONS));
conf.setInt(Constants.MAX_ERROR_RETRIES, 0);
return conf;
}
/**
* Which DT binding class to use.
* @return the binding config option.
*/
protected String getDelegationBinding() {
return DELEGATION_TOKEN_SESSION_BINDING;
}
@Override
public void setup() throws Exception {
super.setup();
assumeSessionTestsEnabled(getConfiguration());
S3AFileSystem fileSystem = getFileSystem();
assertNotNull(
"No delegation tokens in FS",
fileSystem.getCanonicalServiceName());
dataDir = GenericTestUtils.getTestDir("kerberos");
dataDir.mkdirs();
}
protected String getFilePrefix() {
return "session";
}
@Test
public void testCreate10Tokens() throws Throwable {
File file = fetchTokens(10);
String csv = FileUtils.readFileToString(file, "UTF-8");
LOG.info("CSV data\n{}", csv);
}
@Test
public void testCreateManyTokens() throws Throwable {
fetchTokens(50000);
}
/**
* Fetch tokens.
* @param tokens number of tokens.
* @return file the timings were
* @throws Exception failure
*/
private File fetchTokens(final int tokens)
throws Exception {
File filename = new File(dataDir, getFilePrefix() + "-" + tokens + ".csv");
fetchTokens(tokens, filename);
return filename;
}
/**
* Fetch tokens.
* @param tokens number of tokens.
* @param csvFile file to save this to.
* @throws Exception failure
*/
private void fetchTokens(final int tokens, final File csvFile)
throws Exception {
describe("Fetching %d tokens, saving log to %s", tokens, csvFile);
final FileWriter out = new FileWriter(csvFile);
Csvout csvout = new Csvout(out, "\t", "\n");
Outcome.writeSchema(csvout);
final S3AFileSystem fileSystem = getFileSystem();
final ContractTestUtils.NanoTimer jobTimer =
new ContractTestUtils.NanoTimer();
for (int i = 0; i < tokens; i++) {
final int id = i;
completionService.submit(() -> {
final long startTime = System.currentTimeMillis();
final ContractTestUtils.NanoTimer timer =
new ContractTestUtils.NanoTimer();
Exception ex = null;
try {
fileSystem.getDelegationToken("Count ");
} catch (IOException e) {
ex = e;
}
timer.end("Request");
return new Outcome(id, startTime, timer, ex);
});
}
NanoTimerStats stats = new NanoTimerStats("Overall");
NanoTimerStats success = new NanoTimerStats("Successful");
NanoTimerStats throttled = new NanoTimerStats("Throttled");
List<Outcome> throttledEvents = new ArrayList<>();
for (int i = 0; i < tokens; i++) {
Outcome outcome = completionService.take().get();
ContractTestUtils.NanoTimer timer = outcome.timer;
Exception ex = outcome.exception;
outcome.writeln(csvout);
stats.add(timer);
if (ex != null) {
// throttling event occurred.
LOG.info("Throttled at event {}", i, ex);
throttled.add(timer);
throttledEvents.add(outcome);
} else {
success.add(timer);
}
}
csvout.close();
jobTimer.end("Execution of fetch calls");
// now print the stats
LOG.info("Summary file is " + csvFile);
LOG.info("Fetched {} tokens with {} throttle events\n: {}\n{}\n{}",
tokens,
throttled.getCount(),
stats,
throttled,
success);
double duration = jobTimer.duration();
double iops = tokens * 1.0e9 / duration;
LOG.info(
String.format("Effective IO rate is %3f operations/second", iops));
// log at debug
if (LOG.isDebugEnabled()) {
throttledEvents.stream().forEach((outcome -> {
LOG.debug("{}: duration: {}",
outcome.id, outcome.timer.elapsedTimeMs());
}));
}
}
/**
* Outcome of one of the load operations.
*/
private static class Outcome {
private final int id;
private final long startTime;
private final ContractTestUtils.NanoTimer timer;
private final Exception exception;
Outcome(final int id,
final long startTime,
final ContractTestUtils.NanoTimer timer,
final Exception exception) {
this.id = id;
this.startTime = startTime;
this.timer = timer;
this.exception = exception;
}
/**
* Write this record.
* @param out the csvout to write through.
* @return the csvout instance
* @throws IOException IO failure.
*/
public Csvout writeln(Csvout out) throws IOException {
return out.write(
id,
startTime,
exception == null ? 1: 0,
timer.getStartTime(),
timer.getEndTime(),
timer.duration(),
'"' + (exception == null ? "" : exception.getMessage()) + '"')
.newline();
}
/**
* Write the schema of the outcome records.
* @param out CSV destinatin
* @throws IOException IO failure.
*/
public static void writeSchema(Csvout out) throws IOException {
out.write("id", "starttime", "success", "started", "ended",
"duration", "error");
}
}
}

View File

@ -0,0 +1,272 @@
/*
* 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.auth.delegation;
import java.util.Arrays;
import java.util.Collection;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.examples.WordCount;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.MockJob;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeSessionTestsEnabled;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.deployService;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyInt;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.terminateService;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.probeForAssumedRoleARN;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.closeUserFileSystems;
/**
* Submit a job with S3 delegation tokens.
*
* YARN will not collect DTs unless it is running secure, and turning
* security on complicates test setup "significantly".
* Specifically: buts of MR refuse to work on a local FS unless the
* native libraries are loaded and it can use lower level POSIX APIs
* for creating files and directories with specific permissions.
* In production, this is a good thing. In tests, this is not.
*
* To address this, Job to YARN communications are mocked.
* The client-side job submission is as normal, but the implementation
* of org.apache.hadoop.mapreduce.protocol.ClientProtocol is mock.
*
* It's still an ITest though, as it does use S3A as the source and
* dest so as to collect URLs.
*/
@RunWith(Parameterized.class)
public class ITestDelegatedMRJob extends AbstractDelegationIT {
private static final Logger LOG =
LoggerFactory.getLogger(ITestDelegatedMRJob.class);
/**
* Created in static {@link #setupCluster()} call.
*/
@SuppressWarnings("StaticNonFinalField")
private static MiniKerberizedHadoopCluster cluster;
private final String name;
private final String tokenBinding;
private final Text tokenKind;
/**
* Created in test setup.
*/
private MiniMRYarnCluster yarn;
private Path destPath;
/**
* Test array for parameterized test runs.
* @return a list of parameter tuples.
*/
@Parameterized.Parameters
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{"session", DELEGATION_TOKEN_SESSION_BINDING, SESSION_TOKEN_KIND},
{"full", DELEGATION_TOKEN_FULL_CREDENTIALS_BINDING, FULL_TOKEN_KIND},
{"role", DELEGATION_TOKEN_ROLE_BINDING, ROLE_TOKEN_KIND},
});
}
public ITestDelegatedMRJob(String name, String tokenBinding, Text tokenKind) {
this.name = name;
this.tokenBinding = tokenBinding;
this.tokenKind = tokenKind;
}
/***
* Set up the clusters.
*/
@BeforeClass
public static void setupCluster() throws Exception {
JobConf conf = new JobConf();
assumeSessionTestsEnabled(conf);
disableFilesystemCaching(conf);
cluster = deployService(conf, new MiniKerberizedHadoopCluster());
}
/**
* Tear down the cluster.
*/
@AfterClass
public static void teardownCluster() throws Exception {
cluster = terminateService(cluster);
}
@Override
protected YarnConfiguration createConfiguration() {
Configuration parent = super.createConfiguration();
YarnConfiguration conf = new YarnConfiguration(parent);
cluster.patchConfigWithYARNBindings(conf);
// fail fairly fast
conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
100);
conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
10_000);
// set up DTs
enableDelegationTokens(conf, tokenBinding);
return conf;
}
@Override
protected YarnConfiguration getConfiguration() {
return (YarnConfiguration) super.getConfiguration();
}
@Override
public void setup() throws Exception {
cluster.loginPrincipal();
super.setup();
Configuration conf = getConfiguration();
if (DELEGATION_TOKEN_ROLE_BINDING.equals(tokenBinding)) {
// get the ARN or skip the test
probeForAssumedRoleARN(getConfiguration());
}
// filesystems are cached across the test so that
// instrumentation fields can be asserted on
UserGroupInformation.setConfiguration(conf);
assertSecurityEnabled();
LOG.info("Starting MiniMRCluster");
yarn = deployService(conf,
new MiniMRYarnCluster("ITestDelegatedMRJob", 1));
}
@Override
public void teardown() throws Exception {
describe("Teardown operations");
S3AFileSystem fs = getFileSystem();
if (fs != null && destPath != null) {
fs.delete(destPath, true);
}
yarn = terminateService(yarn);
super.teardown();
closeUserFileSystems(UserGroupInformation.getCurrentUser());
}
/**
* Get the test timeout in seconds.
* @return the test timeout as set in system properties or the default.
*/
protected int getTestTimeoutSeconds() {
return getTestPropertyInt(new Configuration(),
KEY_TEST_TIMEOUT,
SCALE_TEST_TIMEOUT_SECONDS);
}
@Override
protected int getTestTimeoutMillis() {
return getTestTimeoutSeconds() * 1000;
}
@Test
public void testJobSubmissionCollectsTokens() throws Exception {
describe("Mock Job test");
JobConf conf = new JobConf(getConfiguration());
// the input here is the landsat file; which lets
// us differentiate source URI from dest URI
Path input = new Path(DEFAULT_CSVTEST_FILE);
final FileSystem sourceFS = input.getFileSystem(conf);
// output is in the writable test FS.
final S3AFileSystem fs = getFileSystem();
destPath = path(getMethodName());
fs.delete(destPath, true);
fs.mkdirs(destPath);
Path output = new Path(destPath, "output/");
output = output.makeQualified(fs.getUri(), fs.getWorkingDirectory());
MockJob job = new MockJob(conf, "word count");
job.setJarByClass(WordCount.class);
job.setMapperClass(WordCount.TokenizerMapper.class);
job.setCombinerClass(WordCount.IntSumReducer.class);
job.setReducerClass(WordCount.IntSumReducer.class);
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(IntWritable.class);
FileInputFormat.addInputPath(job, input);
FileOutputFormat.setOutputPath(job, output);
job.setMaxMapAttempts(1);
job.setMaxReduceAttempts(1);
describe("Executing Mock Job Submission to %s", output);
job.submit();
final JobStatus status = job.getStatus();
assertEquals("not a mock job",
MockJob.NAME, status.getSchedulingInfo());
assertEquals("Job State",
JobStatus.State.RUNNING, status.getState());
final Credentials submittedCredentials =
requireNonNull(job.getSubmittedCredentials(),
"job submitted credentials");
final Collection<Token<? extends TokenIdentifier>> tokens
= submittedCredentials.getAllTokens();
// log all the tokens for debugging failed test runs
LOG.info("Token Count = {}", tokens.size());
for (Token<? extends TokenIdentifier> token : tokens) {
LOG.info("{}", token);
}
// verify the source token exists
lookupToken(submittedCredentials, sourceFS.getUri(), tokenKind);
// look up the destination token
lookupToken(submittedCredentials, fs.getUri(), tokenKind);
}
}

View File

@ -0,0 +1,68 @@
/*
* 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.auth.delegation;
import java.nio.file.AccessDeniedException;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.io.Text;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.probeForAssumedRoleARN;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_ROLE_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.ROLE_TOKEN_KIND;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Subclass of the session test which checks roles; only works if
* a role ARN has been declared.
*/
public class ITestRoleDelegationInFileystem extends
ITestSessionDelegationInFileystem {
@Override
public void setup() throws Exception {
super.setup();
probeForAssumedRoleARN(getConfiguration());
}
@Override
protected String getDelegationBinding() {
return DELEGATION_TOKEN_ROLE_BINDING;
}
@Override
public Text getTokenKind() {
return ROLE_TOKEN_KIND;
}
/**
* This verifies that the granted credentials only access the target bucket
* by using the credentials in a new S3 client to query the AWS-owned landsat
* bucket.
* @param delegatedFS delegated FS with role-restricted access.
* @throws Exception failure
*/
@Override
protected void verifyRestrictedPermissions(final S3AFileSystem delegatedFS)
throws Exception {
intercept(AccessDeniedException.class,
() -> readLandsatMetadata(delegatedFS));
}
}

View File

@ -0,0 +1,122 @@
/*
* 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.auth.delegation;
import java.util.EnumSet;
import java.util.List;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.io.Text;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.probeForAssumedRoleARN;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_ROLE_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.E_NO_SESSION_TOKENS_FOR_ROLE_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.ROLE_TOKEN_KIND;
import static org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding.E_NO_ARN;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Rerun the session token tests with a role binding.
* Some tests will fail as role bindings prevent certain operations.
*/
public class ITestRoleDelegationTokens extends ITestSessionDelegationTokens {
private static final Logger LOG =
LoggerFactory.getLogger(ITestRoleDelegationTokens.class);
@Override
protected String getDelegationBinding() {
return DELEGATION_TOKEN_ROLE_BINDING;
}
@Override
public Text getTokenKind() {
return ROLE_TOKEN_KIND;
}
@Override
public void setup() throws Exception {
super.setup();
probeForAssumedRoleARN(getConfiguration());
}
/**
* Session credentials will not propagate with role tokens,
* so the superclass's method will fail.
* This subclass intercepts the exception which is expected.
* @param fs base FS to bond to.
* @param marshalledCredentials session credentials from first DT.
* @param conf config to use
* @return null
* @throws Exception failure
*/
@Override
protected AbstractS3ATokenIdentifier verifyCredentialPropagation(
final S3AFileSystem fs,
final MarshalledCredentials marshalledCredentials,
final Configuration conf) throws Exception {
intercept(DelegationTokenIOException.class,
E_NO_SESSION_TOKENS_FOR_ROLE_BINDING,
() -> super.verifyCredentialPropagation(fs,
marshalledCredentials, conf));
return null;
}
@Test
public void testBindingWithoutARN() throws Throwable {
describe("verify that a role binding only needs a role ARN when creating"
+ " a new token");
Configuration conf = new Configuration(getConfiguration());
conf.unset(DelegationConstants.DELEGATION_TOKEN_ROLE_ARN);
try (S3ADelegationTokens delegationTokens2 = new S3ADelegationTokens()) {
final S3AFileSystem fs = getFileSystem();
delegationTokens2.bindToFileSystem(fs.getUri(), fs);
delegationTokens2.init(conf);
delegationTokens2.start();
// cannot create a DT at this point
intercept(IllegalStateException.class,
E_NO_ARN,
() -> delegationTokens2.createDelegationToken(
new EncryptionSecrets()));
}
}
@Test
public void testCreateRoleModel() throws Throwable {
describe("self contained role model retrieval");
EnumSet<AWSPolicyProvider.AccessLevel> access
= EnumSet.of(
AWSPolicyProvider.AccessLevel.READ,
AWSPolicyProvider.AccessLevel.WRITE);
S3AFileSystem fs = getFileSystem();
List<RoleModel.Statement> rules = fs.listAWSPolicyRules(
access);
assertTrue("No AWS policy rules from FS", !rules.isEmpty());
String ruleset = new RoleModel().toJson(new RoleModel.Policy(rules));
LOG.info("Access policy for {}\n{}", fs.getUri(), ruleset);
}
}

View File

@ -0,0 +1,727 @@
/*
* 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.auth.delegation;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.net.URI;
import java.nio.file.AccessDeniedException;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.ObjectMetadata;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.DtUtilShell;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.service.ServiceOperations;
import org.apache.hadoop.service.ServiceStateException;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeSessionTestsEnabled;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.unsetHadoopCredentialProviders;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenIOException.TOKEN_MISMATCH;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken;
import static org.apache.hadoop.test.LambdaTestUtils.doAs;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.collection.IsCollectionWithSize.hasSize;
/**
* Tests use of Hadoop delegation tokens within the FS itself.
* This instantiates a MiniKDC as some of the operations tested require
* UGI to be initialized with security enabled.
*/
@SuppressWarnings("StaticNonFinalField")
public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
private static final Logger LOG =
LoggerFactory.getLogger(ITestSessionDelegationInFileystem.class);
private static MiniKerberizedHadoopCluster cluster;
private UserGroupInformation bobUser;
private UserGroupInformation aliceUser;
private S3ADelegationTokens delegationTokens;
/***
* Set up a mini Cluster with two users in the keytab.
*/
@BeforeClass
public static void setupCluster() throws Exception {
cluster = new MiniKerberizedHadoopCluster();
cluster.init(new Configuration());
cluster.start();
}
/**
* Tear down the Cluster.
*/
@SuppressWarnings("ThrowableNotThrown")
@AfterClass
public static void teardownCluster() throws Exception {
ServiceOperations.stopQuietly(LOG, cluster);
}
protected static MiniKerberizedHadoopCluster getCluster() {
return cluster;
}
/**
* Get the delegation token binding for this test suite.
* @return which DT binding to use.
*/
protected String getDelegationBinding() {
return DELEGATION_TOKEN_SESSION_BINDING;
}
/**
* Get the kind of the tokens which are generated.
* @return the kind of DT
*/
public Text getTokenKind() {
return SESSION_TOKEN_KIND;
}
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
// disable if assume role opts are off
assumeSessionTestsEnabled(conf);
disableFilesystemCaching(conf);
conf.set(HADOOP_SECURITY_AUTHENTICATION,
UserGroupInformation.AuthenticationMethod.KERBEROS.name());
enableDelegationTokens(conf, getDelegationBinding());
conf.set(AWS_CREDENTIALS_PROVIDER, " ");
// switch to SSE_S3.
if (conf.getBoolean(KEY_ENCRYPTION_TESTS, true)) {
conf.set(SERVER_SIDE_ENCRYPTION_ALGORITHM,
S3AEncryptionMethods.SSE_S3.getMethod());
}
// set the YARN RM up for YARN tests.
conf.set(YarnConfiguration.RM_PRINCIPAL, YARN_RM);
return conf;
}
@Override
public void setup() throws Exception {
// clear any existing tokens from the FS
resetUGI();
UserGroupInformation.setConfiguration(createConfiguration());
aliceUser = cluster.createAliceUser();
bobUser = cluster.createBobUser();
UserGroupInformation.setLoginUser(aliceUser);
assertSecurityEnabled();
// only now do the setup, so that any FS created is secure
super.setup();
S3AFileSystem fs = getFileSystem();
// make sure there aren't any tokens
assertNull("Unexpectedly found an S3A token",
lookupS3ADelegationToken(
UserGroupInformation.getCurrentUser().getCredentials(),
fs.getUri()));
// DTs are inited but not started.
delegationTokens = instantiateDTSupport(getConfiguration());
}
@SuppressWarnings("ThrowableNotThrown")
@Override
public void teardown() throws Exception {
super.teardown();
ServiceOperations.stopQuietly(LOG, delegationTokens);
FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser());
MiniKerberizedHadoopCluster.closeUserFileSystems(aliceUser);
MiniKerberizedHadoopCluster.closeUserFileSystems(bobUser);
cluster.resetUGI();
}
/**
* Are encryption tests enabled?
* @return true if encryption is turned on.
*/
protected boolean encryptionTestEnabled() {
return getConfiguration().getBoolean(KEY_ENCRYPTION_TESTS, true);
}
@Test
public void testGetDTfromFileSystem() throws Throwable {
describe("Enable delegation tokens and request one");
delegationTokens.start();
S3AFileSystem fs = getFileSystem();
assertNotNull("No tokens from " + fs,
fs.getCanonicalServiceName());
S3ATestUtils.MetricDiff invocationDiff = new S3ATestUtils.MetricDiff(fs,
Statistic.INVOCATION_GET_DELEGATION_TOKEN);
S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(fs,
Statistic.DELEGATION_TOKENS_ISSUED);
Token<AbstractS3ATokenIdentifier> token =
requireNonNull(fs.getDelegationToken(""),
"no token from filesystem " + fs);
assertEquals("token kind", getTokenKind(), token.getKind());
assertTokenCreationCount(fs, 1);
final String fsInfo = fs.toString();
invocationDiff.assertDiffEquals("getDelegationToken() in " + fsInfo,
1);
issueDiff.assertDiffEquals("DTs issued in " + delegationTokens,
1);
Text service = delegationTokens.getService();
assertEquals("service name", service, token.getService());
Credentials creds = new Credentials();
creds.addToken(service, token);
assertEquals("retrieve token from " + creds,
token, creds.getToken(service));
}
@Test
public void testAddTokensFromFileSystem() throws Throwable {
describe("verify FileSystem.addDelegationTokens() collects tokens");
S3AFileSystem fs = getFileSystem();
Credentials cred = new Credentials();
Token<?>[] tokens = fs.addDelegationTokens(YARN_RM, cred);
assertEquals("Number of tokens", 1, tokens.length);
Token<?> token = requireNonNull(tokens[0], "token");
LOG.info("FS token is {}", token);
Text service = delegationTokens.getService();
Token<? extends TokenIdentifier> retrieved = requireNonNull(
cred.getToken(service),
"retrieved token with key " + service + "; expected " + token);
delegationTokens.start();
// this only sneaks in because there isn't a state check here
delegationTokens.resetTokenBindingToDT(
(Token<AbstractS3ATokenIdentifier>) retrieved);
assertTrue("bind to existing DT failed",
delegationTokens.isBoundToDT());
AWSCredentialProviderList providerList = requireNonNull(
delegationTokens.getCredentialProviders(), "providers");
providerList.getCredentials();
}
@Test
public void testCanRetrieveTokenFromCurrentUserCreds() throws Throwable {
describe("Create a DT, add it to the current UGI credentials,"
+ " then retrieve");
delegationTokens.start();
Credentials cred = createDelegationTokens();
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
ugi.addCredentials(cred);
Token<?>[] tokens = cred.getAllTokens().toArray(new Token<?>[0]);
Token<?> token0 = tokens[0];
Text service = token0.getService();
LOG.info("Token = " + token0);
Token<?> token1 = requireNonNull(
ugi.getCredentials().getToken(service), "Token from " + service);
assertEquals("retrieved token", token0, token1);
assertNotNull("token identifier of " + token1,
token1.getIdentifier());
}
@Test
public void testDTCredentialProviderFromCurrentUserCreds() throws Throwable {
describe("Add credentials to the current user, "
+ "then verify that they can be found when S3ADelegationTokens binds");
Credentials cred = createDelegationTokens();
assertThat("Token size", cred.getAllTokens(), hasSize(1));
UserGroupInformation.getCurrentUser().addCredentials(cred);
delegationTokens.start();
assertTrue("bind to existing DT failed",
delegationTokens.isBoundToDT());
}
/**
* Create credentials with the DTs of the current FS.
* @return a non-empty set of credentials.
* @throws IOException failure to create.
*/
protected Credentials createDelegationTokens() throws IOException {
return mkTokens(getFileSystem());
}
/**
* Create a FS with a delegated token, verify it works as a filesystem,
* and that you can pick up the same DT from that FS too.
*/
@Test
public void testDelegatedFileSystem() throws Throwable {
describe("Delegation tokens can be passed to a new filesystem;"
+ " if role restricted, permissions are tightened.");
S3AFileSystem fs = getFileSystem();
readLandsatMetadata(fs);
URI uri = fs.getUri();
// create delegation tokens from the test suites FS.
Credentials creds = createDelegationTokens();
final Text tokenKind = getTokenKind();
AbstractS3ATokenIdentifier origTokenId = requireNonNull(
lookupToken(
creds,
uri,
tokenKind), "original");
// attach to the user, so that when tokens are looked for, they get picked
// up
final UserGroupInformation currentUser
= UserGroupInformation.getCurrentUser();
currentUser.addCredentials(creds);
// verify that the tokens went over
requireNonNull(lookupToken(
currentUser.getCredentials(),
uri,
tokenKind), "user credentials");
Configuration conf = new Configuration(getConfiguration());
String bucket = fs.getBucket();
disableFilesystemCaching(conf);
unsetHadoopCredentialProviders(conf);
// remove any secrets we don't want the delegated FS to accidentally
// pick up.
// this is to simulate better a remote deployment.
removeBaseAndBucketOverrides(bucket, conf,
ACCESS_KEY, SECRET_KEY, SESSION_TOKEN,
SERVER_SIDE_ENCRYPTION_ALGORITHM,
DELEGATION_TOKEN_ROLE_ARN,
DELEGATION_TOKEN_ENDPOINT);
// this is done to make sure you cannot create an STS session no
// matter how you pick up credentials.
conf.set(DELEGATION_TOKEN_ENDPOINT, "http://localhost:8080/");
bindProviderList(bucket, conf, CountInvocationsProvider.NAME);
long originalCount = CountInvocationsProvider.getInvocationCount();
// create a new FS instance, which is expected to pick up the
// existing token
Path testPath = path("testDTFileSystemClient");
try (S3AFileSystem delegatedFS = newS3AInstance(uri, conf)) {
LOG.info("Delegated filesystem is: {}", delegatedFS);
assertBoundToDT(delegatedFS, tokenKind);
if (encryptionTestEnabled()) {
assertEquals("Encryption propagation failed",
S3AEncryptionMethods.SSE_S3,
delegatedFS.getServerSideEncryptionAlgorithm());
}
verifyRestrictedPermissions(delegatedFS);
executeDelegatedFSOperations(delegatedFS, testPath);
delegatedFS.mkdirs(testPath);
S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(
delegatedFS,
Statistic.DELEGATION_TOKENS_ISSUED);
// verify that the FS returns the existing token when asked
// so that chained deployments will work
AbstractS3ATokenIdentifier tokenFromDelegatedFS
= requireNonNull(delegatedFS.getDelegationToken(""),
"New token").decodeIdentifier();
assertEquals("Newly issued token != old one",
origTokenId,
tokenFromDelegatedFS);
issueDiff.assertDiffEquals("DTs issued in " + delegatedFS,
0);
}
// the DT auth chain should override the original one.
assertEquals("invocation count",
originalCount,
CountInvocationsProvider.getInvocationCount());
// create a second instance, which will pick up the same value
try (S3AFileSystem secondDelegate = newS3AInstance(uri, conf)) {
assertBoundToDT(secondDelegate, tokenKind);
if (encryptionTestEnabled()) {
assertEquals("Encryption propagation failed",
S3AEncryptionMethods.SSE_S3,
secondDelegate.getServerSideEncryptionAlgorithm());
}
ContractTestUtils.assertDeleted(secondDelegate, testPath, true);
assertNotNull("unbounded DT",
secondDelegate.getDelegationToken(""));
}
}
/**
* Override/extension point: run operations within a delegated FS.
* @param delegatedFS filesystem.
* @param testPath path to work on.
* @throws IOException failures
*/
protected void executeDelegatedFSOperations(final S3AFileSystem delegatedFS,
final Path testPath) throws Exception {
ContractTestUtils.assertIsDirectory(delegatedFS, new Path("/"));
ContractTestUtils.touch(delegatedFS, testPath);
ContractTestUtils.assertDeleted(delegatedFS, testPath, false);
delegatedFS.mkdirs(testPath);
ContractTestUtils.assertIsDirectory(delegatedFS, testPath);
Path srcFile = new Path(testPath, "src.txt");
Path destFile = new Path(testPath, "dest.txt");
ContractTestUtils.touch(delegatedFS, srcFile);
ContractTestUtils.rename(delegatedFS, srcFile, destFile);
// this file is deleted afterwards, so leave alone
ContractTestUtils.assertIsFile(delegatedFS, destFile);
ContractTestUtils.assertDeleted(delegatedFS, testPath, true);
}
/**
* Session tokens can read the landsat bucket without problems.
* @param delegatedFS delegated FS
* @throws Exception failure
*/
protected void verifyRestrictedPermissions(final S3AFileSystem delegatedFS)
throws Exception {
readLandsatMetadata(delegatedFS);
}
@Test
public void testDelegationBindingMismatch1() throws Throwable {
describe("Verify that when the DT client and remote bindings are different,"
+ " the failure is meaningful");
S3AFileSystem fs = getFileSystem();
URI uri = fs.getUri();
UserGroupInformation.getCurrentUser().addCredentials(
createDelegationTokens());
// create the remote FS with a full credential binding
Configuration conf = new Configuration(getConfiguration());
String bucket = fs.getBucket();
removeBaseAndBucketOverrides(bucket, conf,
ACCESS_KEY, SECRET_KEY, SESSION_TOKEN);
conf.set(ACCESS_KEY, "aaaaa");
conf.set(SECRET_KEY, "bbbb");
bindProviderList(bucket, conf, CountInvocationsProvider.NAME);
conf.set(DELEGATION_TOKEN_BINDING,
DELEGATION_TOKEN_FULL_CREDENTIALS_BINDING);
ServiceStateException e = intercept(
ServiceStateException.class,
TOKEN_MISMATCH,
() -> {
S3AFileSystem remote = newS3AInstance(uri, conf);
// if we get this far, provide info for the exception which will
// be raised.
String s = remote.toString();
remote.close();
return s;
});
if (!(e.getCause() instanceof DelegationTokenIOException)) {
throw e;
}
}
@Test
public void testDelegationBindingMismatch2() throws Throwable {
describe("assert mismatch reported when client DT is a "
+ "subclass of the remote one");
S3AFileSystem fs = getFileSystem();
URI uri = fs.getUri();
// create the remote FS with a full credential binding
Configuration conf = new Configuration(getConfiguration());
String bucket = fs.getBucket();
enableDelegationTokens(conf, DELEGATION_TOKEN_FULL_CREDENTIALS_BINDING);
// create a new FS with Full tokens
Credentials fullTokens;
Token<AbstractS3ATokenIdentifier> firstDT;
try (S3AFileSystem fullFS = newS3AInstance(uri, conf)) {
// add the tokens to the user group
fullTokens = mkTokens(fullFS);
assertTokenCreationCount(fullFS, 1);
firstDT = fullFS.getDelegationToken(
"first");
assertTokenCreationCount(fullFS, 2);
Token<AbstractS3ATokenIdentifier> secondDT = fullFS.getDelegationToken(
"second");
assertTokenCreationCount(fullFS, 3);
assertNotEquals("DT identifiers",
firstDT.getIdentifier(), secondDT.getIdentifier());
}
// expect a token
AbstractS3ATokenIdentifier origTokenId = requireNonNull(
lookupToken(
fullTokens,
uri,
FULL_TOKEN_KIND), "token from credentials");
UserGroupInformation.getCurrentUser().addCredentials(
fullTokens);
// a remote FS with those tokens
try (S3AFileSystem delegatedFS = newS3AInstance(uri, conf)) {
assertBoundToDT(delegatedFS, FULL_TOKEN_KIND);
delegatedFS.getFileStatus(new Path("/"));
SessionTokenIdentifier tokenFromDelegatedFS
= (SessionTokenIdentifier) requireNonNull(
delegatedFS.getDelegationToken(""), "New token")
.decodeIdentifier();
assertTokenCreationCount(delegatedFS, 0);
assertEquals("Newly issued token != old one",
origTokenId,
tokenFromDelegatedFS);
}
// now create a configuration which expects a session token.
Configuration conf2 = new Configuration(getConfiguration());
removeBaseAndBucketOverrides(bucket, conf2,
ACCESS_KEY, SECRET_KEY, SESSION_TOKEN);
conf.set(DELEGATION_TOKEN_BINDING,
getDelegationBinding());
ServiceStateException e = intercept(ServiceStateException.class,
TOKEN_MISMATCH,
() -> {
S3AFileSystem remote = newS3AInstance(uri, conf);
// if we get this far, provide info for the exception which will
// be raised.
String s = remote.toString();
remote.close();
return s;
});
if (!(e.getCause() instanceof DelegationTokenIOException)) {
throw e;
}
}
/**
* This verifies that the granted credentials only access the target bucket
* by using the credentials in a new S3 client to query the AWS-owned landsat
* bucket.
* @param delegatedFS delegated FS with role-restricted access.
* @throws AccessDeniedException if the delegated FS's credentials can't
* access the bucket.
* @return result of the HEAD
* @throws Exception failure
*/
protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS)
throws Exception {
AWSCredentialProviderList testing
= delegatedFS.shareCredentials("testing");
URI landsat = new URI(DEFAULT_CSVTEST_FILE);
DefaultS3ClientFactory factory
= new DefaultS3ClientFactory();
Configuration conf = new Configuration(delegatedFS.getConf());
conf.set(ENDPOINT, "");
factory.setConf(conf);
String host = landsat.getHost();
AmazonS3 s3 = factory.createS3Client(landsat, host, testing,
"ITestSessionDelegationInFileystem");
return Invoker.once("HEAD", host,
() -> s3.getObjectMetadata(host, landsat.getPath().substring(1)));
}
/**
* YARN job submission uses
* {@link TokenCache#obtainTokensForNamenodes(Credentials, Path[], Configuration)}
* for token retrieval: call it here to verify it works.
*/
@Test
public void testYarnCredentialPickup() throws Throwable {
describe("Verify tokens are picked up by the YARN"
+ " TokenCache.obtainTokensForNamenodes() API Call");
Credentials cred = new Credentials();
Path yarnPath = path("testYarnCredentialPickup");
Path[] paths = new Path[] {yarnPath};
Configuration conf = getConfiguration();
S3AFileSystem fs = getFileSystem();
TokenCache.obtainTokensForNamenodes(cred, paths, conf);
assertNotNull("No Token in credentials file",
lookupToken(
cred,
fs.getUri(),
getTokenKind()));
}
/**
* Test the {@code hdfs fetchdt} command works with S3A tokens.
*/
@Test
public void testHDFSFetchDTCommand() throws Throwable {
describe("Use the HDFS fetchdt CLI to fetch a token");
ExitUtil.disableSystemExit();
S3AFileSystem fs = getFileSystem();
Configuration conf = fs.getConf();
URI fsUri = fs.getUri();
String fsurl = fsUri.toString();
File tokenfile = createTempTokenFile();
// this will create (& leak) a new FS instance as caching is disabled.
// but as teardown destroys all filesystems for this user, it
// gets cleaned up at the end of the test
String tokenFilePath = tokenfile.getAbsolutePath();
// create the tokens as Bob.
doAs(bobUser,
() -> DelegationTokenFetcher.main(conf,
args("--webservice", fsurl, tokenFilePath)));
assertTrue("token file was not created: " + tokenfile,
tokenfile.exists());
// print to stdout
String s = DelegationTokenFetcher.printTokensToString(conf,
new Path(tokenfile.toURI()),
false);
LOG.info("Tokens: {}", s);
DelegationTokenFetcher.main(conf,
args("--print", tokenFilePath));
DelegationTokenFetcher.main(conf,
args("--print", "--verbose", tokenFilePath));
// read in and retrieve token
Credentials creds = Credentials.readTokenStorageFile(tokenfile, conf);
AbstractS3ATokenIdentifier identifier = requireNonNull(
lookupToken(
creds,
fsUri,
getTokenKind()), "Token lookup");
assertEquals("encryption secrets",
fs.getEncryptionSecrets(),
identifier.getEncryptionSecrets());
assertEquals("Username of decoded token",
bobUser.getUserName(), identifier.getUser().getUserName());
// renew
DelegationTokenFetcher.main(conf, args("--renew", tokenFilePath));
// cancel
DelegationTokenFetcher.main(conf, args("--cancel", tokenFilePath));
}
protected File createTempTokenFile() throws IOException {
File tokenfile = File.createTempFile("tokens", ".bin",
cluster.getWorkDir());
tokenfile.delete();
return tokenfile;
}
/**
* Convert a vargs list to an array.
* @param args vararg list of arguments
* @return the generated array.
*/
private String[] args(String...args) {
return args;
}
/**
* This test looks at the identity which goes with a DT.
* It assumes that the username of a token == the user who created it.
* Some tokens may change that in future (maybe use Role ARN?).
*/
@Test
public void testFileSystemBoundToCreator() throws Throwable {
describe("Run tests to verify the DT Setup is bound to the creator");
// quick sanity check to make sure alice and bob are different
assertNotEquals("Alice and Bob logins",
aliceUser.getUserName(), bobUser.getUserName());
final S3AFileSystem fs = getFileSystem();
assertEquals("FS username in doAs()",
ALICE,
doAs(bobUser, () -> fs.getUsername()));
UserGroupInformation fsOwner = doAs(bobUser,
() -> fs.getDelegationTokens().get().getOwner());
assertEquals("username mismatch",
aliceUser.getUserName(), fsOwner.getUserName());
Token<AbstractS3ATokenIdentifier> dt = fs.getDelegationToken(ALICE);
AbstractS3ATokenIdentifier identifier
= dt.decodeIdentifier();
UserGroupInformation user = identifier.getUser();
assertEquals("User in DT",
aliceUser.getUserName(), user.getUserName());
}
protected String dtutil(int expected, String...args) throws Exception {
final ByteArrayOutputStream dtUtilContent = new ByteArrayOutputStream();
DtUtilShell dt = new DtUtilShell();
dt.setOut(new PrintStream(dtUtilContent));
dtUtilContent.reset();
int r = doAs(aliceUser,
() ->ToolRunner.run(getConfiguration(), dt, args));
String s = dtUtilContent.toString();
LOG.info("\n{}", s);
assertEquals(expected, r);
return s;
}
@Test
public void testDTUtilShell() throws Throwable {
describe("Verify the dtutil shell command can fetch tokens");
File tokenfile = createTempTokenFile();
String tfs = tokenfile.toString();
String fsURI = getFileSystem().getCanonicalUri().toString();
dtutil(0,
"get", fsURI,
"-format", "protobuf",
tfs);
assertTrue("not created: " + tokenfile,
tokenfile.exists());
assertTrue("File is empty" + tokenfile,
tokenfile.length() > 0);
assertTrue("File only contains header" + tokenfile,
tokenfile.length() > 6);
String printed = dtutil(0, "print", tfs);
assertThat(printed, containsString(fsURI));
assertThat(printed, containsString(getTokenKind().toString()));
}
}

View File

@ -0,0 +1,282 @@
/*
* 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.auth.delegation;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSSessionCredentials;
import org.hamcrest.Matchers;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeSessionTestsEnabled;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.roundTrip;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.unsetHadoopCredentialProviders;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromAWSCredentials;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_SESSION_BINDING;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.SESSION_TOKEN_KIND;
import static org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding.CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN;
/**
* Tests use of Hadoop delegation tokens to marshall S3 credentials.
*/
public class ITestSessionDelegationTokens extends AbstractDelegationIT {
private static final Logger LOG =
LoggerFactory.getLogger(ITestSessionDelegationTokens.class);
public static final String KMS_KEY = "arn:kms:key";
private S3ADelegationTokens delegationTokens;
/**
* Get the delegation token binding for this test suite.
* @return which DT binding to use.
*/
protected String getDelegationBinding() {
return DELEGATION_TOKEN_SESSION_BINDING;
}
public Text getTokenKind() {
return SESSION_TOKEN_KIND;
}
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
enableDelegationTokens(conf, getDelegationBinding());
return conf;
}
@Override
public void setup() throws Exception {
super.setup();
assumeSessionTestsEnabled(getConfiguration());
resetUGI();
delegationTokens = instantiateDTSupport(getConfiguration());
delegationTokens.start();
}
@Override
public void teardown() throws Exception {
IOUtils.cleanupWithLogger(LOG, delegationTokens);
resetUGI();
super.teardown();
}
/**
* Checks here to catch any regressions in canonicalization
* logic.
*/
@Test
public void testCanonicalization() throws Throwable {
S3AFileSystem fs = getFileSystem();
assertEquals("Default port has changed",
0, fs.getDefaultPort());
URI uri = fs.getCanonicalUri();
String service = fs.getCanonicalServiceName();
assertEquals("canonical URI and service name mismatch",
uri, new URI(service));
}
@Test
public void testSaveLoadTokens() throws Throwable {
File tokenFile = File.createTempFile("token", "bin");
EncryptionSecrets encryptionSecrets = new EncryptionSecrets(
S3AEncryptionMethods.SSE_KMS, KMS_KEY);
Token<AbstractS3ATokenIdentifier> dt
= delegationTokens.createDelegationToken(encryptionSecrets);
final SessionTokenIdentifier origIdentifier
= (SessionTokenIdentifier) dt.decodeIdentifier();
assertEquals("kind in " + dt, getTokenKind(), dt.getKind());
Configuration conf = getConfiguration();
saveDT(tokenFile, dt);
assertTrue("Empty token file", tokenFile.length() > 0);
Credentials creds = Credentials.readTokenStorageFile(tokenFile, conf);
Text serviceId = delegationTokens.getService();
Token<? extends TokenIdentifier> token = requireNonNull(
creds.getToken(serviceId),
() -> "No token for \"" + serviceId + "\" in: " + creds.getAllTokens());
SessionTokenIdentifier decoded =
(SessionTokenIdentifier) token.decodeIdentifier();
decoded.validate();
assertEquals("token identifier ", origIdentifier, decoded);
assertEquals("Origin in " + decoded,
origIdentifier.getOrigin(), decoded.getOrigin());
assertEquals("Expiry time",
origIdentifier.getExpiryTime(), decoded.getExpiryTime());
assertEquals("Encryption Secrets",
encryptionSecrets, decoded.getEncryptionSecrets());
}
/**
* This creates a DT from a set of credentials, then verifies
* that you can use the round-tripped credentials as a source of
* authentication for another DT binding, and when
* that is asked for a DT token, the secrets it returns are
* the same as the original.
*
* That is different from DT propagation, as here the propagation
* is by setting the fs.s3a session/secret/id keys from the marshalled
* values, and using session token auth.
* This verifies that session token authentication can be used
* for DT credential auth, and that new tokens aren't created.
*
* From a testing perspective, this is not as "good" as having
* separate tests, but given the effort to create session tokens
* is all hidden in the first FS, it is actually easier to write
* and now forms an extra test on those generated tokens as well
* as the marshalling.
*/
@Test
public void testCreateAndUseDT() throws Throwable {
describe("Create a Delegation Token, round trip then reuse");
final S3AFileSystem fs = getFileSystem();
final Configuration conf = fs.getConf();
assertNull("Current User has delegation token",
delegationTokens.selectTokenFromFSOwner());
EncryptionSecrets secrets = new EncryptionSecrets(
S3AEncryptionMethods.SSE_KMS, KMS_KEY);
Token<AbstractS3ATokenIdentifier> originalDT
= delegationTokens.createDelegationToken(secrets);
assertEquals("Token kind mismatch", getTokenKind(), originalDT.getKind());
// decode to get the binding info
SessionTokenIdentifier issued =
requireNonNull(
(SessionTokenIdentifier) originalDT.decodeIdentifier(),
() -> "no identifier in " + originalDT);
issued.validate();
final MarshalledCredentials creds;
try(S3ADelegationTokens dt2 = instantiateDTSupport(getConfiguration())) {
dt2.start();
dt2.resetTokenBindingToDT(originalDT);
final AWSSessionCredentials awsSessionCreds
= verifySessionCredentials(
dt2.getCredentialProviders().getCredentials());
final MarshalledCredentials origCreds = fromAWSCredentials(
awsSessionCreds);
Token<AbstractS3ATokenIdentifier> boundDT =
dt2.getBoundOrNewDT(secrets);
assertEquals("Delegation Tokens", originalDT, boundDT);
// simulate marshall and transmission
creds = roundTrip(origCreds, conf);
SessionTokenIdentifier reissued
= (SessionTokenIdentifier) dt2.createDelegationToken(secrets)
.decodeIdentifier();
reissued.validate();
String userAgentField = dt2.getUserAgentField();
assertThat("UA field does not contain UUID",
userAgentField,
Matchers.containsString(issued.getUuid()));
}
// now use those chained credentials to create a new FS instance
// and then get a session DT from it and expect equality
verifyCredentialPropagation(fs, creds, new Configuration(conf));
}
/**
* This verifies that AWS Session credentials can be picked up and
* returned in a DT.
* With a session binding, this holds; for role binding it will fail.
* @param fs base FS to bond to.
* @param session session credentials from first DT.
* @param conf config to use
* @return the retrieved DT. This is only for error reporting.
* @throws IOException failure.
*/
@SuppressWarnings("OptionalGetWithoutIsPresent")
protected AbstractS3ATokenIdentifier verifyCredentialPropagation(
final S3AFileSystem fs,
final MarshalledCredentials session,
final Configuration conf)
throws Exception {
describe("Verify Token Propagation");
// clear any credential paths to ensure they don't get picked up and used
// for authentication.
unsetHadoopCredentialProviders(conf);
conf.set(DELEGATION_TOKEN_CREDENTIALS_PROVIDER,
TemporaryAWSCredentialsProvider.NAME);
session.setSecretsInConfiguration(conf);
try(S3ADelegationTokens delegationTokens2 = new S3ADelegationTokens()) {
delegationTokens2.bindToFileSystem(fs.getCanonicalUri(), fs);
delegationTokens2.init(conf);
delegationTokens2.start();
final Token<AbstractS3ATokenIdentifier> newDT
= delegationTokens2.getBoundOrNewDT(new EncryptionSecrets());
delegationTokens2.resetTokenBindingToDT(newDT);
final AbstractS3ATokenIdentifier boundId
= delegationTokens2.getDecodedIdentifier().get();
LOG.info("Regenerated DT is {}", newDT);
final MarshalledCredentials creds2 = fromAWSCredentials(
verifySessionCredentials(
delegationTokens2.getCredentialProviders().getCredentials()));
assertEquals("Credentials", session, creds2);
assertTrue("Origin in " + boundId,
boundId.getOrigin()
.contains(CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN));
return boundId;
}
}
private AWSSessionCredentials verifySessionCredentials(
final AWSCredentials creds) {
AWSSessionCredentials session = (AWSSessionCredentials) creds;
assertNotNull("access key", session.getAWSAccessKeyId());
assertNotNull("secret key", session.getAWSSecretKey());
assertNotNull("session token", session.getSessionToken());
return session;
}
@Test
public void testDBindingReentrancyLock() throws Throwable {
describe("Verify that S3ADelegationTokens cannot be bound twice when there"
+ " is no token");
S3ADelegationTokens delegation = instantiateDTSupport(getConfiguration());
delegation.start();
assertFalse("Delegation is bound to a DT: " + delegation,
delegation.isBoundToDT());
}
}

View File

@ -0,0 +1,378 @@
/*
* 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.auth.delegation;
import java.io.File;
import java.io.IOException;
import java.util.Properties;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.KDiag;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
import static org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig.DEFAULT_MR_HISTORY_PORT;
import static org.apache.hadoop.security.UserGroupInformation.loginUserFromKeytabAndReturnUGI;
import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
import static org.junit.Assert.assertTrue;
/**
* This is intended to support setting up an mini-secure Hadoop + YARN + MR
* cluster.
* It does not do this, yet, for the following reason: things don't work.
* It is designed to be started/stopped at the class level.
* however, users should be logged in in test cases, so that their local state
* (credentials etc) are reset in every test.
*/
public class MiniKerberizedHadoopCluster extends CompositeService {
private static final Logger LOG =
LoggerFactory.getLogger(MiniKerberizedHadoopCluster.class);
public static final String ALICE = "alice";
public static final String BOB = "bob";
public static final String HTTP_LOCALHOST = "HTTP/localhost@$LOCALHOST";
/**
* The hostname is dynamically determined based on OS, either
* "localhost" (non-windows) or 127.0.0.1 (windows).
*/
public static final String LOCALHOST_NAME = Path.WINDOWS
? "127.0.0.1"
: "localhost";
private MiniKdc kdc;
private File keytab;
private File workDir;
private String krbInstance;
private String loginUsername;
private String loginPrincipal;
private String sslConfDir;
private String clientSSLConfigFileName;
private String serverSSLConfigFileName;
private String alicePrincipal;
private String bobPrincipal;
/**
* Create the cluster.
* If this class's log is at DEBUG level, this also turns
* Kerberos diagnostics on in the JVM.
*/
public MiniKerberizedHadoopCluster() {
super("MiniKerberizedHadoopCluster");
// load all the configs to force in the -default.xml files
new HdfsConfiguration();
new YarnConfiguration();
new JobConf();
if (LOG.isDebugEnabled()) {
// turn on kerberos logging @ debug.
System.setProperty(KDiag.SUN_SECURITY_KRB5_DEBUG, "true");
System.setProperty(KDiag.SUN_SECURITY_SPNEGO_DEBUG, "true");
}
}
public MiniKdc getKdc() {
return kdc;
}
public File getKeytab() {
return keytab;
}
public String getKeytabPath() {
return keytab.getAbsolutePath();
}
public UserGroupInformation createBobUser() throws IOException {
return loginUserFromKeytabAndReturnUGI(bobPrincipal,
keytab.getAbsolutePath());
}
public UserGroupInformation createAliceUser() throws IOException {
return loginUserFromKeytabAndReturnUGI(alicePrincipal,
keytab.getAbsolutePath());
}
public File getWorkDir() {
return workDir;
}
public String getKrbInstance() {
return krbInstance;
}
public String getLoginUsername() {
return loginUsername;
}
public String getLoginPrincipal() {
return loginPrincipal;
}
public String withRealm(String user) {
return user + "@EXAMPLE.COM";
}
/**
* Service init creates the KDC.
* @param conf configuration
*/
@Override
protected void serviceInit(final Configuration conf) throws Exception {
patchConfigAtInit(conf);
super.serviceInit(conf);
Properties kdcConf = MiniKdc.createConf();
workDir = GenericTestUtils.getTestDir("kerberos");
workDir.mkdirs();
kdc = new MiniKdc(kdcConf, workDir);
krbInstance = LOCALHOST_NAME;
}
/**
* Start the KDC, create the keytab and the alice and bob users,
* and UGI instances of them logged in from the keytab.
*/
@Override
protected void serviceStart() throws Exception {
super.serviceStart();
kdc.start();
keytab = new File(workDir, "keytab.bin");
loginUsername = UserGroupInformation.getLoginUser().getShortUserName();
loginPrincipal = loginUsername + "/" + krbInstance;
alicePrincipal = ALICE + "/" + krbInstance;
bobPrincipal = BOB + "/" + krbInstance;
kdc.createPrincipal(keytab,
alicePrincipal,
bobPrincipal,
"HTTP/" + krbInstance,
HTTP_LOCALHOST,
loginPrincipal);
final File keystoresDir = new File(workDir, "ssl");
keystoresDir.mkdirs();
sslConfDir = KeyStoreTestUtil.getClasspathDir(
this.getClass());
KeyStoreTestUtil.setupSSLConfig(keystoresDir.getAbsolutePath(),
sslConfDir, getConfig(), false);
clientSSLConfigFileName = KeyStoreTestUtil.getClientSSLConfigFileName();
serverSSLConfigFileName = KeyStoreTestUtil.getServerSSLConfigFileName();
}
@Override
protected void serviceStop() throws Exception {
super.serviceStop();
// this can throw an exception, but it will get caught by the superclass.
kdc.stop();
}
protected void patchConfigAtInit(final Configuration conf) {
// turn off some noise during debugging
int timeout = 60 * 60_1000;
conf.setInt("jvm.pause.info-threshold.ms", timeout);
conf.setInt("jvm.pause.warn-threshold.ms", timeout);
}
/**
* Set up HDFS to run securely.
* In secure mode, HDFS goes out of its way to refuse to start if it
* doesn't consider the configuration safe.
* This is good in production, and it stops an HDFS cluster coming
* up where things can't reliably talk to each other.
* But it does complicate test setup.
* Look at {@code org.apache.hadoop.hdfs.TestDFSInotifyEventInputStreamKerberized}
* and {@code org.apache.hadoop.hdfs.qjournal.TestSecureNNWithQJM}
* for the details on what options to set here.
* @param conf configuration to patch.
*/
protected void patchConfigWithHDFSBindings(final Configuration conf) {
Preconditions.checkState(isInState(STATE.STARTED));
enableKerberos(conf);
String path = getKeytabPath();
String spnegoPrincipal = "*";
String localhost = LOCALHOST_NAME;
String instance = getKrbInstance();
String hdfsPrincipal = getLoginPrincipal();
patchConfigAtInit(conf);
conf.setLong(CommonConfigurationKeys.FS_DU_INTERVAL_KEY, Long.MAX_VALUE);
conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, path);
conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, path);
conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
conf.set(DFS_JOURNALNODE_KEYTAB_FILE_KEY, path);
conf.set(DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
conf.set(DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
spnegoPrincipal);
conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
conf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
conf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
conf.set(DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,
KeyStoreTestUtil.getClientSSLConfigFileName());
conf.set(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
KeyStoreTestUtil.getServerSSLConfigFileName());
}
/**
* Patch the YARN settings.
* Note how the yarn principal has to include the realm.
* @param conf configuration to patch.
*/
protected void patchConfigWithYARNBindings(final Configuration conf) {
Preconditions.checkState(isInState(STATE.STARTED));
enableKerberos(conf);
patchConfigAtInit(conf);
String path = getKeytabPath();
String localhost = LOCALHOST_NAME;
String yarnPrincipal = withRealm(getLoginPrincipal());
conf.set(RM_PRINCIPAL, yarnPrincipal);
conf.set(RM_KEYTAB, path);
conf.set(RM_HOSTNAME, localhost);
conf.set(RM_BIND_HOST, localhost);
conf.set(RM_ADDRESS,
localhost + ":" + DEFAULT_RM_PORT);
conf.set(NM_PRINCIPAL, yarnPrincipal);
conf.set(NM_KEYTAB, path);
conf.set(NM_ADDRESS,
localhost + ":" + DEFAULT_NM_PORT);
conf.setBoolean(TIMELINE_SERVICE_ENABLED, false);
conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, false);
conf.set(JHAdminConfig.MR_HISTORY_KEYTAB, path);
conf.set(JHAdminConfig.MR_HISTORY_PRINCIPAL, yarnPrincipal);
conf.set(JHAdminConfig.MR_HISTORY_ADDRESS,
localhost + ":" + DEFAULT_MR_HISTORY_PORT);
conf.setBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, false);
conf.setInt(RM_AM_MAX_ATTEMPTS, 1);
conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
100);
conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
10_000);
}
public void resetUGI() {
UserGroupInformation.reset();
}
/**
* Given a shortname, built a long name with the krb instance and realm info.
* @param shortname short name of the user
* @return a long name
*/
private String userOnHost(final String shortname) {
return shortname + "/" + krbInstance + "@" + getRealm();
}
public String getRealm() {
return kdc.getRealm();
}
/**
* Log in a user to UGI.currentUser.
* @param user user to log in from
* @throws IOException failure
*/
public void loginUser(final String user) throws IOException {
UserGroupInformation.loginUserFromKeytab(user, getKeytabPath());
}
/**
* Log in the login principal as the current user.
* @throws IOException failure
*/
public void loginPrincipal() throws IOException {
loginUser(getLoginPrincipal());
}
/**
* General assertion that security is turred on for a cluster.
*/
public static void assertSecurityEnabled() {
assertTrue("Security is needed for this test",
UserGroupInformation.isSecurityEnabled());
}
/**
* Close filesystems for a user, downgrading a null user to a no-op.
* @param ugi user
* @throws IOException if a close operation raised one.
*/
public static void closeUserFileSystems(UserGroupInformation ugi)
throws IOException {
if (ugi != null) {
FileSystem.closeAllForUGI(ugi);
}
}
/**
* Modify a configuration to use Kerberos as the auth method.
* @param conf configuration to patch.
*/
public static void enableKerberos(Configuration conf) {
conf.set(HADOOP_SECURITY_AUTHENTICATION,
UserGroupInformation.AuthenticationMethod.KERBEROS.name());
}
}

View File

@ -0,0 +1,171 @@
/*
* 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.auth.delegation;
import java.net.URI;
import org.junit.BeforeClass;
import org.junit.Test;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.Token;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.FULL_TOKEN_KIND;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.SESSION_TOKEN_KIND;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
/**
* Unit tests related to S3A DT support.
*/
public class TestS3ADelegationTokenSupport {
private static URI landsatUri;
@BeforeClass
public static void classSetup() throws Exception {
landsatUri = new URI(S3ATestConstants.DEFAULT_CSVTEST_FILE);
}
@Test
public void testSessionTokenKind() throws Throwable {
AbstractS3ATokenIdentifier identifier
= new SessionTokenIdentifier();
assertEquals(SESSION_TOKEN_KIND, identifier.getKind());
}
@Test
public void testSessionTokenDecode() throws Throwable {
Text alice = new Text("alice");
AbstractS3ATokenIdentifier identifier
= new SessionTokenIdentifier(SESSION_TOKEN_KIND,
alice,
new URI("s3a://landsat-pds/"),
new MarshalledCredentials("a", "b", ""),
new EncryptionSecrets(S3AEncryptionMethods.SSE_S3, ""),
"origin");
Token<AbstractS3ATokenIdentifier> t1 =
new Token<>(identifier,
new SessionSecretManager());
AbstractS3ATokenIdentifier decoded = t1.decodeIdentifier();
decoded.validate();
MarshalledCredentials creds
= ((SessionTokenIdentifier) decoded).getMarshalledCredentials();
assertNotNull("credentials",
MarshalledCredentialBinding.toAWSCredentials(creds,
MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty, ""));
assertEquals(alice, decoded.getOwner());
UserGroupInformation decodedUser = decoded.getUser();
assertEquals("name of " + decodedUser,
"alice",
decodedUser.getUserName());
assertEquals("Authentication method of " + decodedUser,
UserGroupInformation.AuthenticationMethod.TOKEN,
decodedUser.getAuthenticationMethod());
assertEquals("origin", decoded.getOrigin());
}
@Test
public void testFullTokenKind() throws Throwable {
AbstractS3ATokenIdentifier identifier
= new FullCredentialsTokenIdentifier();
assertEquals(FULL_TOKEN_KIND, identifier.getKind());
}
@Test
public void testSessionTokenIdentifierRoundTrip() throws Throwable {
SessionTokenIdentifier id = new SessionTokenIdentifier(
SESSION_TOKEN_KIND,
new Text(),
landsatUri,
new MarshalledCredentials("a", "b", "c"),
new EncryptionSecrets(), "");
SessionTokenIdentifier result = S3ATestUtils.roundTrip(id, null);
String ids = id.toString();
assertEquals("URI in " + ids, id.getUri(), result.getUri());
assertEquals("credentials in " + ids,
id.getMarshalledCredentials(),
result.getMarshalledCredentials());
}
@Test
public void testRoleTokenIdentifierRoundTrip() throws Throwable {
RoleTokenIdentifier id = new RoleTokenIdentifier(
landsatUri,
new Text(),
new MarshalledCredentials("a", "b", "c"),
new EncryptionSecrets(), "");
RoleTokenIdentifier result = S3ATestUtils.roundTrip(id, null);
String ids = id.toString();
assertEquals("URI in " + ids, id.getUri(), result.getUri());
assertEquals("credentials in " + ids,
id.getMarshalledCredentials(),
result.getMarshalledCredentials());
}
@Test
public void testFullTokenIdentifierRoundTrip() throws Throwable {
FullCredentialsTokenIdentifier id = new FullCredentialsTokenIdentifier(
landsatUri,
new Text(),
new MarshalledCredentials("a", "b", ""),
new EncryptionSecrets(), "");
FullCredentialsTokenIdentifier result = S3ATestUtils.roundTrip(id, null);
String ids = id.toString();
assertEquals("URI in " + ids, id.getUri(), result.getUri());
assertEquals("credentials in " + ids,
id.getMarshalledCredentials(),
result.getMarshalledCredentials());
}
/**
* The secret manager always uses the same secret; the
* factory for new identifiers is that of the token manager.
*/
private class SessionSecretManager
extends SecretManager<AbstractS3ATokenIdentifier> {
@Override
protected byte[] createPassword(AbstractS3ATokenIdentifier identifier) {
return "PASSWORD".getBytes();
}
@Override
public byte[] retrievePassword(AbstractS3ATokenIdentifier identifier)
throws InvalidToken {
return "PASSWORD".getBytes();
}
@Override
public AbstractS3ATokenIdentifier createIdentifier() {
return new SessionTokenIdentifier();
}
}
}

View File

@ -56,9 +56,9 @@
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.service.ServiceOperations;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.terminateService;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
@ -88,22 +88,17 @@ public static void setupClusters() throws IOException {
conf.setBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, false);
conf.setLong(CommonConfigurationKeys.FS_DU_INTERVAL_KEY, Long.MAX_VALUE);
hdfs = new MiniDFSClusterService();
hdfs.init(conf);
hdfs.start();
yarn = new MiniMRYarnCluster("ITCommitMRJob", 2);
yarn.init(conf);
yarn.start();
hdfs = deployService(conf, new MiniDFSClusterService());
yarn = deployService(conf,
new MiniMRYarnCluster("ITCommitMRJob", 2));
}
@SuppressWarnings("ThrowableNotThrown")
@AfterClass
public static void teardownClusters() throws IOException {
conf = null;
ServiceOperations.stopQuietly(yarn);
ServiceOperations.stopQuietly(hdfs);
hdfs = null;
yarn = null;
yarn = terminateService(yarn);
hdfs = terminateService(hdfs);
}
public static MiniDFSCluster getHdfs() {

View File

@ -97,9 +97,16 @@ public class StagingTestBase {
public static final String BUCKET = MockS3AFileSystem.BUCKET;
public static final String OUTPUT_PREFIX = "output/path";
public static final Path OUTPUT_PATH =
/** The raw bucket URI Path before any canonicalization. */
public static final Path RAW_BUCKET_PATH =
new Path("s3a://" + BUCKET + "/");
/** The raw bucket URI Path before any canonicalization. */
public static final URI RAW_BUCKET_URI =
RAW_BUCKET_PATH.toUri();
public static Path outputPath =
new Path("s3a://" + BUCKET + "/" + OUTPUT_PREFIX);
public static final URI OUTPUT_PATH_URI = OUTPUT_PATH.toUri();
public static URI outputPathUri = outputPath.toUri();
public static Path root;
protected StagingTestBase() {
}
@ -119,8 +126,11 @@ protected static S3AFileSystem createAndBindMockFSInstance(Configuration conf,
throws IOException {
S3AFileSystem mockFs = mockS3AFileSystemRobustly();
MockS3AFileSystem wrapperFS = new MockS3AFileSystem(mockFs, outcome);
URI uri = OUTPUT_PATH_URI;
URI uri = RAW_BUCKET_URI;
wrapperFS.initialize(uri, conf);
root = wrapperFS.makeQualified(new Path("/"));
outputPath = new Path(root, OUTPUT_PREFIX);
outputPathUri = outputPath.toUri();
FileSystemTestHelper.addFileSystemForTesting(uri, conf, wrapperFS);
return mockFs;
}
@ -142,7 +152,7 @@ private static S3AFileSystem mockS3AFileSystemRobustly() {
*/
public static MockS3AFileSystem lookupWrapperFS(Configuration conf)
throws IOException {
return (MockS3AFileSystem) FileSystem.get(OUTPUT_PATH_URI, conf);
return (MockS3AFileSystem) FileSystem.get(outputPathUri, conf);
}
public static void verifyCompletion(FileSystem mockS3) throws IOException {
@ -157,13 +167,13 @@ public static void verifyDeleted(FileSystem mockS3, Path path)
public static void verifyDeleted(FileSystem mockS3, String child)
throws IOException {
verifyDeleted(mockS3, new Path(OUTPUT_PATH, child));
verifyDeleted(mockS3, new Path(outputPath, child));
}
public static void verifyCleanupTempFiles(FileSystem mockS3)
throws IOException {
verifyDeleted(mockS3,
new Path(OUTPUT_PATH, CommitConstants.TEMPORARY));
new Path(outputPath, CommitConstants.TEMPORARY));
}
protected static void assertConflictResolution(
@ -177,7 +187,7 @@ protected static void assertConflictResolution(
public static void pathsExist(FileSystem mockS3, String... children)
throws IOException {
for (String child : children) {
pathExists(mockS3, new Path(OUTPUT_PATH, child));
pathExists(mockS3, new Path(outputPath, child));
}
}
@ -194,7 +204,7 @@ public static void pathDoesNotExist(FileSystem mockS3, Path path)
public static void canDelete(FileSystem mockS3, String... children)
throws IOException {
for (String child : children) {
canDelete(mockS3, new Path(OUTPUT_PATH, child));
canDelete(mockS3, new Path(outputPath, child));
}
}
@ -206,7 +216,7 @@ public static void canDelete(FileSystem mockS3, Path f) throws IOException {
public static void verifyExistenceChecked(FileSystem mockS3, String child)
throws IOException {
verifyExistenceChecked(mockS3, new Path(OUTPUT_PATH, child));
verifyExistenceChecked(mockS3, new Path(outputPath, child));
}
public static void verifyExistenceChecked(FileSystem mockS3, Path path)

View File

@ -157,7 +157,7 @@ public void setupCommitter() throws Exception {
this.tac = new TaskAttemptContextImpl(
new Configuration(job.getConfiguration()), AID);
this.jobCommitter = new MockedStagingCommitter(OUTPUT_PATH, tac);
this.jobCommitter = new MockedStagingCommitter(outputPath, tac);
jobCommitter.setupJob(job);
// get the task's configuration copy so modifications take effect
@ -172,7 +172,7 @@ public void setupCommitter() throws Exception {
this.conf.set(BUFFER_DIR,
String.format("%s/local-0/, %s/local-1 ", tmp, tmp));
this.committer = new MockedStagingCommitter(OUTPUT_PATH, tac);
this.committer = new MockedStagingCommitter(outputPath, tac);
Paths.resetTempFolderCache();
}
@ -608,7 +608,7 @@ private Set<String> runTasks(JobContext jobContext,
TaskAttemptContext attempt = new TaskAttemptContextImpl(
new Configuration(jobContext.getConfiguration()), attemptID);
MockedStagingCommitter taskCommitter = new MockedStagingCommitter(
OUTPUT_PATH, attempt);
outputPath, attempt);
commitTask(taskCommitter, attempt, numFiles);
}

View File

@ -36,7 +36,7 @@ public class TestStagingDirectoryOutputCommitter
@Override
DirectoryStagingCommitter newJobCommitter() throws Exception {
return new DirectoryStagingCommitter(OUTPUT_PATH,
return new DirectoryStagingCommitter(outputPath,
createTaskAttemptForJob());
}
@ -64,7 +64,7 @@ public void testFailConflictResolution() throws Exception {
protected void verifyFailureConflictOutcome() throws Exception {
FileSystem mockS3 = getMockS3A();
pathExists(mockS3, OUTPUT_PATH);
pathExists(mockS3, outputPath);
final DirectoryStagingCommitter committer = newJobCommitter();
// this should fail
@ -77,14 +77,14 @@ protected void verifyFailureConflictOutcome() throws Exception {
committer.commitJob(getJob());
reset(mockS3);
pathDoesNotExist(mockS3, OUTPUT_PATH);
pathDoesNotExist(mockS3, outputPath);
committer.setupJob(getJob());
verifyExistenceChecked(mockS3, OUTPUT_PATH);
verifyExistenceChecked(mockS3, outputPath);
verifyNoMoreInteractions(mockS3);
reset(mockS3);
pathDoesNotExist(mockS3, OUTPUT_PATH);
pathDoesNotExist(mockS3, outputPath);
committer.commitJob(getJob());
verifyCompletion(mockS3);
}
@ -93,7 +93,7 @@ protected void verifyFailureConflictOutcome() throws Exception {
public void testAppendConflictResolution() throws Exception {
FileSystem mockS3 = getMockS3A();
pathExists(mockS3, OUTPUT_PATH);
pathExists(mockS3, outputPath);
getJob().getConfiguration().set(
FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND);
@ -104,7 +104,7 @@ public void testAppendConflictResolution() throws Exception {
verifyNoMoreInteractions(mockS3);
Mockito.reset(mockS3);
pathExists(mockS3, OUTPUT_PATH);
pathExists(mockS3, outputPath);
committer.commitJob(getJob());
verifyCompletion(mockS3);
@ -114,7 +114,7 @@ public void testAppendConflictResolution() throws Exception {
public void testReplaceConflictResolution() throws Exception {
FileSystem mockS3 = getMockS3A();
pathExists(mockS3, OUTPUT_PATH);
pathExists(mockS3, outputPath);
getJob().getConfiguration().set(
FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_REPLACE);
@ -125,11 +125,11 @@ public void testReplaceConflictResolution() throws Exception {
verifyNoMoreInteractions(mockS3);
Mockito.reset(mockS3);
pathExists(mockS3, OUTPUT_PATH);
canDelete(mockS3, OUTPUT_PATH);
pathExists(mockS3, outputPath);
canDelete(mockS3, outputPath);
committer.commitJob(getJob());
verifyDeleted(mockS3, OUTPUT_PATH);
verifyDeleted(mockS3, outputPath);
verifyCompletion(mockS3);
}

View File

@ -51,13 +51,13 @@ public class TestStagingPartitionedFileListing
@Override
PartitionedStagingCommitter newJobCommitter() throws IOException {
return new PartitionedStagingCommitter(OUTPUT_PATH,
return new PartitionedStagingCommitter(outputPath,
createTaskAttemptForJob());
}
@Override
PartitionedStagingCommitter newTaskCommitter() throws IOException {
return new PartitionedStagingCommitter(OUTPUT_PATH, getTAC());
return new PartitionedStagingCommitter(outputPath, getTAC());
}
private FileSystem attemptFS;

View File

@ -66,7 +66,7 @@ private static final class PartitionedStagingCommitterForTesting
private PartitionedStagingCommitterForTesting(TaskAttemptContext context)
throws IOException {
super(OUTPUT_PATH, context);
super(StagingTestBase.outputPath, context);
}
@Override
@ -219,7 +219,7 @@ public void testReplaceWithDeleteFailure() throws Exception {
pathsExist(mockS3, "dateint=20161116/hour=14");
when(mockS3
.delete(
new Path(OUTPUT_PATH, "dateint=20161116/hour=14"),
new Path(outputPath, "dateint=20161116/hour=14"),
true))
.thenThrow(new PathCommitException("fake",
"Fake IOException for delete"));

View File

@ -47,13 +47,13 @@ public class TestStagingPartitionedTaskCommit
@Override
PartitionedStagingCommitter newJobCommitter() throws IOException {
return new PartitionedStagingCommitter(OUTPUT_PATH,
return new PartitionedStagingCommitter(outputPath,
createTaskAttemptForJob());
}
@Override
PartitionedStagingCommitter newTaskCommitter() throws Exception {
return new PartitionedStagingCommitter(OUTPUT_PATH, getTAC());
return new PartitionedStagingCommitter(outputPath, getTAC());
}
// The set of files used by this test
@ -94,12 +94,17 @@ public void testDefault() throws Exception {
// test failure when one partition already exists
reset(mockS3);
pathExists(mockS3, new Path(OUTPUT_PATH, relativeFiles.get(0)).getParent());
Path exists = new Path(outputPath, relativeFiles.get(0)).getParent();
pathExists(mockS3, exists);
intercept(PathExistsException.class,
InternalCommitterConstants.E_DEST_EXISTS,
"Expected a PathExistsException as a partition already exists",
() -> committer.commitTask(getTAC()));
"Expected a PathExistsException as a partition"
+ " already exists:" + exists,
() -> {
committer.commitTask(getTAC());
mockS3.getFileStatus(exists);
});
// test success
reset(mockS3);
@ -134,10 +139,11 @@ public void testFail() throws Exception {
// test failure when one partition already exists
reset(mockS3);
pathExists(mockS3, new Path(OUTPUT_PATH, relativeFiles.get(1)).getParent());
Path existsPath = new Path(outputPath, relativeFiles.get(1)).getParent();
pathExists(mockS3, existsPath);
intercept(PathExistsException.class, "",
"Should complain because a partition already exists",
"Should complain because a partition already exists: " + existsPath,
() -> committer.commitTask(getTAC()));
// test success
@ -173,7 +179,7 @@ public void testAppend() throws Exception {
// test success when one partition already exists
reset(mockS3);
pathExists(mockS3, new Path(OUTPUT_PATH, relativeFiles.get(2)).getParent());
pathExists(mockS3, new Path(outputPath, relativeFiles.get(2)).getParent());
committer.commitTask(getTAC());
Set<String> files = Sets.newHashSet();
@ -207,7 +213,7 @@ public void testReplace() throws Exception {
// test success when one partition already exists
reset(mockS3);
pathExists(mockS3, new Path(OUTPUT_PATH, relativeFiles.get(3)).getParent());
pathExists(mockS3, new Path(outputPath, relativeFiles.get(3)).getParent());
committer.commitTask(getTAC());
Set<String> files = Sets.newHashSet();

View File

@ -14,11 +14,18 @@
package org.apache.hadoop.fs.s3a.fileContext;
import java.net.URI;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FCStatisticsBaseTest;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -28,20 +35,25 @@
*/
public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest {
private static final Logger LOG =
LoggerFactory.getLogger(STSClientFactory.class);
private Path testRootPath;
@Before
public void setUp() throws Exception {
Configuration conf = new Configuration();
fc = S3ATestUtils.createTestFileContext(conf);
fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"),
testRootPath = fileContextTestHelper.getTestRootPath(fc, "test");
fc.mkdir(testRootPath,
FileContext.DEFAULT_PERM, true);
FileContext.clearStatistics();
}
@After
public void tearDown() throws Exception {
if (fc != null) {
fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
}
S3ATestUtils.callQuietly(LOG,
() -> fc != null && fc.delete(testRootPath, true));
}
@Override

View File

@ -0,0 +1,192 @@
/*
* 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.scale;
import org.apache.hadoop.fs.contract.ContractTestUtils;
/**
* Collect statistics from duration data from
* {@link ContractTestUtils.NanoTimer} values.
*
* The mean and standard deviation is built up as the stats are collected,
* using "Welford's Online algorithm" for the variance.
* Trends in statistics (e.g. slowing down) are not tracked.
* Not synchronized.
*/
public class NanoTimerStats {
private static final double ONE_NS = 1.0e9;
private final String operation;
private int count;
private double sum;
private double min;
private double max;
private double mean;
private double m2;
/**
* Construct statistics for a given operation.
* @param operation operation
*/
public NanoTimerStats(String operation) {
this.operation = operation;
reset();
}
/**
* construct from another stats entry;
* all value are copied.
* @param that the source statistics
*/
public NanoTimerStats(NanoTimerStats that) {
operation = that.operation;
count = that.count;
sum = that.sum;
min = that.min;
max = that.max;
mean = that.mean;
m2 = that.m2;
}
/**
* Add a duration.
* @param duration the new duration
*/
public void add(ContractTestUtils.NanoTimer duration) {
add(duration.elapsedTime());
}
/**
* Add a number.
* @param x the number
*/
public void add(long x) {
count++;
sum += x;
double delta = x - mean;
mean += delta / count;
double delta2 = x - mean;
m2 += delta * delta2;
if (min < 0 || x < min) {
min = x;
}
if (x > max) {
max = x;
}
}
/**
* Reset the data.
*/
public void reset() {
count = 0;
sum = 0;
sum = 0;
min = -1;
max = 0;
mean = 0;
m2 = 0;
}
/**
* Get the number of entries sampled.
* @return the number of durations added
*/
public int getCount() {
return count;
}
/**
* Get the sum of all durations.
* @return all the durations
*/
public double getSum() {
return sum;
}
/**
* Get the arithmetic mean of the aggregate statistics.
* @return the arithmetic mean
*/
public double getArithmeticMean() {
return mean;
}
/**
* Variance, {@code sigma^2}.
* @return variance, or, if no samples are there, 0.
*/
public double getVariance() {
return count > 0 ? (m2 / (count - 1)) :
Double.NaN;
}
/**
* Get the std deviation, sigma.
* @return the stddev, 0 may mean there are no samples.
*/
public double getDeviation() {
double variance = getVariance();
return (!Double.isNaN(variance) && variance > 0) ? Math.sqrt(variance) : 0;
}
private double toSeconds(double nano) {
return nano / ONE_NS;
}
/**
* Covert to a useful string.
* @return a human readable summary
*/
@Override
public String toString() {
return String.format(
"%s count=%d total=%.3fs mean=%.3fs stddev=%.3fs min=%.3fs max=%.3fs",
operation,
count,
toSeconds(sum),
toSeconds(mean),
getDeviation() / ONE_NS,
toSeconds(min),
toSeconds(max));
}
public String getOperation() {
return operation;
}
public double getMin() {
return min;
}
public double getMax() {
return max;
}
public double getMean() {
return mean;
}
}

View File

@ -27,35 +27,50 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.MiniYARNCluster;
import org.junit.Test;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_NAME;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS;
/**
* Tests that S3A is usable through a YARN application.
*/
public class ITestS3AMiniYarnCluster extends AbstractS3ATestBase {
private final Configuration conf = new YarnConfiguration();
private S3AFileSystem fs;
private MiniYARNCluster yarnCluster;
private Path rootPath;
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
// and set up commit code
conf.setBoolean(FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES,
false);
conf.set(FS_S3A_COMMITTER_NAME, StagingCommitter.NAME);
return conf;
}
@Override
public void setup() throws Exception {
super.setup();
fs = S3ATestUtils.createTestFileSystem(conf);
S3AFileSystem fs = getFileSystem();
Configuration conf = getConfiguration();
rootPath = path("MiniClusterWordCount");
Path workingDir = path("working");
fs.setWorkingDirectory(workingDir);
@ -79,6 +94,9 @@ public void teardown() throws Exception {
@Test
public void testWithMiniCluster() throws Exception {
S3AFileSystem fs = getFileSystem();
Configuration conf = getConfiguration();
Path input = new Path(rootPath, "input/in.txt");
input = input.makeQualified(fs.getUri(), fs.getWorkingDirectory());
Path output = new Path(rootPath, "output/");
@ -99,7 +117,13 @@ public void testWithMiniCluster() throws Exception {
int exitCode = (job.waitForCompletion(true) ? 0 : 1);
assertEquals("Returned error code.", 0, exitCode);
assertTrue(fs.exists(new Path(output, "_SUCCESS")));
Path success = new Path(output, _SUCCESS);
FileStatus status = fs.getFileStatus(success);
assertTrue("0 byte success file - not a s3guard committer " + success,
status.getLen() > 0);
SuccessData successData = SuccessData.load(fs, success);
String commitDetails = successData.toString();
LOG.info("Committer details \n{}", commitDetails);
String outputAsStr = readStringFromFile(new Path(output, "part-r-00000"));
Map<String, Integer> resAsMap = getResultAsMap(outputAsStr);
@ -130,6 +154,8 @@ private Map<String, Integer> getResultAsMap(String outputAsStr)
* helper method.
*/
private void writeStringToFile(Path path, String string) throws IOException {
Configuration conf = getConfiguration();
FileContext fc = S3ATestUtils.createTestFileContext(conf);
try (FSDataOutputStream file = fc.create(path,
EnumSet.of(CreateFlag.CREATE))) {
@ -141,6 +167,8 @@ private void writeStringToFile(Path path, String string) throws IOException {
* helper method.
*/
private String readStringFromFile(Path path) throws IOException {
S3AFileSystem fs = getFileSystem();
try (FSDataInputStream in = fs.open(path)) {
long bytesLen = fs.getFileStatus(path).getLen();
byte[] buffer = new byte[(int) bytesLen];

View File

@ -0,0 +1,115 @@
/*
* 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.mapreduce;
import java.io.IOException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.authorize.AccessControlList;
import static org.mockito.Matchers.*;
import static org.mockito.Mockito.*;
/**
* This is a mock job which doesn't talk to YARN.
* It's in this package as the JobSubmitter API is package-scoped.
*/
public class MockJob extends Job {
private static final Logger LOG =
LoggerFactory.getLogger(MockJob.class);
public static final String NAME = "mock";
private final ClientProtocol mockClient;
private static int jobIdCounter;
private static String trackerId = Long.toString(System.currentTimeMillis());
private Credentials submittedCredentials;
public MockJob(final JobConf conf, final String jobName)
throws IOException, InterruptedException {
super(conf);
setJobName(jobName);
mockClient = mock(ClientProtocol.class);
init();
}
public void init() throws IOException, InterruptedException {
when(mockClient.submitJob(any(JobID.class),
any(String.class),
any(Credentials.class)))
.thenAnswer(invocation -> {
final Object[] args = invocation.getArguments();
String name = (String) args[1];
LOG.info("Submitted Job {}", name);
submittedCredentials = (Credentials) args[2];
final JobStatus status = new JobStatus();
status.setState(JobStatus.State.RUNNING);
status.setSchedulingInfo(NAME);
status.setTrackingUrl("http://localhost:8080/");
return status;
});
when(mockClient.getNewJobID())
.thenReturn(
new JobID(trackerId, jobIdCounter++));
when(mockClient.getQueueAdmins(any(String.class)))
.thenReturn(
new AccessControlList(AccessControlList.WILDCARD_ACL_VALUE));
}
@Override
public boolean isSuccessful() throws IOException {
return true;
}
/** Only for mocking via unit tests. */
@InterfaceAudience.Private
JobSubmitter getJobSubmitter(FileSystem fs,
ClientProtocol submitClient) throws IOException {
return new JobSubmitter(fs, mockClient);
}
@Override
synchronized void connect()
throws IOException, InterruptedException, ClassNotFoundException {
super.connect();
}
public Credentials getSubmittedCredentials() {
return submittedCredentials;
}
@Override
synchronized void updateStatus() throws IOException {
// no-op
}
}